NIFI-40: merged changes from NIFI-250 branch and develop
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java b/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
index f4bea5e..e2183f4 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java
@@ -23,7 +23,7 @@
import java.util.Map;
public abstract class AbstractConfigurableComponent implements ConfigurableComponent {
-
+
/**
* Allows subclasses to perform their own validation on the already set
* properties. Since each property is validated as it is set this allows
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java
index 4b1b1a0..4e83e30 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java
@@ -98,6 +98,14 @@
* @return
*/
Map<String, String> getAttributes();
+
+ /**
+ * Returns the value of FlowFile attribute with the given name, or <code>null</code>
+ * if no attribute exists with that name
+ *
+ * @return
+ */
+ String getAttribute(String attributeName);
/**
* Returns all FlowFile attributes that existed on the FlowFile before this
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
index 39c829e..b969038 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
@@ -16,11 +16,14 @@
*/
package org.apache.nifi.provenance;
+import java.io.Closeable;
import java.io.IOException;
+import java.util.Collection;
import java.util.List;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
import org.apache.nifi.provenance.search.Query;
import org.apache.nifi.provenance.search.QuerySubmission;
import org.apache.nifi.provenance.search.SearchableField;
@@ -31,7 +34,7 @@
* has stored, and providing query capabilities against the events.
*
*/
-public interface ProvenanceEventRepository {
+public interface ProvenanceEventRepository extends Closeable {
/**
* Performs any initialization needed. This should be called only by the
@@ -56,7 +59,7 @@
*
* @param event
*/
- void registerEvent(ProvenanceEventRecord event);
+ void registerEvent(ProvenanceEventRecord event) throws IOException;
/**
* Adds the given events to the repository.
@@ -68,7 +71,7 @@
*
* @param events
*/
- void registerEvents(Iterable<ProvenanceEventRecord> events);
+ void registerEvents(Collection<ProvenanceEventRecord> events) throws IOException;
/**
* Returns a List of all <code>ProvenanceEventRecord</code>s in the
@@ -80,16 +83,47 @@
* @return
* @throws java.io.IOException
*/
- List<ProvenanceEventRecord> getEvents(long firstRecordId, final int maxRecords) throws IOException;
+ List<StoredProvenanceEvent> getEvents(long firstRecordId, final int maxRecords) throws IOException;
/**
+ * Returns a List of all <code>ProvenanceEventRecord</code>s in the repository whose locations
+ * match those specified. If any event cannot be found, it will be skipped.
+ *
+ * @param storageLocations
+ * @return
+ * @throws IOException
+ */
+ List<StoredProvenanceEvent> getEvents(List<StorageLocation> storageLocations) throws IOException;
+
+
+ /**
* Returns the largest ID of any event that is queryable in the repository.
* If no queryable events exists, returns null
*
* @return
*/
- Long getMaxEventId();
+ Long getMaxEventId() throws IOException;
+
+ /**
+ * Submits an asynchronous request to process the given Provenance Query Language query,
+ * returning an identifier that can be used to fetch the results at a later time
+ *
+ * @param query
+ * @return
+ */
+ ProvenanceQuerySubmission submitQuery(String query);
+
+ /**
+ * Returns the ProvenanceQuerySubmission associated with the given identifier, or <code>null</code>
+ * if no query exists with the given identifier.
+ *
+ * @param queryIdentifier
+ *
+ * @return
+ */
+ ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(String queryIdentifier);
+
/**
* Submits an asynchronous request to process the given query, returning an
* identifier that can be used to fetch the results at a later time
@@ -98,11 +132,11 @@
* @return
*/
QuerySubmission submitQuery(Query query);
-
+
+
/**
- * Returns the QueryResult associated with the given identifier, if the
- * query has finished processing. If the query has not yet finished running,
- * returns <code>null</code>.
+ * Returns the QuerySubmission associated with the given identifier, or <code>null</code>
+ * if no query exists with the provided identifier
*
* @param queryIdentifier
*
@@ -139,9 +173,19 @@
* @return
* @throws IOException
*/
- ProvenanceEventRecord getEvent(long id) throws IOException;
+ StoredProvenanceEvent getEvent(long id) throws IOException;
/**
+ * Returns the Provenance Event Record with the given location, if it exists, or
+ * <code>null</code> otherwise
+ *
+ * @param location
+ * @return
+ * @throws IOException
+ */
+ StoredProvenanceEvent getEvent(StorageLocation location) throws IOException;
+
+ /**
* Submits a request to expand the parents of the event with the given id
*
* @param eventId the one-up id of the Event to expand
@@ -166,15 +210,8 @@
ComputeLineageSubmission submitExpandChildren(long eventId);
/**
- * Closes the repository, freeing any resources
- *
- * @throws IOException
- */
- void close() throws IOException;
-
- /**
* Returns a list of all fields that can be searched via the
- * {@link #submitQuery(nifi.provenance.search.Query)} method
+ * {@link #submitQuery(Query.provenance.search.Query)} method
*
* @return
*/
@@ -182,9 +219,16 @@
/**
* Returns a list of all FlowFile attributes that can be searched via the
- * {@link #submitQuery(nifi.provenance.search.Query)} method
+ * {@link #submitQuery(Query.provenance.search.Query)} method
*
* @return
*/
List<SearchableField> getSearchableAttributes();
+
+ /**
+ * Returns the timestamp of the earliest event that is available in the repository, or <code>null</code> if no
+ * events exist in the repository
+ * @return
+ */
+ Long getEarliestEventTime() throws IOException;
}
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StorageLocation.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StorageLocation.java
new file mode 100644
index 0000000..ad45065
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StorageLocation.java
@@ -0,0 +1,25 @@
+/*
+ * 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.provenance;
+
+/**
+ * Marker interface that a {@link ProvenanceEventRepository} provides when it stores a provenance event.
+ * This storage location can then be used to retrieve the record at a later date.
+ */
+public interface StorageLocation {
+
+}
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StoredProvenanceEvent.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StoredProvenanceEvent.java
new file mode 100644
index 0000000..67f9d9f
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StoredProvenanceEvent.java
@@ -0,0 +1,31 @@
+/*
+ * 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.provenance;
+
+
+/**
+ * Represents a Provenance event that has been stored in a Provenance Event Repository.
+ */
+public interface StoredProvenanceEvent extends ProvenanceEventRecord {
+
+ /**
+ * Returns the location at which this Provenance Event has been stored.
+ * @return
+ */
+ StorageLocation getStorageLocation();
+
+}
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceQueryResult.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceQueryResult.java
new file mode 100644
index 0000000..cbe2705
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceQueryResult.java
@@ -0,0 +1,60 @@
+/*
+ * 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.provenance.query;
+
+import java.util.Date;
+
+public interface ProvenanceQueryResult {
+ /**
+ * Returns the Provenance events that match the query (up to the limit
+ * specified in the query)
+ *
+ * @return
+ */
+ ProvenanceResultSet getResultSet();
+
+
+ /**
+ * Returns the date at which this QueryResult will expire
+ *
+ * @return
+ */
+ Date getExpiration();
+
+ /**
+ * If an error occurred while computing the lineage, this will return the
+ * serialized error; otherwise, returns <code>null</code>.
+ *
+ * @return
+ */
+ String getError();
+
+ /**
+ * returns an integer between 0 and 100 (inclusive) that indicates what
+ * percentage of completion the query has reached
+ *
+ * @return
+ */
+ int getPercentComplete();
+
+ /**
+ * Indicates whether or not the query has finished running
+ *
+ * @return
+ */
+ boolean isFinished();
+}
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceQuerySubmission.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceQuerySubmission.java
new file mode 100644
index 0000000..e4a9edc
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceQuerySubmission.java
@@ -0,0 +1,64 @@
+/*
+ * 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.provenance.query;
+
+import java.util.Date;
+
+import org.apache.nifi.provenance.search.QueryResult;
+
+public interface ProvenanceQuerySubmission {
+
+ /**
+ * Returns the query that was submitted
+ * @return
+ */
+ String getQuery();
+
+ /**
+ * Returns the {@link QueryResult} for this query. Note that the result is
+ * only a partial result if the result of calling
+ * {@link QueryResult#isFinished()} is <code>false</code>.
+ *
+ * @return
+ */
+ ProvenanceQueryResult getResult();
+
+ /**
+ * Returns the date at which this query was submitted
+ *
+ * @return
+ */
+ Date getSubmissionTime();
+
+ /**
+ * Returns the generated identifier for this query result
+ *
+ * @return
+ */
+ String getQueryIdentifier();
+
+ /**
+ * Cancels the query
+ */
+ void cancel();
+
+ /**
+ * @return <code>true</code> if {@link #cancel()} has been called,
+ * <code>false</code> otherwise
+ */
+ boolean isCanceled();
+}
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceResultSet.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceResultSet.java
new file mode 100644
index 0000000..37c3d90
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/query/ProvenanceResultSet.java
@@ -0,0 +1,36 @@
+package org.apache.nifi.provenance.query;
+
+import java.util.List;
+
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+
+
+/**
+ * Represents a set of results from issuing a query against a {@link ProvenanceEventRepository}.
+ */
+public interface ProvenanceResultSet {
+ /**
+ * Returns the labels for the columns (aka column headers)
+ * @return
+ */
+ List<String> getLabels();
+
+ /**
+ * Returns the types of the columns returned for each row
+ * @return
+ */
+ List<Class<?>> getReturnType();
+
+ /**
+ * Indicates whether or not another result exists in the result set.
+ * @return
+ */
+ boolean hasNext();
+
+ /**
+ * Returns the next result for this query
+ * @return
+ */
+ List<?> next();
+
+}
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
index 3dd0b71..5561014 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
@@ -19,7 +19,7 @@
import java.util.Date;
import java.util.List;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
public interface QueryResult {
@@ -29,7 +29,7 @@
*
* @return
*/
- List<ProvenanceEventRecord> getMatchingEvents();
+ List<StoredProvenanceEvent> getMatchingEvents();
/**
* Returns the total number of Provenance Events that hit
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java
index efcf2a3..7d41813 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java
@@ -17,7 +17,10 @@
package org.apache.nifi.reporting;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.components.AbstractConfigurableComponent;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.logging.ComponentLog;
@@ -25,6 +28,8 @@
public abstract class AbstractReportingTask extends AbstractConfigurableComponent implements ReportingTask {
+ private final AtomicBoolean scheduled = new AtomicBoolean(false);
+
private String identifier;
private String name;
private long schedulingNanos;
@@ -42,6 +47,20 @@
init(config);
}
+ @OnUnscheduled
+ public final void setUnscheduledFlag() {
+ scheduled.set(false);
+ }
+
+ @OnScheduled
+ public final void setScheduledFlag() {
+ scheduled.set(true);
+ }
+
+ protected final boolean isScheduled() {
+ return scheduled.get();
+ }
+
/**
* Returns the {@link ControllerServiceLookup} that was passed to the
* {@link #init(ProcessorInitializationContext)} method
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/EventIdLocation.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/EventIdLocation.java
new file mode 100644
index 0000000..9cc6c4d
--- /dev/null
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/EventIdLocation.java
@@ -0,0 +1,31 @@
+/*
+ * 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.provenance;
+
+import org.apache.nifi.provenance.StorageLocation;
+
+public class EventIdLocation implements StorageLocation {
+ private final long id;
+
+ public EventIdLocation(final long id) {
+ this.id = id;
+ }
+
+ public long getId() {
+ return id;
+ }
+}
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/IdEnrichedProvenanceEvent.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/IdEnrichedProvenanceEvent.java
new file mode 100644
index 0000000..85ce91c
--- /dev/null
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/IdEnrichedProvenanceEvent.java
@@ -0,0 +1,179 @@
+/*
+ * 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.provenance;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+
+public class IdEnrichedProvenanceEvent implements StoredProvenanceEvent {
+
+ private final ProvenanceEventRecord event;
+ private final long id;
+
+ public IdEnrichedProvenanceEvent(final ProvenanceEventRecord event) {
+ this(event, event.getEventId());
+ }
+
+ public IdEnrichedProvenanceEvent(final ProvenanceEventRecord event, final long id) {
+ this.event = event;
+ this.id = id;
+ }
+
+ @Override
+ public StorageLocation getStorageLocation() {
+ return new EventIdLocation(id);
+ }
+
+ public long getEventId() {
+ return id;
+ }
+
+ public long getEventTime() {
+ return event.getEventTime();
+ }
+
+ public long getFlowFileEntryDate() {
+ return event.getFlowFileEntryDate();
+ }
+
+ public long getLineageStartDate() {
+ return event.getLineageStartDate();
+ }
+
+ public Set<String> getLineageIdentifiers() {
+ return event.getLineageIdentifiers();
+ }
+
+ public long getFileSize() {
+ return event.getFileSize();
+ }
+
+ public Long getPreviousFileSize() {
+ return event.getPreviousFileSize();
+ }
+
+ public long getEventDuration() {
+ return event.getEventDuration();
+ }
+
+ public ProvenanceEventType getEventType() {
+ return event.getEventType();
+ }
+
+ public String getAttribute(final String attributeName) {
+ return event.getAttribute(attributeName);
+ }
+
+ public Map<String, String> getAttributes() {
+ return event.getAttributes();
+ }
+
+ public Map<String, String> getPreviousAttributes() {
+ return event.getPreviousAttributes();
+ }
+
+ public Map<String, String> getUpdatedAttributes() {
+ return event.getUpdatedAttributes();
+ }
+
+ public String getComponentId() {
+ return event.getComponentId();
+ }
+
+ public String getComponentType() {
+ return event.getComponentType();
+ }
+
+ public String getTransitUri() {
+ return event.getTransitUri();
+ }
+
+ public String getSourceSystemFlowFileIdentifier() {
+ return event.getSourceSystemFlowFileIdentifier();
+ }
+
+ public String getFlowFileUuid() {
+ return event.getFlowFileUuid();
+ }
+
+ public List<String> getParentUuids() {
+ return event.getParentUuids();
+ }
+
+ public List<String> getChildUuids() {
+ return event.getChildUuids();
+ }
+
+ public String getAlternateIdentifierUri() {
+ return event.getAlternateIdentifierUri();
+ }
+
+ public String getDetails() {
+ return event.getDetails();
+ }
+
+ public String getRelationship() {
+ return event.getRelationship();
+ }
+
+ public String getSourceQueueIdentifier() {
+ return event.getSourceQueueIdentifier();
+ }
+
+ public String getContentClaimSection() {
+ return event.getContentClaimSection();
+ }
+
+ public String getPreviousContentClaimSection() {
+ return event.getPreviousContentClaimSection();
+ }
+
+ public String getContentClaimContainer() {
+ return event.getContentClaimContainer();
+ }
+
+ public String getPreviousContentClaimContainer() {
+ return event.getPreviousContentClaimContainer();
+ }
+
+ public String getContentClaimIdentifier() {
+ return event.getContentClaimIdentifier();
+ }
+
+ public String getPreviousContentClaimIdentifier() {
+ return event.getPreviousContentClaimIdentifier();
+ }
+
+ public Long getContentClaimOffset() {
+ return event.getContentClaimOffset();
+ }
+
+ public Long getPreviousContentClaimOffset() {
+ return event.getPreviousContentClaimOffset();
+ }
+
+ @Override
+ public String toString() {
+ return event.toString();
+ }
+}
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
index dc2903f..38d65b5 100644
--- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
@@ -32,11 +32,11 @@
private final String friendlyName;
private final boolean attribute;
- NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute) {
+ public NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute) {
this(identifier, searchableName, friendlyName, attribute, SearchableFieldType.STRING);
}
- NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute, final SearchableFieldType fieldType) {
+ public NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute, final SearchableFieldType fieldType) {
this.identifier = requireNonNull(identifier);
this.searchableName = requireNonNull(searchableName);
this.friendlyName = requireNonNull(friendlyName);
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
index afb56e8..0f454bd 100644
--- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
@@ -178,7 +178,7 @@
}
}
- public void update(final Collection<ProvenanceEventRecord> records) {
+ public void update(final Collection<StoredProvenanceEvent> records) {
writeLock.lock();
try {
relevantRecords.addAll(records);
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
index cfbae88..bd2c2ad 100644
--- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java
@@ -161,6 +161,15 @@
}
@Override
+ public String getAttribute(final String attributeName) {
+ if ( updatedAttributes.containsKey(attributeName) ) {
+ return updatedAttributes.get(attributeName);
+ }
+
+ return previousAttributes.get(attributeName);
+ }
+
+ @Override
public Map<String, String> getAttributes() {
final Map<String, String> allAttrs = new HashMap<>(previousAttributes.size() + updatedAttributes.size());
allAttrs.putAll(previousAttributes);
@@ -424,7 +433,6 @@
private String uuid = null;
private List<String> parentUuids = null;
private List<String> childrenUuids = null;
- private String contentType = null;
private String alternateIdentifierUri = null;
private String details = null;
private String relationship = null;
@@ -631,11 +639,6 @@
return this;
}
- public Builder setContentType(String contentType) {
- this.contentType = contentType;
- return this;
- }
-
@Override
public Builder setAlternateIdentifierUri(String alternateIdentifierUri) {
this.alternateIdentifierUri = alternateIdentifierUri;
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
index 9a9a27d..7e75357 100644
--- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
@@ -40,7 +40,7 @@
private final Lock writeLock = rwLock.writeLock();
// guarded by writeLock
- private final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>();
+ private final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>();
private long totalHitCount;
private int numCompletedSteps = 0;
private Date expirationDate;
@@ -58,14 +58,14 @@
}
@Override
- public List<ProvenanceEventRecord> getMatchingEvents() {
+ public List<StoredProvenanceEvent> getMatchingEvents() {
readLock.lock();
try {
if (matchingRecords.size() <= query.getMaxResults()) {
return new ArrayList<>(matchingRecords);
}
- final List<ProvenanceEventRecord> copy = new ArrayList<>(query.getMaxResults());
+ final List<StoredProvenanceEvent> copy = new ArrayList<>(query.getMaxResults());
for (int i = 0; i < query.getMaxResults(); i++) {
copy.add(matchingRecords.get(i));
}
@@ -141,7 +141,7 @@
}
}
- public void update(final Collection<ProvenanceEventRecord> matchingRecords, final long totalHits) {
+ public void update(final Collection<StoredProvenanceEvent> matchingRecords, final long totalHits) {
writeLock.lock();
try {
this.matchingRecords.addAll(matchingRecords);
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/.gitignore b/nifi/nifi-commons/nifi-provenance-query-language/.gitignore
new file mode 100644
index 0000000..b83d222
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/.gitignore
@@ -0,0 +1 @@
+/target/
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/pom.xml b/nifi/nifi-commons/nifi-provenance-query-language/pom.xml
new file mode 100644
index 0000000..905ed5b
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/pom.xml
@@ -0,0 +1,68 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-commons</artifactId>
+ <version>0.0.2-incubating-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>nifi-provenance-query-language</artifactId>
+ <packaging>jar</packaging>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.antlr</groupId>
+ <artifactId>antlr3-maven-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>antlr</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-data-provenance-utils</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>org.antlr</groupId>
+ <artifactId>antlr-runtime</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.lucene</groupId>
+ <artifactId>lucene-core</artifactId>
+ </dependency>
+
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-volatile-provenance-repository</artifactId>
+ <version>0.0.2-incubating-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-properties</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/antlr3/org/apache/nifi/pql/ProvenanceQueryLexer.g b/nifi/nifi-commons/nifi-provenance-query-language/src/main/antlr3/org/apache/nifi/pql/ProvenanceQueryLexer.g
new file mode 100644
index 0000000..0815475
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/antlr3/org/apache/nifi/pql/ProvenanceQueryLexer.g
@@ -0,0 +1,193 @@
+lexer grammar ProvenanceQueryLexer;
+
+@header {
+ package org.apache.nifi.pql;
+ import org.apache.nifi.pql.exception.ProvenanceQueryLanguageParsingException;
+}
+
+@rulecatch {
+ catch(final Exception e) {
+ throw new ProvenanceQueryLanguageParsingException(e);
+ }
+}
+
+@members {
+ public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
+ final StringBuilder sb = new StringBuilder();
+ if ( e.token == null ) {
+ sb.append("Unrecognized token ");
+ } else {
+ sb.append("Unexpected token '").append(e.token.getText()).append("' ");
+ }
+ sb.append("at line ").append(e.line);
+ if ( e.approximateLineInfo ) {
+ sb.append(" (approximately)");
+ }
+ sb.append(", column ").append(e.charPositionInLine);
+ sb.append(". Query: ").append(e.input.toString());
+
+ throw new ProvenanceQueryLanguageParsingException(sb.toString());
+ }
+
+ public void recover(RecognitionException e) {
+ final StringBuilder sb = new StringBuilder();
+ if ( e.token == null ) {
+ sb.append("Unrecognized token ");
+ } else {
+ sb.append("Unexpected token '").append(e.token.getText()).append("' ");
+ }
+ sb.append("at line ").append(e.line);
+ if ( e.approximateLineInfo ) {
+ sb.append(" (approximately)");
+ }
+ sb.append(", column ").append(e.charPositionInLine);
+ sb.append(". Query: ").append(e.input.toString());
+
+ throw new ProvenanceQueryLanguageParsingException(sb.toString());
+ }
+}
+
+
+// PUNCTUATION & SPECIAL CHARACTERS
+WHITESPACE : (' '|'\t'|'\n'|'\r')+ { $channel = HIDDEN; };
+COMMENT : '#' ( ~('\n') )* '\n' { $channel = HIDDEN; };
+
+LPAREN : '(';
+RPAREN : ')';
+LBRACKET : '[';
+RBRACKET : ']';
+COMMA : ',';
+DOT : '.';
+SEMICOLON : ';';
+ASTERISK : '*';
+EQUALS : '=';
+NOT_EQUALS : '!=' | '<>';
+GT : '>';
+LT : '<';
+GE : '>=';
+LE : '<=';
+PIPE : '|';
+NUMBER : ('0'..'9')+;
+
+
+// Keywords
+SELECT : 'SELECT' | 'select' | 'Select';
+AS : 'AS' | 'as' | 'As';
+FROM : 'FROM' | 'from' | 'From';
+WHERE : 'WHERE' | 'where' | 'Where';
+HAVING : 'HAVING' | 'having' | 'Having';
+ORDER_BY : 'ORDER BY' | 'order by' | 'Order By';
+ASC : 'ASC' | 'asc' | 'Asc';
+DESC : 'DESC' | 'desc' | 'Desc';
+GROUP_BY : 'GROUP BY' | 'group by' | 'Group By';
+EVENT : 'EVENT' | 'event' | 'Event';
+RELATIONSHIP : 'RELATIONSHIP' | 'relationship' | 'Relationship';
+
+
+// Operators
+WITHIN : 'WITHIN' | 'within' | 'Within';
+MATCHES : 'MATCHES' | 'matches' | 'Matches';
+CONTAINS : 'CONTAINS' | 'contains' | 'Contains';
+IS_NULL : 'IS NULL' | 'is null' | 'Is Null';
+NOT_NULL : 'NOT NULL' | 'not null' | 'Not Null';
+IN : 'IN' | 'in' | 'In';
+BETWEEN : 'BETWEEN' | 'between' | 'Between';
+AND : 'AND' | 'and' | 'And';
+OR : 'OR' | 'or' | 'Or';
+NOT : 'NOT' | 'not' | 'Not';
+LIMIT : 'LIMIT' | 'limit' | 'Limit';
+STARTS_WITH : 'STARTS WITH' | 'starts with' | 'Starts with' | 'Starts With';
+
+// Functions
+COUNT : 'COUNT' | 'count' | 'Count';
+SUM : 'SUM' | 'sum' | 'Sum';
+MIN : 'MIN' | 'min' | 'Min';
+MAX : 'MAX' | 'max' | 'Max';
+AVG : 'AVG' | 'avg' | 'Avg';
+HOUR : 'HOUR' | 'hour' | 'Hour';
+MINUTE : 'MINUTE' | 'minute' | 'Minute';
+SECOND : 'SECOND' | 'second' | 'Second';
+DAY : 'DAY' | 'day' | 'Day';
+MONTH : 'MONTH' | 'month' | 'Month';
+YEAR : 'YEAR' | 'year' | 'Year';
+
+
+// Event Properties
+TRANSIT_URI : 'TRANSITURI' | 'transituri' | 'TransitUri';
+TIMESTAMP : 'TIME' | 'time' | 'Time';
+FILESIZE : 'SIZE' | 'size' | 'Size';
+TYPE : 'TYPE' | 'type' | 'Type';
+COMPONENT_ID : 'COMPONENTID' | 'componentid' | 'ComponentId' | 'componentId' | 'componentID' | 'ComponentID';
+UUID : 'UUID' | 'uuid' | 'Uuid';
+
+// Event Types
+RECEIVE : 'RECEIVE' | 'receive' | 'Receive';
+SEND : 'SEND' | 'send' | 'Send';
+DROP : 'DROP' | 'drop' | 'Drop';
+CREATE : 'CREATE' | 'create' | 'Create';
+EXPIRE : 'EXPIRE' | 'expire' | 'Expire';
+FORK : 'FORK' | 'fork' | 'Fork';
+JOIN : 'JOIN' | 'join' | 'Join';
+CLONE : 'CLONE' | 'clone' | 'Clone';
+CONTENT_MODIFIED : 'CONTENT_MODIFIED' | 'content_modified' | 'Content_Modified';
+ATTRIBUTES_MODIFIED : 'ATTRIBUTES_MODIFIED' | 'attributes_modified' | 'Attributes_Modified';
+ROUTE : 'ROUTE' | 'route' | 'Route';
+REPLAY : 'REPLAY' | 'replay' | 'Replay';
+
+
+
+IDENTIFIER : (
+ ('a'..'z' | 'A'..'Z' | '$')
+ ('a'..'z' | 'A'..'Z' | '$' | '0'..'9' | '_' )*
+ );
+
+
+
+// STRINGS
+STRING_LITERAL
+@init{StringBuilder lBuf = new StringBuilder();}
+ :
+ (
+ '"'
+ (
+ escaped=ESC {lBuf.append(getText());} |
+ normal = ~( '"' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);}
+ )*
+ '"'
+ )
+ {
+ setText(lBuf.toString());
+ }
+ |
+ (
+ '\''
+ (
+ escaped=ESC {lBuf.append(getText());} |
+ normal = ~( '\'' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);}
+ )*
+ '\''
+ )
+ {
+ setText(lBuf.toString());
+ }
+ ;
+
+
+fragment
+ESC
+ : '\\'
+ (
+ '"' { setText("\""); }
+ | '\'' { setText("\'"); }
+ | 'r' { setText("\r"); }
+ | 'n' { setText("\n"); }
+ | 't' { setText("\t"); }
+ | '\\' { setText("\\\\"); }
+ | nextChar = ~('"' | '\'' | 'r' | 'n' | 't' | '\\')
+ {
+ StringBuilder lBuf = new StringBuilder(); lBuf.append("\\\\").appendCodePoint(nextChar); setText(lBuf.toString());
+ }
+ )
+ ;
+
+
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/antlr3/org/apache/nifi/pql/ProvenanceQueryParser.g b/nifi/nifi-commons/nifi-provenance-query-language/src/main/antlr3/org/apache/nifi/pql/ProvenanceQueryParser.g
new file mode 100644
index 0000000..25410bb
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/antlr3/org/apache/nifi/pql/ProvenanceQueryParser.g
@@ -0,0 +1,141 @@
+parser grammar ProvenanceQueryParser;
+
+options {
+ output=AST;
+ tokenVocab=ProvenanceQueryLexer;
+}
+
+tokens {
+ PQL;
+ QUERY;
+ EVENT_PROPERTY;
+ ATTRIBUTE;
+ ORDER;
+}
+
+@header {
+ package org.apache.nifi.pql;
+ import org.apache.nifi.pql.exception.ProvenanceQueryLanguageParsingException;
+}
+
+
+@members {
+ public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
+ final StringBuilder sb = new StringBuilder();
+ if ( e.token == null ) {
+ sb.append("Unrecognized token ");
+ } else {
+ sb.append("Unexpected token '").append(e.token.getText()).append("' ");
+ }
+ sb.append("at line ").append(e.line);
+ if ( e.approximateLineInfo ) {
+ sb.append(" (approximately)");
+ }
+ sb.append(", column ").append(e.charPositionInLine);
+ sb.append(". Query: ").append(e.input.toString());
+
+ throw new ProvenanceQueryLanguageParsingException(sb.toString());
+ }
+
+ public void recover(final RecognitionException e) {
+ final StringBuilder sb = new StringBuilder();
+ if ( e.token == null ) {
+ sb.append("Unrecognized token ");
+ } else {
+ sb.append("Unexpected token '").append(e.token.getText()).append("' ");
+ }
+ sb.append("at line ").append(e.line);
+ if ( e.approximateLineInfo ) {
+ sb.append(" (approximately)");
+ }
+ sb.append(", column ").append(e.charPositionInLine);
+ sb.append(". Query: ").append(e.input.toString());
+
+ throw new ProvenanceQueryLanguageParsingException(sb.toString());
+ }
+}
+
+
+
+pql : query ->
+ ^(PQL query);
+
+query : selectClause
+ fromClause?
+ whereClause?
+ groupByClause?
+// havingClause?
+ orderByClause?
+ limitClause?
+ SEMICOLON?
+ EOF ->
+ ^(QUERY selectClause fromClause? whereClause? groupByClause? orderByClause? limitClause?);
+
+
+selectClause : SELECT^ selectable (COMMA! selectable)*;
+
+
+selectable : function | (selectableSource ( (DOT! eventProperty^) | (LBRACKET! attribute^ RBRACKET!) )?);
+
+selectableSource : EVENT | IDENTIFIER;
+
+eventProperty : propertyName ->
+ ^(EVENT_PROPERTY propertyName );
+
+propertyName : UUID | TRANSIT_URI | TIMESTAMP | FILESIZE | TYPE | COMPONENT_ID | COMPONENT_TYPE | RELATIONSHIP;
+
+attribute : STRING_LITERAL ->
+ ^(ATTRIBUTE STRING_LITERAL);
+
+
+fromClause : FROM^ source (COMMA! source)*;
+
+source : RECEIVE | SEND | DROP | CREATE | EXPIRE | FORK | JOIN | CLONE | CONTENT_MODIFIED | ATTRIBUTES_MODIFIED | ROUTE | REPLAY | ASTERISK;
+
+
+
+whereClause : WHERE^ conditions;
+
+conditions : condition ((AND^ | OR^) condition)*;
+
+condition : NOT^ condition | LPAREN! conditions RPAREN! | evaluation;
+
+evaluation : expression
+ (
+ unaryOperator^
+ | (binaryOperator^ expression)
+ | (BETWEEN^ NUMBER AND! NUMBER)
+ );
+
+expression : (LPAREN! expr RPAREN!) | expr;
+
+expr : constant | ref;
+
+ref : selectableSource ( (DOT! eventProperty^) | (LBRACKET! attribute^ RBRACKET!) )?;
+
+unaryOperator : IS_NULL | NOT_NULL;
+
+binaryOperator : EQUALS | NOT_EQUALS | GT | LT | GE | LE | MATCHES | STARTS_WITH;
+
+constant : NUMBER | STRING_LITERAL;
+
+
+function : functionName^ LPAREN! ref RPAREN!;
+
+functionName : COUNT | SUM | MIN | MAX | AVG | HOUR | MINUTE | SECOND | DAY | MONTH | YEAR;
+
+
+groupByClause : GROUP_BY^ group (COMMA! group)*;
+
+group : ref^ | function^;
+
+orderByClause : ORDER_BY^ order (COMMA! order)*;
+
+order: selectable direction? ->
+ ^(ORDER selectable direction?);
+
+direction : ASC | DESC;
+
+limitClause : LIMIT^ NUMBER;
+
+
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/Formatter.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/Formatter.java
new file mode 100644
index 0000000..7090d26
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/Formatter.java
@@ -0,0 +1,5 @@
+package org.apache.nifi.pql;
+
+public interface Formatter {
+ String format(Object value);
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/LuceneTranslator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/LuceneTranslator.java
new file mode 100644
index 0000000..96b4dbf
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/LuceneTranslator.java
@@ -0,0 +1,189 @@
+package org.apache.nifi.pql;
+
+import static org.apache.nifi.pql.ProvenanceQueryParser.*;
+
+import java.text.SimpleDateFormat;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RegexpQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.EqualsEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.GreaterThanEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.LessThanEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.MatchesEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.StartsWithEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.AttributeEvaluator;
+import org.apache.nifi.pql.evaluation.literals.LongLiteralEvaluator;
+import org.apache.nifi.pql.evaluation.literals.StringLiteralEvaluator;
+import org.apache.nifi.pql.evaluation.logic.AndEvaluator;
+import org.apache.nifi.pql.evaluation.logic.OrEvaluator;
+import org.apache.nifi.provenance.SearchableFields;
+
+
+public class LuceneTranslator {
+
+ public static Query toLuceneQuery(final RecordEvaluator<Boolean> whereClause) {
+ if ( whereClause == null ) {
+ return new MatchAllDocsQuery();
+ }
+
+ final BooleanQuery query = new BooleanQuery();
+ switch (whereClause.getEvaluatorType()) {
+ case AND:
+ final AndEvaluator and = (AndEvaluator) whereClause;
+ query.add(toLuceneQuery(and.getLHS()), Occur.MUST);
+ query.add(toLuceneQuery(and.getRHS()), Occur.MUST);
+ break;
+ case OR:
+ final OrEvaluator or = (OrEvaluator) whereClause;
+ query.add(toLuceneQuery(or.getLHS()), Occur.SHOULD);
+ query.add(toLuceneQuery(or.getRHS()), Occur.SHOULD);
+ query.setMinimumNumberShouldMatch(1);
+ break;
+ case GT: {
+ final GreaterThanEvaluator gt = (GreaterThanEvaluator) whereClause;
+ final OperandEvaluator<?> lhs = gt.getLHS();
+ final OperandEvaluator<?> rhs = gt.getRHS();
+
+ final String fieldName = getFieldName(lhs);
+ if ( fieldName != null ) {
+ Long rhsValue = null;
+ if ( rhs.getEvaluatorType() == NUMBER ) {
+ rhsValue = ((LongLiteralEvaluator) rhs).evaluate(null);
+ } else if ( rhs.getEvaluatorType() == STRING_LITERAL ) {
+ final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+ try {
+ rhsValue = sdf.parse(((StringLiteralEvaluator) rhs).evaluate(null)).getTime();
+ } catch (final Exception e) {
+ }
+ }
+
+ if ( rhsValue != null ) {
+ query.add(NumericRangeQuery.newLongRange(fieldName, rhsValue, Long.MAX_VALUE, true, true), Occur.MUST);
+ }
+ }
+ }
+ break;
+ case LT: {
+ final LessThanEvaluator lt = (LessThanEvaluator) whereClause;
+ final OperandEvaluator<?> lhs = lt.getLHS();
+ final OperandEvaluator<?> rhs = lt.getRHS();
+
+ final String fieldName = getFieldName(lhs);
+ if ( fieldName != null ) {
+ Long rhsValue = null;
+ if ( rhs.getEvaluatorType() == NUMBER ) {
+ rhsValue = ((LongLiteralEvaluator) rhs).evaluate(null);
+ } else if ( rhs.getEvaluatorType() == STRING_LITERAL ) {
+ final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+ try {
+ rhsValue = sdf.parse(((StringLiteralEvaluator) rhs).evaluate(null)).getTime();
+ } catch (final Exception e) {
+ }
+ }
+
+ if ( rhsValue != null ) {
+ query.add(NumericRangeQuery.newLongRange(fieldName, Long.MIN_VALUE, rhsValue, true, true), Occur.MUST);
+ }
+ }
+ }
+ break;
+ case MATCHES: {
+ final MatchesEvaluator me = (MatchesEvaluator) whereClause;
+ final OperandEvaluator<?> lhs = me.getLHS();
+ final OperandEvaluator<?> rhs = me.getRHS();
+ addMatches(lhs, rhs, query);
+ }
+ break;
+ case STARTS_WITH: {
+ final StartsWithEvaluator startsWith = (StartsWithEvaluator) whereClause;
+ final OperandEvaluator<?> lhs = startsWith.getLHS();
+ final OperandEvaluator<?> rhs = startsWith.getRHS();
+
+ if ( rhs.getEvaluatorType() == STRING_LITERAL ) {
+ final String base = rhs.evaluate(null).toString();
+
+ final StringLiteralEvaluator regexEval = new StringLiteralEvaluator(Pattern.quote(base) + ".*");
+ addMatches(lhs, regexEval, query);
+ }
+ }
+ break;
+ case EQUALS: {
+ final EqualsEvaluator equals = (EqualsEvaluator) whereClause;
+ final OperandEvaluator<?> lhs = equals.getLHS();
+ final OperandEvaluator<?> rhs = equals.getRHS();
+
+ final String fieldName = getFieldName(lhs);
+ if ( fieldName != null && rhs.getEvaluatorType() == STRING_LITERAL ) {
+ query.add(new TermQuery(new Term(fieldName, toLower(rhs.evaluate(null).toString()))), Occur.MUST);
+ }
+ }
+ break;
+ }
+
+ return query;
+ }
+
+ private static String toLower(final String value) {
+ return value == null ? null : value.toLowerCase();
+ }
+
+ private static String getFieldName(final OperandEvaluator<?> eval) {
+ switch (eval.getEvaluatorType()) {
+ case TIMESTAMP:
+ return SearchableFields.EventTime.getSearchableFieldName();
+ case FILESIZE:
+ return SearchableFields.FileSize.getSearchableFieldName();
+ case ATTRIBUTE:
+ return ((AttributeEvaluator) eval).getAttributeNameEvaluator().evaluate(null).toLowerCase();
+ case TRANSIT_URI:
+ return SearchableFields.TransitURI.getSearchableFieldName();
+ case RELATIONSHIP:
+ return SearchableFields.Relationship.getSearchableFieldName();
+ case TYPE:
+ return SearchableFields.EventType.getSearchableFieldName();
+ case COMPONENT_ID:
+ return SearchableFields.ComponentID.getSearchableFieldName();
+ case UUID:
+ return SearchableFields.FlowFileUUID.getSearchableFieldName();
+ }
+
+ return null;
+ }
+
+ private static void addMatches(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final BooleanQuery query) {
+ String field = null;
+ switch (lhs.getEvaluatorType()) {
+ case ATTRIBUTE:
+ final AttributeEvaluator attr = (AttributeEvaluator) lhs;
+ final OperandEvaluator<?> attrEval = attr.getAttributeNameEvaluator();
+ if ( attrEval.getEvaluatorType() == STRING_LITERAL ) {
+ field = (String) attrEval.evaluate(null);
+ }
+ break;
+ case COMPONENT_ID:
+ case TRANSIT_URI:
+ case TYPE:
+ field = lhs.evaluate(null).toString();
+ break;
+ }
+
+ String regex = null;
+ if ( rhs.getEvaluatorType() == STRING_LITERAL ) {
+ regex = rhs.evaluate(null).toString();
+ }
+
+ if ( field != null && regex != null ) {
+ query.add(new RegexpQuery(new Term(field, regex)), Occur.MUST);
+ }
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/ProvenanceQuery.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/ProvenanceQuery.java
new file mode 100644
index 0000000..4dda39b
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/ProvenanceQuery.java
@@ -0,0 +1,670 @@
+package org.apache.nifi.pql;
+
+import static org.apache.nifi.pql.ProvenanceQueryParser.AND;
+import static org.apache.nifi.pql.ProvenanceQueryParser.ASC;
+import static org.apache.nifi.pql.ProvenanceQueryParser.ATTRIBUTE;
+import static org.apache.nifi.pql.ProvenanceQueryParser.AVG;
+import static org.apache.nifi.pql.ProvenanceQueryParser.COMPONENT_ID;
+import static org.apache.nifi.pql.ProvenanceQueryParser.COUNT;
+import static org.apache.nifi.pql.ProvenanceQueryParser.DAY;
+import static org.apache.nifi.pql.ProvenanceQueryParser.EQUALS;
+import static org.apache.nifi.pql.ProvenanceQueryParser.EVENT;
+import static org.apache.nifi.pql.ProvenanceQueryParser.EVENT_PROPERTY;
+import static org.apache.nifi.pql.ProvenanceQueryParser.FILESIZE;
+import static org.apache.nifi.pql.ProvenanceQueryParser.FROM;
+import static org.apache.nifi.pql.ProvenanceQueryParser.GROUP_BY;
+import static org.apache.nifi.pql.ProvenanceQueryParser.GT;
+import static org.apache.nifi.pql.ProvenanceQueryParser.HOUR;
+import static org.apache.nifi.pql.ProvenanceQueryParser.IDENTIFIER;
+import static org.apache.nifi.pql.ProvenanceQueryParser.LIMIT;
+import static org.apache.nifi.pql.ProvenanceQueryParser.LT;
+import static org.apache.nifi.pql.ProvenanceQueryParser.MATCHES;
+import static org.apache.nifi.pql.ProvenanceQueryParser.MINUTE;
+import static org.apache.nifi.pql.ProvenanceQueryParser.MONTH;
+import static org.apache.nifi.pql.ProvenanceQueryParser.NOT;
+import static org.apache.nifi.pql.ProvenanceQueryParser.NOT_EQUALS;
+import static org.apache.nifi.pql.ProvenanceQueryParser.NUMBER;
+import static org.apache.nifi.pql.ProvenanceQueryParser.OR;
+import static org.apache.nifi.pql.ProvenanceQueryParser.ORDER_BY;
+import static org.apache.nifi.pql.ProvenanceQueryParser.RELATIONSHIP;
+import static org.apache.nifi.pql.ProvenanceQueryParser.SECOND;
+import static org.apache.nifi.pql.ProvenanceQueryParser.STARTS_WITH;
+import static org.apache.nifi.pql.ProvenanceQueryParser.STRING_LITERAL;
+import static org.apache.nifi.pql.ProvenanceQueryParser.SUM;
+import static org.apache.nifi.pql.ProvenanceQueryParser.TIMESTAMP;
+import static org.apache.nifi.pql.ProvenanceQueryParser.TRANSIT_URI;
+import static org.apache.nifi.pql.ProvenanceQueryParser.TYPE;
+import static org.apache.nifi.pql.ProvenanceQueryParser.UUID;
+import static org.apache.nifi.pql.ProvenanceQueryParser.WHERE;
+import static org.apache.nifi.pql.ProvenanceQueryParser.YEAR;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.tree.Tree;
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.pql.evaluation.RepositoryEvaluator;
+import org.apache.nifi.pql.evaluation.accumulation.AverageAccumulator;
+import org.apache.nifi.pql.evaluation.accumulation.CountAccumulator;
+import org.apache.nifi.pql.evaluation.accumulation.EventAccumulator;
+import org.apache.nifi.pql.evaluation.accumulation.SumAccumulator;
+import org.apache.nifi.pql.evaluation.comparison.EqualsEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.GreaterThanEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.LessThanEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.MatchesEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.RecordTypeEvaluator;
+import org.apache.nifi.pql.evaluation.comparison.StartsWithEvaluator;
+import org.apache.nifi.pql.evaluation.conversion.StringToLongEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.AttributeEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.ComponentIdEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.RelationshipEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.SizeEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.TimestampEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.TransitUriEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.TypeEvaluator;
+import org.apache.nifi.pql.evaluation.extraction.UuidEvaluator;
+import org.apache.nifi.pql.evaluation.function.TimeFieldEvaluator;
+import org.apache.nifi.pql.evaluation.literals.LongLiteralEvaluator;
+import org.apache.nifi.pql.evaluation.literals.StringLiteralEvaluator;
+import org.apache.nifi.pql.evaluation.logic.AndEvaluator;
+import org.apache.nifi.pql.evaluation.logic.OrEvaluator;
+import org.apache.nifi.pql.evaluation.order.FieldSorter;
+import org.apache.nifi.pql.evaluation.order.GroupedSorter;
+import org.apache.nifi.pql.evaluation.order.RowSorter;
+import org.apache.nifi.pql.evaluation.order.SortDirection;
+import org.apache.nifi.pql.evaluation.repository.SelectAllRecords;
+import org.apache.nifi.pql.exception.ProvenanceQueryLanguageException;
+import org.apache.nifi.pql.exception.ProvenanceQueryLanguageParsingException;
+import org.apache.nifi.pql.results.GroupingResultSet;
+import org.apache.nifi.pql.results.StandardOrderedResultSet;
+import org.apache.nifi.pql.results.StandardUnorderedResultSet;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+import org.apache.nifi.provenance.search.SearchableField;
+
+public class ProvenanceQuery {
+ private final Tree tree;
+ private final String pql;
+ private final List<Accumulator<?>> selectAccumulators;
+ private final List<RecordEvaluator<?>> groupEvaluators;
+ private final RecordEvaluator<Boolean> sourceEvaluator;
+ private final RecordEvaluator<Boolean> conditionEvaluator;
+ private final RowSorter sorter;
+ private final Long limit;
+
+ private final Set<SearchableField> searchableFields;
+ private final Set<String> searchableAttributes;
+ private long accumulatorIdGenerator = 0L;
+
+ private final Set<String> referencedFields = new HashSet<>();
+
+ public static ProvenanceQuery compile(final String pql, final Collection<SearchableField> searchableFields, final Collection<SearchableField> searchableAttributes) {
+ try {
+ final CommonTokenStream lexerTokenStream = createTokenStream(pql);
+ final ProvenanceQueryParser parser = new ProvenanceQueryParser(lexerTokenStream);
+ final Tree ast = (Tree) parser.pql().getTree();
+ final Tree tree = ast.getChild(0);
+
+ return new ProvenanceQuery(tree, pql, searchableFields, searchableAttributes);
+ } catch (final ProvenanceQueryLanguageParsingException e) {
+ throw e;
+ } catch (final Exception e) {
+ throw new ProvenanceQueryLanguageParsingException(e);
+ }
+ }
+
+ private static CommonTokenStream createTokenStream(final String expression) throws ProvenanceQueryLanguageParsingException {
+ final CharStream input = new ANTLRStringStream(expression);
+ final ProvenanceQueryLexer lexer = new ProvenanceQueryLexer(input);
+ return new CommonTokenStream(lexer);
+ }
+
+ private ProvenanceQuery(final Tree tree, final String pql, final Collection<SearchableField> searchableFields, final Collection<SearchableField> searchableAttributes) {
+ this.tree = tree;
+ this.pql = pql;
+ this.searchableFields = searchableFields == null ? null : Collections.unmodifiableSet(new HashSet<>(searchableFields));
+ if (searchableAttributes == null) {
+ this.searchableAttributes = null;
+ } else {
+ final Set<String> attributes = new HashSet<>();
+ for ( final SearchableField attr : searchableAttributes ) {
+ attributes.add(attr.getSearchableFieldName());
+ }
+ this.searchableAttributes = Collections.unmodifiableSet(attributes);
+ }
+
+ Tree fromTree = null;
+ Tree whereTree = null;
+ Tree groupByTree = null;
+ Tree limitTree = null;
+ Tree orderByTree = null;
+
+ for (int i=1; i < tree.getChildCount(); i++) {
+ final Tree subTree = tree.getChild(i);
+ switch (subTree.getType()) {
+ case FROM:
+ fromTree = subTree;
+ break;
+ case WHERE:
+ whereTree = subTree;
+ break;
+ case GROUP_BY:
+ groupByTree = subTree;
+ break;
+ case LIMIT:
+ limitTree = subTree;
+ break;
+ case ORDER_BY:
+ orderByTree = subTree;
+ break;
+ default:
+ // TODO: Handle other types!
+ continue;
+ }
+ }
+
+ sourceEvaluator = (fromTree == null) ? null : buildSourceEvaluator(fromTree);
+
+ final BooleanEvaluator where = (whereTree == null) ? null : buildConditionEvaluator(whereTree.getChild(0), Clause.WHERE);
+ conditionEvaluator = where;
+
+ groupEvaluators = (groupByTree == null) ? null : buildGroupEvaluators(groupByTree);
+ limit = (limitTree == null) ? null : Long.parseLong(limitTree.getChild(0).getText());
+ sorter = (orderByTree == null) ? null : buildSorter(orderByTree, groupByTree != null);
+
+ boolean requiresAggregate = false;
+ if ( groupEvaluators != null && !groupEvaluators.isEmpty() ) {
+ requiresAggregate = true;
+ }
+ if ( requiresAggregate ) {
+ selectAccumulators = buildAccumulators(tree.getChild(0), true);
+ } else {
+ final List<Accumulator<?>> accumulators = buildAccumulators(tree.getChild(0), false);
+
+ for ( final Accumulator<?> accumulator : accumulators ) {
+ if ( accumulator.isAggregateFunction() ) {
+ requiresAggregate = true;
+ break;
+ }
+ }
+
+ if ( requiresAggregate ) {
+ selectAccumulators = buildAccumulators(tree.getChild(0), true);
+ } else {
+ selectAccumulators = accumulators;
+ }
+ }
+ }
+
+ public Set<String> getReferencedFields() {
+ return Collections.unmodifiableSet(referencedFields);
+ }
+
+ @Override
+ public String toString() {
+ return printTree(tree);
+ }
+
+ public String getQuery() {
+ return pql;
+
+ }
+
+ private String printTree(final Tree tree) {
+ final StringBuilder sb = new StringBuilder();
+ printTree(tree, 0, sb);
+
+ return sb.toString();
+ }
+
+ private void printTree(final Tree tree, final int spaces, final StringBuilder sb) {
+ for (int i=0; i < spaces; i++) {
+ sb.append(" ");
+ }
+
+ if ( tree.getText().trim().isEmpty() ) {
+ sb.append(tree.toString()).append("\n");
+ } else {
+ sb.append(tree.getText()).append("\n");
+ }
+
+ for (int i=0; i < tree.getChildCount(); i++) {
+ printTree(tree.getChild(i), spaces + 2, sb);
+ }
+ }
+
+ private List<Accumulator<?>> buildAccumulators(final Tree selectTree, final boolean distinct) {
+ final List<Accumulator<?>> accumulators = new ArrayList<>();
+
+ if ( selectTree.getType() != ProvenanceQueryParser.SELECT ) {
+ throw new IllegalArgumentException("Cannot build accumulators for a non-SELECT tree");
+ }
+
+ for (int i=0; i < selectTree.getChildCount(); i++) {
+ final Tree childTree = selectTree.getChild(i);
+ accumulators.add(buildAccumulator(childTree, distinct));
+ }
+
+ return accumulators;
+ }
+
+ private Accumulator<?> buildAccumulator(final Tree tree, final boolean distinct) {
+ switch (tree.getType()) {
+ case SUM:
+ return new SumAccumulator(accumulatorIdGenerator++, toLongEvaluator(buildOperandEvaluator(tree.getChild(0), Clause.SELECT), tree), "SUM(" + getLabel(tree.getChild(0)) + ")");
+ case AVG:
+ return new AverageAccumulator(accumulatorIdGenerator++, toLongEvaluator(buildOperandEvaluator(tree.getChild(0), Clause.SELECT), tree), "AVG(" + getLabel(tree.getChild(0)) + ")");
+ case EVENT:
+ return new EventAccumulator(accumulatorIdGenerator++, getLabel(tree), distinct);
+ case IDENTIFIER:
+ return new EventAccumulator(accumulatorIdGenerator++, getLabel(tree.getChild(0)), distinct);
+ case EVENT_PROPERTY:
+ case ATTRIBUTE:
+ return new EventAccumulator(accumulatorIdGenerator++, getLabel(tree.getChild(0)), buildOperandEvaluator(tree, Clause.SELECT), distinct);
+ case YEAR:
+ case DAY:
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ return new EventAccumulator(accumulatorIdGenerator++, getLabel(tree), buildOperandEvaluator(tree, Clause.SELECT), distinct);
+ case COUNT:
+ if ( "Event".equalsIgnoreCase(tree.getChild(0).getText() ) ) {
+ return new CountAccumulator(accumulatorIdGenerator++, null, "COUNT(" + getLabel(tree.getChild(0)) + ")");
+ }
+ return new CountAccumulator(accumulatorIdGenerator++, buildOperandEvaluator(tree.getChild(0), Clause.SELECT), "COUNT(" + getLabel(tree.getChild(0)) + ")");
+ default:
+ throw new UnsupportedOperationException("Haven't implemented accumulators yet for " + tree);
+ }
+ }
+
+ private String getLabel(final Tree tree) {
+ final int type = tree.getType();
+
+ switch (type) {
+ case EVENT_PROPERTY:
+ case ATTRIBUTE:
+ return tree.getChild(0).getText();
+ case YEAR:
+ case DAY:
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ return tree.getText() + "(" + getLabel(tree.getChild(0)) + ")";
+ }
+
+ return tree.getText();
+ }
+
+ private OperandEvaluator<?> buildOperandEvaluator(final Tree tree, final Clause clause) {
+ // When events are pulled back from an index, for efficiency purposes, we may want to know which
+ // fields to pull back. The fields in the WHERE clause are irrelevant because they are not shown
+ // to the user, so no need to pull those back.
+ final boolean isReferenceInteresting = clause != Clause.WHERE;
+
+ switch (tree.getType()) {
+ case EVENT_PROPERTY:
+ switch (tree.getChild(0).getType()) {
+ case FILESIZE:
+ if ( searchableFields != null && !searchableFields.contains(SearchableFields.FileSize) ) {
+ throw new ProvenanceQueryLanguageException("Query cannot reference FileSize because this field is not searchable by the repository");
+ }
+ if ( isReferenceInteresting ) {
+ referencedFields.add(SearchableFields.FileSize.getSearchableFieldName());
+ }
+ return new SizeEvaluator();
+ case TRANSIT_URI:
+ if ( searchableFields != null && !searchableFields.contains(SearchableFields.TransitURI) ) {
+ throw new ProvenanceQueryLanguageException("Query cannot reference TransitURI because this field is not searchable by the repository");
+ }
+ if ( isReferenceInteresting ) {
+ referencedFields.add(SearchableFields.TransitURI.getSearchableFieldName());
+ }
+ return new TransitUriEvaluator();
+ case TIMESTAMP:
+ // time is always indexed
+ if ( isReferenceInteresting ) {
+ referencedFields.add(SearchableFields.EventTime.getSearchableFieldName());
+ }
+ return new TimestampEvaluator();
+ case TYPE:
+ // type is always indexed so no need to check it
+ if ( isReferenceInteresting ) {
+ referencedFields.add(SearchableFields.EventType.getSearchableFieldName());
+ }
+ return new TypeEvaluator();
+ case COMPONENT_ID:
+ if ( searchableFields != null && !searchableFields.contains(SearchableFields.ComponentID) ) {
+ throw new ProvenanceQueryLanguageException("Query cannot reference Component ID because this field is not searchable by the repository");
+ }
+ if ( isReferenceInteresting ) {
+ referencedFields.add(SearchableFields.ComponentID.getSearchableFieldName());
+ }
+
+ return new ComponentIdEvaluator();
+ // TODO: Allow Component Type to be indexed and searched
+ case RELATIONSHIP:
+ if ( searchableFields != null && !searchableFields.contains(SearchableFields.Relationship) ) {
+ throw new ProvenanceQueryLanguageException("Query cannot reference Relationship because this field is not searchable by the repository");
+ }
+ if ( isReferenceInteresting ) {
+ referencedFields.add(SearchableFields.Relationship.getSearchableFieldName());
+ }
+
+ return new RelationshipEvaluator();
+ case UUID:
+ if ( searchableFields != null && !searchableFields.contains(SearchableFields.FlowFileUUID) ) {
+ throw new ProvenanceQueryLanguageException("Query cannot reference FlowFile UUID because this field is not searchable by the repository");
+ }
+ if ( isReferenceInteresting ) {
+ referencedFields.add(SearchableFields.FlowFileUUID.getSearchableFieldName());
+ }
+
+ return new UuidEvaluator();
+ default:
+ // TODO: IMPLEMENT
+ throw new UnsupportedOperationException("Haven't implemented extraction of property " + tree.getChild(0).getText());
+ }
+ case ATTRIBUTE:
+ final String attributeName = tree.getChild(0).getText();
+ if ( searchableAttributes != null && !searchableAttributes.contains(attributeName) ) {
+ throw new ProvenanceQueryLanguageException("Query cannot attribute '" + attributeName + "' because this attribute is not searchable by the repository");
+ }
+
+ if ( isReferenceInteresting ) {
+ referencedFields.add(attributeName);
+ }
+
+ return new AttributeEvaluator(toStringEvaluator(buildOperandEvaluator(tree.getChild(0), clause), tree));
+ case STRING_LITERAL:
+ return new StringLiteralEvaluator(tree.getText());
+ case NUMBER:
+ return new LongLiteralEvaluator(Long.valueOf(tree.getText()));
+ case YEAR:
+ return new TimeFieldEvaluator(toLongEvaluator(buildOperandEvaluator(tree.getChild(0), clause), tree), Calendar.YEAR, YEAR);
+ case MONTH:
+ return new TimeFieldEvaluator(toLongEvaluator(buildOperandEvaluator(tree.getChild(0), clause), tree), Calendar.MONTH, MONTH);
+ case DAY:
+ return new TimeFieldEvaluator(toLongEvaluator(buildOperandEvaluator(tree.getChild(0), clause), tree), Calendar.DAY_OF_YEAR, DAY);
+ case HOUR:
+ return new TimeFieldEvaluator(toLongEvaluator(buildOperandEvaluator(tree.getChild(0), clause), tree), Calendar.HOUR_OF_DAY, HOUR);
+ case MINUTE:
+ return new TimeFieldEvaluator(toLongEvaluator(buildOperandEvaluator(tree.getChild(0), clause), tree), Calendar.MINUTE, MINUTE);
+ case SECOND:
+ return new TimeFieldEvaluator(toLongEvaluator(buildOperandEvaluator(tree.getChild(0), clause), tree), Calendar.SECOND, SECOND);
+ default:
+ throw new ProvenanceQueryLanguageParsingException("Unable to extract value '" + tree.toString() + "' from event because it is not a valid ");
+ }
+ }
+
+
+ private RecordEvaluator<Boolean> buildSourceEvaluator(final Tree fromTree) {
+ if ( fromTree == null ) {
+ throw new NullPointerException();
+ }
+ if ( fromTree.getType() != FROM ) {
+ throw new IllegalArgumentException("Cannot build Soruce Evaluator from a Tree that is not a FROM-tree");
+ }
+
+ final Set<ProvenanceEventType> types = new HashSet<>();
+ for ( int i=0; i < fromTree.getChildCount(); i++ ) {
+ final Tree typeTree = fromTree.getChild(i);
+ if ( "*".equals(typeTree.getText()) ) {
+ return null;
+ } else {
+ types.add(ProvenanceEventType.valueOf(typeTree.getText().toUpperCase()));
+ }
+ }
+
+ return new RecordTypeEvaluator(types);
+ }
+
+
+ private BooleanEvaluator buildConditionEvaluator(final Tree tree, final Clause clause) {
+ switch (tree.getType()) {
+ case AND:
+ return new AndEvaluator(buildConditionEvaluator(tree.getChild(0), clause), buildConditionEvaluator(tree.getChild(1), clause));
+ case OR:
+ return new OrEvaluator(buildConditionEvaluator(tree.getChild(0), clause), buildConditionEvaluator(tree.getChild(1), clause));
+ case EQUALS:
+ return new EqualsEvaluator(buildOperandEvaluator(tree.getChild(0), clause), buildOperandEvaluator(tree.getChild(1), clause));
+ case NOT_EQUALS:
+ return new EqualsEvaluator(buildOperandEvaluator(tree.getChild(0), clause), buildOperandEvaluator(tree.getChild(1), clause), true);
+ case GT:
+ return new GreaterThanEvaluator(buildOperandEvaluator(tree.getChild(0), clause), buildOperandEvaluator(tree.getChild(1), clause));
+ case LT:
+ return new LessThanEvaluator(buildOperandEvaluator(tree.getChild(0), clause), buildOperandEvaluator(tree.getChild(1), clause));
+ case NOT:
+ return buildConditionEvaluator(tree.getChild(0), clause).negate();
+ case MATCHES: {
+ final OperandEvaluator<?> rhs = buildOperandEvaluator(tree.getChild(1), clause);
+ if ( !String.class.equals( rhs.getType() ) ) {
+ throw new ProvenanceQueryLanguageParsingException("Right-hand side of MATCHES operator must be a Regular Expression but found " + rhs);
+ }
+ return new MatchesEvaluator(buildOperandEvaluator(tree.getChild(0), clause), rhs);
+ }
+ case STARTS_WITH: {
+ final OperandEvaluator<?> rhs = buildOperandEvaluator(tree.getChild(1), clause);
+ if ( !String.class.equals( rhs.getType() ) ) {
+ throw new ProvenanceQueryLanguageParsingException("Right-hand side of STARTS WITH operator must be a String but found " + rhs);
+ }
+ return new StartsWithEvaluator(buildOperandEvaluator(tree.getChild(0), clause), rhs);
+ }
+ default:
+ // TODO: Implement
+ throw new UnsupportedOperationException("Have not yet implemented condition evaluator for " + tree);
+ }
+ }
+
+
+ private <T> OperandEvaluator<T> castEvaluator(final OperandEvaluator<?> eval, final Tree tree, final Class<T> expectedType) {
+ if ( eval.getType() != expectedType ) {
+ throw new ProvenanceQueryLanguageParsingException("Expected type " + expectedType.getSimpleName() + " but found type " + eval.getType() + " for term: " + tree);
+ }
+
+ @SuppressWarnings("unchecked")
+ final OperandEvaluator<T> retEvaluator = ((OperandEvaluator<T>) eval);
+ return retEvaluator;
+
+ }
+
+ private OperandEvaluator<String> toStringEvaluator(final OperandEvaluator<?> eval, final Tree tree) {
+ return castEvaluator(eval, tree, String.class);
+ }
+
+ private OperandEvaluator<Long> toLongEvaluator(final OperandEvaluator<?> eval, final Tree tree) {
+ if ( eval.getType() == Long.class ) {
+ @SuppressWarnings("unchecked")
+ final OperandEvaluator<Long> retEvaluator = ((OperandEvaluator<Long>) eval);
+ return retEvaluator;
+ } else if ( eval.getType() == String.class ) {
+ @SuppressWarnings("unchecked")
+ final OperandEvaluator<String> stringEval = ((OperandEvaluator<String>) eval);
+ return new StringToLongEvaluator(stringEval);
+ }
+
+ return castEvaluator(eval, tree, Long.class);
+ }
+
+
+ private List<RecordEvaluator<?>> buildGroupEvaluators(final Tree groupByTree) {
+ if ( groupByTree == null ) {
+ return null;
+ }
+
+ if ( groupByTree.getType() != GROUP_BY ) {
+ throw new IllegalArgumentException("Expected GroupBy tree but got " + groupByTree);
+ }
+
+ final List<RecordEvaluator<?>> evaluators = new ArrayList<>(groupByTree.getChildCount());
+ for (int i=0; i < groupByTree.getChildCount(); i++) {
+ final Tree tree = groupByTree.getChild(i);
+ final RecordEvaluator<?> evaluator;
+
+ switch (tree.getType()) {
+ case EVENT_PROPERTY:
+ case STRING_LITERAL:
+ case ATTRIBUTE:
+ case YEAR:
+ case DAY:
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ evaluator = buildOperandEvaluator(tree, Clause.GROUP);
+ break;
+ default:
+ evaluator = buildConditionEvaluator(tree, Clause.GROUP);
+ break;
+ }
+
+ evaluators.add(evaluator);
+ }
+
+ return evaluators;
+ }
+
+ private RowSorter buildSorter(final Tree orderByTree, final boolean grouped) {
+ if ( orderByTree.getType() != ORDER_BY ) {
+ throw new IllegalArgumentException();
+ }
+
+ if ( grouped ) {
+ final Map<Accumulator<?>, SortDirection> accumulators = new LinkedHashMap<>(orderByTree.getChildCount());
+ for (int i=0; i < orderByTree.getChildCount(); i++) {
+ final Tree orderTree = orderByTree.getChild(i);
+ final Accumulator<?> accumulator = buildAccumulator(orderTree.getChild(0), true);
+
+ final SortDirection sortDir;
+ if ( orderTree.getChildCount() > 1 ) {
+ final int sortDirType = orderTree.getChild(1).getType();
+ sortDir = (sortDirType == ASC) ? SortDirection.ASC : SortDirection.DESC;
+ } else {
+ sortDir = SortDirection.ASC;
+ }
+
+ accumulators.put(accumulator, sortDir);
+ }
+
+ return new GroupedSorter(accumulators);
+ } else {
+ // TODO: Allow ORDER BY of aggregate values
+ final Map<OperandEvaluator<?>, SortDirection> evaluators = new LinkedHashMap<>(orderByTree.getChildCount());
+ for (int i=0; i < orderByTree.getChildCount(); i++) {
+ final Tree orderTree = orderByTree.getChild(i);
+ final OperandEvaluator<?> evaluator = buildOperandEvaluator(orderTree.getChild(0), Clause.ORDER);
+
+ final SortDirection sortDir;
+ if ( orderTree.getChildCount() > 1 ) {
+ final int sortDirType = orderTree.getChild(1).getType();
+ sortDir = (sortDirType == ASC) ? SortDirection.ASC : SortDirection.DESC;
+ } else {
+ sortDir = SortDirection.ASC;
+ }
+
+ evaluators.put(evaluator, sortDir);
+ }
+
+ return new FieldSorter(evaluators);
+ }
+ }
+
+
+ public static ProvenanceResultSet execute(final String query, final ProvenanceEventRepository repo) throws IOException {
+ return ProvenanceQuery.compile(query, null, null).execute(repo);
+ }
+
+
+ public ProvenanceResultSet evaluate(final Iterator<? extends StoredProvenanceEvent> matchedEvents) {
+ final List<String> labels = new ArrayList<>();
+ final List<Class<?>> returnTypes = new ArrayList<>(selectAccumulators.size());
+
+ for ( final Accumulator<?> accumulator : selectAccumulators ) {
+ labels.add(accumulator.getLabel());
+ returnTypes.add(accumulator.getReturnType());
+ }
+
+ ProvenanceResultSet rs;
+ if ( isAggregateRequired() ) {
+ rs = new GroupingResultSet(matchedEvents,
+ selectAccumulators, sourceEvaluator, conditionEvaluator,
+ labels, returnTypes, groupEvaluators, sorter, limit);
+ } else if (sorter == null) {
+ rs = new StandardUnorderedResultSet(matchedEvents, selectAccumulators, sourceEvaluator, conditionEvaluator, labels, returnTypes, limit);
+ } else {
+ rs = new StandardOrderedResultSet(matchedEvents, selectAccumulators, sourceEvaluator, conditionEvaluator, labels, returnTypes, sorter, limit);
+ }
+
+ return rs;
+ }
+
+ public ProvenanceResultSet execute(final ProvenanceEventRepository repo) throws IOException {
+ final RepositoryEvaluator repoEvaluator = new SelectAllRecords();
+
+ final Iterator<StoredProvenanceEvent> potentialMatches = repoEvaluator.evaluate(repo);
+ final List<String> labels = new ArrayList<>();
+ final List<Class<?>> returnTypes = new ArrayList<>(selectAccumulators.size());
+
+ for ( final Accumulator<?> accumulator : selectAccumulators ) {
+ labels.add(accumulator.getLabel());
+ returnTypes.add(accumulator.getReturnType());
+ }
+
+ ProvenanceResultSet rs;
+ if ( isAggregateRequired() ) {
+ rs = new GroupingResultSet(potentialMatches,
+ selectAccumulators, sourceEvaluator, conditionEvaluator,
+ labels, returnTypes, groupEvaluators, sorter, limit);
+ } else if (sorter == null) {
+ rs = new StandardUnorderedResultSet(potentialMatches, selectAccumulators, sourceEvaluator, conditionEvaluator, labels, returnTypes, limit);
+ } else {
+ rs = new StandardOrderedResultSet(potentialMatches, selectAccumulators, sourceEvaluator, conditionEvaluator, labels, returnTypes, sorter, limit);
+ }
+
+ return rs;
+ }
+
+ private boolean isAggregateRequired() {
+ if ( groupEvaluators != null && !groupEvaluators.isEmpty() ) {
+ return true;
+ }
+
+ for ( final Accumulator<?> accumulator : selectAccumulators ) {
+ if ( accumulator.isAggregateFunction() ) {
+ return true;
+ }
+ }
+
+
+ return false;
+ }
+
+
+ public RecordEvaluator<Boolean> getWhereClause() {
+ return conditionEvaluator;
+ }
+
+ private static enum Clause {
+ SELECT,
+ FROM,
+ WHERE,
+ GROUP,
+ ORDER;
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/Accumulator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/Accumulator.java
new file mode 100644
index 0000000..db49091
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/Accumulator.java
@@ -0,0 +1,28 @@
+package org.apache.nifi.pql.evaluation;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface Accumulator<T> extends Cloneable {
+
+ T accumulate(ProvenanceEventRecord record, Group group);
+
+ String getLabel();
+
+ boolean isAggregateFunction();
+
+ Class<? extends T> getReturnType();
+
+ Accumulator<T> clone();
+
+ long getId();
+
+ void reset();
+
+ Map<Group, List<T>> getValues();
+
+ List<T> getValues(Group group);
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/BooleanEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/BooleanEvaluator.java
new file mode 100644
index 0000000..2c1549a
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/BooleanEvaluator.java
@@ -0,0 +1,5 @@
+package org.apache.nifi.pql.evaluation;
+
+public interface BooleanEvaluator extends RecordEvaluator<Boolean> {
+ BooleanEvaluator negate();
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/OperandEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/OperandEvaluator.java
new file mode 100644
index 0000000..0daace6
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/OperandEvaluator.java
@@ -0,0 +1,7 @@
+package org.apache.nifi.pql.evaluation;
+
+public interface OperandEvaluator<T> extends RecordEvaluator<T> {
+
+ Class<T> getType();
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/RecordEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/RecordEvaluator.java
new file mode 100644
index 0000000..98f3b04
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/RecordEvaluator.java
@@ -0,0 +1,11 @@
+package org.apache.nifi.pql.evaluation;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface RecordEvaluator<T> {
+
+ T evaluate(ProvenanceEventRecord record);
+
+ int getEvaluatorType();
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/RepositoryEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/RepositoryEvaluator.java
new file mode 100644
index 0000000..eb55f75
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/RepositoryEvaluator.java
@@ -0,0 +1,13 @@
+package org.apache.nifi.pql.evaluation;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+
+public interface RepositoryEvaluator {
+
+ Iterator<StoredProvenanceEvent> evaluate(ProvenanceEventRepository repository) throws IOException;
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/AverageAccumulator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/AverageAccumulator.java
new file mode 100644
index 0000000..9b04308
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/AverageAccumulator.java
@@ -0,0 +1,112 @@
+package org.apache.nifi.pql.evaluation.accumulation;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class AverageAccumulator implements Accumulator<Double> {
+
+ private final long id;
+ private final RecordEvaluator<Long> evaluator;
+ private final String label;
+ private final Map<Group, Values> values = new LinkedHashMap<>();
+
+ public AverageAccumulator(final long id, final RecordEvaluator<Long> extractor, final String label) {
+ this.id = id;
+ this.evaluator = extractor;
+ this.label = label;
+ }
+
+ @Override
+ public Map<Group, List<Double>> getValues() {
+ final Map<Group, List<Double>> avgs = new HashMap<>(values.size());
+ for ( final Map.Entry<Group, Values> entry : values.entrySet() ) {
+ final Values values = entry.getValue();
+ final double avg = (double) values.getSum() / (double) values.getCount();
+ avgs.put(entry.getKey(), Collections.singletonList(avg));
+ }
+ return avgs;
+ }
+
+ @Override
+ public List<Double> getValues(final Group group) {
+ final Values v = values.get(group);
+ if ( v == null ) {
+ return Collections.emptyList();
+ }
+
+ final double d = v.getSum() / v.getCount();
+ return Collections.singletonList(d);
+ }
+
+ public Double accumulate(final ProvenanceEventRecord record, final Group group) {
+ final Long val = evaluator.evaluate(record);
+ if ( val != null ) {
+ Values v = values.get(group);
+ if ( v == null ) {
+ v = new Values();
+ values.put(group, v);
+ }
+
+ v.increment(val.longValue());
+ return (double) v.getSum() / (double) v.getCount();
+ }
+
+ return null;
+ }
+
+ @Override
+ public String getLabel() {
+ return label;
+ }
+
+ @Override
+ public boolean isAggregateFunction() {
+ return true;
+ }
+
+ @Override
+ public Class<Double> getReturnType() {
+ return Double.class;
+ }
+
+ @Override
+ public AverageAccumulator clone() {
+ return new AverageAccumulator(id, evaluator, label);
+ }
+
+ @Override
+ public long getId() {
+ return id;
+ }
+
+ @Override
+ public void reset() {
+ values.clear();
+ }
+
+ private static class Values {
+ private long count;
+ private long sum;
+
+ public void increment(final long sum) {
+ count++;
+ this.sum += sum;
+ }
+
+ public long getCount() {
+ return count;
+ }
+
+ public long getSum() {
+ return sum;
+ }
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/CountAccumulator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/CountAccumulator.java
new file mode 100644
index 0000000..d4af4bf
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/CountAccumulator.java
@@ -0,0 +1,91 @@
+package org.apache.nifi.pql.evaluation.accumulation;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class CountAccumulator implements Accumulator<Long> {
+
+ private final long id;
+ private final RecordEvaluator<?> evaluator;
+ private final String label;
+ private final Map<Group, Long> counts = new LinkedHashMap<>();
+
+
+ public CountAccumulator(final long id, final RecordEvaluator<?> extractor, final String label) {
+ this.id = id;
+ this.evaluator = extractor;
+ this.label = label;
+ }
+
+ @Override
+ public Map<Group, List<Long>> getValues() {
+ final Map<Group, List<Long>> map = new HashMap<>();
+ for ( final Map.Entry<Group, Long> entry : counts.entrySet() ) {
+ map.put(entry.getKey(), Collections.singletonList(entry.getValue()));
+ }
+ return map;
+ }
+
+ @Override
+ public List<Long> getValues(final Group group) {
+ final Long count = counts.get(group);
+ if ( count == null ) {
+ return Collections.emptyList();
+ }
+
+ return Collections.singletonList(count);
+ }
+
+ @Override
+ public Long accumulate(final ProvenanceEventRecord record, final Group group) {
+ final Object value = (evaluator == null) ? record : evaluator.evaluate(record);
+ if ( value != null ) {
+ Long val = counts.get(group);
+ if ( val == null ) {
+ val = 0L;
+ }
+ counts.put(group, val + 1);
+ return val + 1;
+ }
+
+ return counts.get(group);
+ }
+
+ @Override
+ public String getLabel() {
+ return label;
+ }
+
+ @Override
+ public boolean isAggregateFunction() {
+ return true;
+ }
+
+ @Override
+ public Class<? extends Long> getReturnType() {
+ return Long.class;
+ }
+
+ @Override
+ public long getId() {
+ return id;
+ }
+
+ @Override
+ public void reset() {
+ counts.clear();
+ }
+
+ public CountAccumulator clone() {
+ return new CountAccumulator(id, evaluator, label);
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/EventAccumulator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/EventAccumulator.java
new file mode 100644
index 0000000..9d4487c
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/EventAccumulator.java
@@ -0,0 +1,98 @@
+package org.apache.nifi.pql.evaluation.accumulation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class EventAccumulator implements Accumulator<Object>, Cloneable {
+
+ private final long id;
+ private final Map<Group, List<Object>> records = new LinkedHashMap<>();
+ private final String label;
+ private final OperandEvaluator<?> valueExtractor;
+ private final boolean distinct;
+
+ public EventAccumulator(final long id, final String label, final boolean distinct) {
+ this(id, label, null, distinct);
+ }
+
+ public EventAccumulator(final long id, final String label, final OperandEvaluator<?> valueExtractor, final boolean distinct) {
+ this.id = id;
+ this.label = label;
+ this.valueExtractor = valueExtractor;
+ this.distinct = distinct;
+ }
+
+ @Override
+ public Map<Group, List<Object>> getValues() {
+ return Collections.unmodifiableMap(records);
+ }
+
+ @Override
+ public List<Object> getValues(final Group group) {
+ final List<Object> objects = records.get(group);
+ if ( objects == null ) {
+ return Collections.emptyList();
+ }
+
+ return Collections.unmodifiableList(objects);
+ }
+
+ @Override
+ public Object accumulate(final ProvenanceEventRecord record, final Group group) {
+ final Object value = valueExtractor == null ? record : valueExtractor.evaluate(record);
+
+ if ( group == null ) {
+ return value;
+ }
+
+ List<Object> groupRecords = records.get(group);
+ if ( groupRecords == null ) {
+ groupRecords = new ArrayList<>();
+ records.put(group, groupRecords);
+ }
+
+ if ( !distinct || !groupRecords.contains(value) ) {
+ groupRecords.add( value );
+ }
+
+ return Collections.unmodifiableList(groupRecords);
+ }
+
+ @Override
+ public String getLabel() {
+ return label;
+ }
+
+ @Override
+ public boolean isAggregateFunction() {
+ return false;
+ }
+
+ @Override
+ public Class<? extends Object> getReturnType() {
+ return valueExtractor == null ? ProvenanceEventRecord.class : valueExtractor.getType();
+ }
+
+ @Override
+ public EventAccumulator clone() {
+ return new EventAccumulator(id, label, valueExtractor, distinct);
+ }
+
+ @Override
+ public long getId() {
+ return id;
+ }
+
+ @Override
+ public void reset() {
+ records.clear();
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/SumAccumulator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/SumAccumulator.java
new file mode 100644
index 0000000..5eacc6e
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/accumulation/SumAccumulator.java
@@ -0,0 +1,91 @@
+package org.apache.nifi.pql.evaluation.accumulation;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class SumAccumulator implements Accumulator<Long>, Cloneable {
+
+ private final long id;
+ private final RecordEvaluator<Long> evaluator;
+ private final String label;
+ private final Map<Group, Long> sums = new LinkedHashMap<>();
+
+ public SumAccumulator(final long id, final RecordEvaluator<Long> extractor, final String label) {
+ this.id = id;
+ this.evaluator = extractor;
+ this.label = label;
+ }
+
+ @Override
+ public Map<Group, List<Long>> getValues() {
+ final Map<Group, List<Long>> map = new HashMap<>();
+ for ( final Map.Entry<Group, Long> entry : sums.entrySet() ) {
+ map.put(entry.getKey(), Collections.singletonList(entry.getValue()));
+ }
+ return map;
+ }
+
+ @Override
+ public List<Long> getValues(final Group group) {
+ final Long sum = sums.get(group);
+ if ( sum == null ) {
+ return Collections.emptyList();
+ }
+
+ return Collections.singletonList(sum);
+ }
+
+ @Override
+ public Long accumulate(final ProvenanceEventRecord record, final Group group) {
+ final Long val = evaluator.evaluate(record);
+ if ( val != null ) {
+ Long curVal = sums.get(group);
+ if ( curVal == null ) {
+ curVal = 0L;
+ }
+
+ final long newVal = curVal + val;
+ sums.put(group, newVal);
+ return newVal;
+ }
+
+ return null;
+ }
+
+ @Override
+ public String getLabel() {
+ return label;
+ }
+
+ @Override
+ public boolean isAggregateFunction() {
+ return true;
+ }
+
+ @Override
+ public Class<Long> getReturnType() {
+ return Long.class;
+ }
+
+ public SumAccumulator clone() {
+ return new SumAccumulator(id, evaluator, label);
+ }
+
+ @Override
+ public long getId() {
+ return id;
+ }
+
+ @Override
+ public void reset() {
+ sums.clear();
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/ConversionUtils.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/ConversionUtils.java
new file mode 100644
index 0000000..75b7a3b
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/ConversionUtils.java
@@ -0,0 +1,97 @@
+package org.apache.nifi.pql.evaluation.comparison;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ConversionUtils {
+
+ public static final String DATE_TO_STRING_FORMAT = "EEE MMM dd HH:mm:ss zzz yyyy";
+ public static final Pattern DATE_TO_STRING_PATTERN = Pattern.compile("(?:[a-zA-Z]{3} ){2}\\d{2} \\d{2}\\:\\d{2}\\:\\d{2} (?:.*?) \\d{4}");
+
+ public static final String ALTERNATE_FORMAT_WITHOUT_MILLIS = "yyyy/MM/dd HH:mm:ss";
+ public static final String ALTERNATE_FORMAT_WITH_MILLIS = "yyyy/MM/dd HH:mm:ss.SSS";
+ public static final Pattern ALTERNATE_PATTERN = Pattern.compile("\\d{4}/\\d{2}/\\d{2} \\d{2}\\:\\d{2}\\:\\d{2}(\\.\\d{3})?");
+
+ public static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+");
+
+
+ public static Long convertToLong(final Object o) {
+ if ( o == null ) {
+ return null;
+ }
+
+ if (o instanceof Long) {
+ return (Long) o;
+ }
+
+ if (o instanceof Number) {
+ return ((Number) o).longValue();
+ }
+
+ if ( o instanceof Date ) {
+ return ((Date) o).getTime();
+ }
+
+ if ( o instanceof Calendar ) {
+ return ((Calendar) o).getTimeInMillis();
+ }
+
+ if ( o instanceof String ) {
+ return convertToLong((String) o);
+ }
+
+ return null;
+ }
+
+ public static Long convertToLong(final String value) {
+ if ( value == null ) {
+ return null;
+ }
+
+ final String trimmed = value.trim();
+ if ( trimmed.isEmpty() ) {
+ return null;
+ }
+
+
+ if ( DATE_TO_STRING_PATTERN.matcher(trimmed).matches() ) {
+ final SimpleDateFormat sdf = new SimpleDateFormat(DATE_TO_STRING_FORMAT);
+
+ try {
+ final Date date = sdf.parse(trimmed);
+ return date.getTime();
+ } catch (final ParseException pe) {
+ return null;
+ }
+ } else if ( NUMBER_PATTERN.matcher(trimmed).matches() ) {
+ return Long.valueOf(trimmed);
+ } else {
+ final Matcher altMatcher = ALTERNATE_PATTERN.matcher(trimmed);
+ if ( altMatcher.matches() ) {
+ final String millisValue = altMatcher.group(1);
+
+ final String format;
+ if ( millisValue == null ) {
+ format = ALTERNATE_FORMAT_WITHOUT_MILLIS;
+ } else {
+ format = ALTERNATE_FORMAT_WITH_MILLIS;
+ }
+
+ final SimpleDateFormat sdf = new SimpleDateFormat(format);
+
+ try {
+ return sdf.parse(trimmed).getTime();
+ } catch (final ParseException pe) {
+ return null;
+ }
+ } else {
+ return null;
+ }
+ }
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/EqualsEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/EqualsEvaluator.java
new file mode 100644
index 0000000..70158fb
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/EqualsEvaluator.java
@@ -0,0 +1,74 @@
+package org.apache.nifi.pql.evaluation.comparison;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+public class EqualsEvaluator implements BooleanEvaluator {
+
+ private final OperandEvaluator<?> lhs;
+ private final OperandEvaluator<?> rhs;
+ private final boolean negated;
+ private final String alias;
+
+ public EqualsEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs) {
+ this(lhs, rhs, false, null);
+ }
+
+ public EqualsEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final String alias) {
+ this(lhs, rhs, false, alias);
+ }
+
+ public EqualsEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final boolean negated) {
+ this(lhs, rhs, negated, null);
+ }
+
+ public EqualsEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final boolean negated, final String alias) {
+ this.lhs = lhs;
+ this.rhs = rhs;
+ this.negated = negated;
+ this.alias = alias;
+ }
+
+ public OperandEvaluator<?> getLHS() {
+ return lhs;
+ }
+
+ public OperandEvaluator<?> getRHS() {
+ return rhs;
+ }
+
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ Object lhsValue = lhs.evaluate(record);
+ Object rhsValue = rhs.evaluate(record);
+
+ if ( lhsValue == null || rhsValue == null ) {
+ return false;
+ }
+
+ if ( lhsValue instanceof ProvenanceEventType ) {
+ lhsValue = ((ProvenanceEventType) lhsValue).name();
+ }
+ if ( rhsValue instanceof ProvenanceEventType ) {
+ rhsValue = ((ProvenanceEventType) rhsValue).name();
+ }
+
+ final boolean equal = lhsValue.equals(rhsValue);
+ return negated ? !equal : equal;
+ }
+
+ public BooleanEvaluator negate() {
+ return new EqualsEvaluator(lhs, rhs, !negated, alias);
+ }
+
+ @Override
+ public String toString() {
+ return alias == null ? lhs.toString() + "=" + rhs.toString() : alias;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.EQUALS;
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/GreaterThanEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/GreaterThanEvaluator.java
new file mode 100644
index 0000000..327fc9f
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/GreaterThanEvaluator.java
@@ -0,0 +1,51 @@
+package org.apache.nifi.pql.evaluation.comparison;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class GreaterThanEvaluator implements BooleanEvaluator {
+ private final OperandEvaluator<?> lhs;
+ private final OperandEvaluator<?> rhs;
+ private final boolean negated;
+
+ public GreaterThanEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs) {
+ this(lhs, rhs, false);
+ }
+
+ public GreaterThanEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final boolean negate) {
+ this.lhs = lhs;
+ this.rhs = rhs;
+ this.negated = negate;
+ }
+
+ public OperandEvaluator<?> getLHS() {
+ return lhs;
+ }
+
+ public OperandEvaluator<?> getRHS() {
+ return rhs;
+ }
+
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ final Long lhsValue = ConversionUtils.convertToLong(lhs.evaluate(record));
+ final Long rhsValue = ConversionUtils.convertToLong(rhs.evaluate(record));
+
+ if ( lhsValue == null || rhsValue == null ) {
+ return false;
+ }
+
+ final boolean greaterThan = lhsValue.longValue() > rhsValue.longValue();
+ return negated ? !greaterThan : greaterThan;
+ }
+
+ public BooleanEvaluator negate() {
+ return new GreaterThanEvaluator(lhs, rhs, !negated);
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.GT;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/LessThanEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/LessThanEvaluator.java
new file mode 100644
index 0000000..5940509
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/LessThanEvaluator.java
@@ -0,0 +1,51 @@
+package org.apache.nifi.pql.evaluation.comparison;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class LessThanEvaluator implements BooleanEvaluator {
+ private final OperandEvaluator<?> lhs;
+ private final OperandEvaluator<?> rhs;
+ private final boolean negated;
+
+ public LessThanEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs) {
+ this(lhs, rhs, false);
+ }
+
+ public LessThanEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final boolean negate) {
+ this.lhs = lhs;
+ this.rhs = rhs;
+ this.negated = negate;
+ }
+
+ public OperandEvaluator<?> getLHS() {
+ return lhs;
+ }
+
+ public OperandEvaluator<?> getRHS() {
+ return rhs;
+ }
+
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ final Long lhsValue = ConversionUtils.convertToLong(lhs.evaluate(record));
+ final Long rhsValue = ConversionUtils.convertToLong(rhs.evaluate(record));
+
+ if ( lhsValue == null || rhsValue == null ) {
+ return false;
+ }
+
+ final boolean lessThan = lhsValue.longValue() < rhsValue.longValue();
+ return negated ? !lessThan : lessThan;
+ }
+
+ public BooleanEvaluator negate() {
+ return new GreaterThanEvaluator(lhs, rhs, !negated);
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.LT;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/MatchesEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/MatchesEvaluator.java
new file mode 100644
index 0000000..b646129
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/MatchesEvaluator.java
@@ -0,0 +1,72 @@
+package org.apache.nifi.pql.evaluation.comparison;
+
+import java.util.regex.Pattern;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.pql.evaluation.literals.StringLiteralEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class MatchesEvaluator implements BooleanEvaluator {
+
+ private final OperandEvaluator<?> lhs;
+ private final OperandEvaluator<?> rhs;
+ private final boolean negated;
+ private final Pattern pattern;
+
+ public MatchesEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs) {
+ this(lhs, rhs, false);
+ }
+
+
+ public MatchesEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final boolean negated) {
+ this.lhs = lhs;
+ this.rhs = rhs;
+ this.negated = negated;
+
+ if ( rhs instanceof StringLiteralEvaluator ) {
+ pattern = Pattern.compile(rhs.evaluate(null).toString());
+ } else {
+ pattern = null;
+ }
+ }
+
+ public OperandEvaluator<?> getLHS() {
+ return lhs;
+ }
+
+ public OperandEvaluator<?> getRHS() {
+ return rhs;
+ }
+
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ Object lhsValue = lhs.evaluate(record);
+ Object rhsValue = rhs.evaluate(record);
+
+ if ( lhsValue == null || rhsValue == null ) {
+ return false;
+ }
+
+ final String lhsString = lhsValue.toString();
+
+ final Pattern compiled;
+ if ( pattern == null ) {
+ compiled = Pattern.compile(rhsValue.toString());
+ } else {
+ compiled = pattern;
+ }
+
+ final boolean matches = compiled.matcher(lhsString).matches();
+ return negated ? !matches : matches;
+ }
+
+ public MatchesEvaluator negate() {
+ return new MatchesEvaluator(lhs, rhs, !negated);
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.MATCHES;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/RecordTypeEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/RecordTypeEvaluator.java
new file mode 100644
index 0000000..d267371
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/RecordTypeEvaluator.java
@@ -0,0 +1,34 @@
+package org.apache.nifi.pql.evaluation.comparison;
+
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+public class RecordTypeEvaluator implements BooleanEvaluator {
+ private final Set<ProvenanceEventType> types;
+
+ public RecordTypeEvaluator(final Set<ProvenanceEventType> types) {
+ this.types = new HashSet<>(types);
+ }
+
+ @Override
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ return types.contains(record.getEventType());
+ }
+
+ @Override
+ public BooleanEvaluator negate() {
+ final Set<ProvenanceEventType> negatedTypes = EnumSet.complementOf(EnumSet.copyOf(types));
+ return new RecordTypeEvaluator(negatedTypes);
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.TYPE;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/StartsWithEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/StartsWithEvaluator.java
new file mode 100644
index 0000000..7c11416
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/comparison/StartsWithEvaluator.java
@@ -0,0 +1,57 @@
+package org.apache.nifi.pql.evaluation.comparison;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class StartsWithEvaluator implements BooleanEvaluator {
+
+ private final boolean negated;
+ private final OperandEvaluator<?> lhs;
+ private final OperandEvaluator<?> rhs;
+
+ public StartsWithEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs) {
+ this(lhs, rhs, false);
+ }
+
+ public StartsWithEvaluator(final OperandEvaluator<?> lhs, final OperandEvaluator<?> rhs, final boolean negated) {
+ this.lhs = lhs;
+ this.rhs = rhs;
+ this.negated = negated;
+ }
+
+ @Override
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ final Object lhsValue = lhs.evaluate(record);
+ final Object rhsValue = rhs.evaluate(record);
+
+ if ( lhsValue == null || rhsValue == null ) {
+ return false;
+ }
+
+ final String lhsString = lhsValue.toString();
+ final String rhsString = rhsValue.toString();
+
+ final boolean startsWith = lhsString.startsWith(rhsString);
+ return negated ? !startsWith : startsWith;
+ }
+
+ public OperandEvaluator<?> getLHS() {
+ return lhs;
+ }
+
+ public OperandEvaluator<?> getRHS() {
+ return rhs;
+ }
+
+ @Override
+ public BooleanEvaluator negate() {
+ return new StartsWithEvaluator(lhs, rhs, !negated);
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.STARTS_WITH;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/conversion/DateToLongEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/conversion/DateToLongEvaluator.java
new file mode 100644
index 0000000..6832e14
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/conversion/DateToLongEvaluator.java
@@ -0,0 +1,27 @@
+package org.apache.nifi.pql.evaluation.conversion;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class DateToLongEvaluator implements OperandEvaluator<Long> {
+
+
+ @Override
+ public Long evaluate(ProvenanceEventRecord record) {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ // TODO Auto-generated method stub
+ return 0;
+ }
+
+ @Override
+ public Class<Long> getType() {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/conversion/StringToLongEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/conversion/StringToLongEvaluator.java
new file mode 100644
index 0000000..38ac9f0
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/conversion/StringToLongEvaluator.java
@@ -0,0 +1,68 @@
+/*
+ * 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.pql.evaluation.conversion;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class StringToLongEvaluator implements OperandEvaluator<Long> {
+ private final OperandEvaluator<String> stringEvaluator;
+
+ public StringToLongEvaluator(final OperandEvaluator<String> stringEvaluator) {
+ this.stringEvaluator = stringEvaluator;
+ }
+
+ @Override
+ public Long evaluate(final ProvenanceEventRecord record) {
+ final String result = stringEvaluator.evaluate(record);
+ if (result == null) {
+ return null;
+ }
+
+ final String trimmed = result.trim();
+ if ( trimmed.isEmpty() ) {
+ return 0L;
+ }
+
+ if ( isNumber(trimmed) ) {
+ return Long.parseLong(trimmed);
+ }
+
+ return null;
+ }
+
+ private boolean isNumber(final String value) {
+ for (int i=0; i < value.length(); i++) {
+ final char c = value.charAt(i);
+ if ( c < '0' || c > '9' ) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return -1;
+ }
+
+ @Override
+ public Class<Long> getType() {
+ return Long.class;
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/AttributeEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/AttributeEvaluator.java
new file mode 100644
index 0000000..c143c4a
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/AttributeEvaluator.java
@@ -0,0 +1,42 @@
+package org.apache.nifi.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class AttributeEvaluator implements OperandEvaluator<String> {
+ private final OperandEvaluator<String> attributeNameEvaluator;
+
+ public AttributeEvaluator(final OperandEvaluator<String> attributeNameEvaluator) {
+ this.attributeNameEvaluator = attributeNameEvaluator;
+ }
+
+ public OperandEvaluator<String> getAttributeNameEvaluator() {
+ return attributeNameEvaluator;
+ }
+
+ @Override
+ public String evaluate(final ProvenanceEventRecord record) {
+ final String attributeName = attributeNameEvaluator.evaluate(record);
+ if ( attributeName == null ) {
+ return null;
+ }
+
+ return record.getAttribute(attributeName);
+ }
+
+
+ @Override
+ public Class<String> getType() {
+ return String.class;
+ }
+
+ @Override
+ public String toString() {
+ return attributeNameEvaluator.toString();
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.ATTRIBUTE;
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/ComponentIdEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/ComponentIdEvaluator.java
new file mode 100644
index 0000000..50850c0
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/ComponentIdEvaluator.java
@@ -0,0 +1,23 @@
+package org.apache.nifi.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class ComponentIdEvaluator implements OperandEvaluator<String> {
+
+ @Override
+ public String evaluate(final ProvenanceEventRecord record) {
+ return record.getComponentId();
+ }
+
+ @Override
+ public Class<String> getType() {
+ return String.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.COMPONENT_ID;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/ComponentTypeEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/ComponentTypeEvaluator.java
new file mode 100644
index 0000000..5455055
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/ComponentTypeEvaluator.java
@@ -0,0 +1,39 @@
+/*
+ * 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.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class ComponentTypeEvaluator implements OperandEvaluator<String> {
+
+ @Override
+ public String evaluate(final ProvenanceEventRecord record) {
+ return record.getComponentType();
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.COMPONENT_TYPE;
+ }
+
+ @Override
+ public Class<String> getType() {
+ return String.class;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/RelationshipEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/RelationshipEvaluator.java
new file mode 100644
index 0000000..a6ff59d
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/RelationshipEvaluator.java
@@ -0,0 +1,23 @@
+package org.apache.nifi.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class RelationshipEvaluator implements OperandEvaluator<String> {
+
+ @Override
+ public String evaluate(final ProvenanceEventRecord record) {
+ return record.getRelationship();
+ }
+
+ @Override
+ public Class<String> getType() {
+ return String.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.RELATIONSHIP;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/SizeEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/SizeEvaluator.java
new file mode 100644
index 0000000..052b88e
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/SizeEvaluator.java
@@ -0,0 +1,23 @@
+package org.apache.nifi.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class SizeEvaluator implements OperandEvaluator<Long> {
+
+ @Override
+ public Long evaluate(final ProvenanceEventRecord record) {
+ return record.getFileSize();
+ }
+
+ @Override
+ public Class<Long> getType() {
+ return Long.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.FILESIZE;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TimestampEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TimestampEvaluator.java
new file mode 100644
index 0000000..fcc1ea1
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TimestampEvaluator.java
@@ -0,0 +1,26 @@
+package org.apache.nifi.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class TimestampEvaluator implements OperandEvaluator<Long> {
+
+ @Override
+ public Long evaluate(final ProvenanceEventRecord record) {
+ if ( record == null ) {
+ return null;
+ }
+ return record.getEventTime();
+ }
+
+ @Override
+ public Class<Long> getType() {
+ return Long.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.TIMESTAMP;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TransitUriEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TransitUriEvaluator.java
new file mode 100644
index 0000000..35e4a39
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TransitUriEvaluator.java
@@ -0,0 +1,23 @@
+package org.apache.nifi.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class TransitUriEvaluator implements OperandEvaluator<String> {
+
+ @Override
+ public String evaluate(final ProvenanceEventRecord record) {
+ return record.getTransitUri();
+ }
+
+ @Override
+ public Class<String> getType() {
+ return String.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.TRANSIT_URI;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TypeEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TypeEvaluator.java
new file mode 100644
index 0000000..2550bc3
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/TypeEvaluator.java
@@ -0,0 +1,24 @@
+package org.apache.nifi.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+public class TypeEvaluator implements OperandEvaluator<ProvenanceEventType> {
+
+ @Override
+ public ProvenanceEventType evaluate(final ProvenanceEventRecord record) {
+ return record.getEventType();
+ }
+
+ @Override
+ public Class<ProvenanceEventType> getType() {
+ return ProvenanceEventType.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.TYPE;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/UuidEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/UuidEvaluator.java
new file mode 100644
index 0000000..39d8243
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/extraction/UuidEvaluator.java
@@ -0,0 +1,39 @@
+/*
+ * 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.pql.evaluation.extraction;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class UuidEvaluator implements OperandEvaluator<String> {
+
+ @Override
+ public String evaluate(final ProvenanceEventRecord record) {
+ return record.getFlowFileUuid();
+ }
+
+ @Override
+ public Class<String> getType() {
+ return String.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.UUID;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/function/TimeFieldEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/function/TimeFieldEvaluator.java
new file mode 100644
index 0000000..373fc72
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/function/TimeFieldEvaluator.java
@@ -0,0 +1,64 @@
+package org.apache.nifi.pql.evaluation.function;
+
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class TimeFieldEvaluator implements OperandEvaluator<Long> {
+ private final OperandEvaluator<Long> timeExtractor;
+ private final int evaluatorType;
+
+ private final List<Integer> fieldsToClear = new ArrayList<>();
+
+ public TimeFieldEvaluator(final OperandEvaluator<Long> timeExtractor, final int timeField, final int evaluatorType) {
+ this.timeExtractor = timeExtractor;
+ this.evaluatorType = evaluatorType;
+
+ // note the case statements below are designed to "bleed through."
+ // I.e., if time field is YEAR, we want to clear all of the fields starting with month.
+ switch (timeField) {
+ case Calendar.YEAR:
+ fieldsToClear.add(Calendar.MONTH);
+ case Calendar.MONTH:
+ fieldsToClear.add(Calendar.DAY_OF_MONTH);
+ case Calendar.DAY_OF_MONTH:
+ fieldsToClear.add(Calendar.HOUR);
+ case Calendar.HOUR:
+ fieldsToClear.add(Calendar.MINUTE);
+ case Calendar.MINUTE:
+ fieldsToClear.add(Calendar.SECOND);
+ default:
+ fieldsToClear.add(Calendar.MILLISECOND);
+ }
+ }
+
+ @Override
+ public Long evaluate(ProvenanceEventRecord record) {
+ final Long epochMillis = timeExtractor.evaluate(record);
+ if ( epochMillis == null ) {
+ return null;
+ }
+
+ final Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+ cal.setTimeInMillis(epochMillis);
+ for ( final Integer field : fieldsToClear ) {
+ cal.set(field, 0);
+ }
+ return Long.valueOf(cal.getTimeInMillis());
+ }
+
+ @Override
+ public Class<Long> getType() {
+ return Long.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return evaluatorType;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/literals/LongLiteralEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/literals/LongLiteralEvaluator.java
new file mode 100644
index 0000000..ee63c48
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/literals/LongLiteralEvaluator.java
@@ -0,0 +1,27 @@
+package org.apache.nifi.pql.evaluation.literals;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class LongLiteralEvaluator implements OperandEvaluator<Long> {
+ private final Long value;
+
+ public LongLiteralEvaluator(final Long value) {
+ this.value = value;
+ }
+
+ @Override
+ public Long evaluate(final ProvenanceEventRecord record) {
+ return value;
+ }
+
+ @Override
+ public Class<Long> getType() {
+ return Long.class;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.NUMBER;
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/literals/StringLiteralEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/literals/StringLiteralEvaluator.java
new file mode 100644
index 0000000..d6a09f1
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/literals/StringLiteralEvaluator.java
@@ -0,0 +1,33 @@
+package org.apache.nifi.pql.evaluation.literals;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class StringLiteralEvaluator implements OperandEvaluator<String> {
+ private final String value;
+
+ public StringLiteralEvaluator(final String value) {
+ this.value = value;
+ }
+
+ @Override
+ public String evaluate(final ProvenanceEventRecord record) {
+ return value;
+ }
+
+ @Override
+ public Class<String> getType() {
+ return String.class;
+ }
+
+ @Override
+ public String toString() {
+ return value;
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.STRING_LITERAL;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/logic/AndEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/logic/AndEvaluator.java
new file mode 100644
index 0000000..fb9b408
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/logic/AndEvaluator.java
@@ -0,0 +1,114 @@
+package org.apache.nifi.pql.evaluation.logic;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class AndEvaluator implements BooleanEvaluator {
+
+ private final BooleanEvaluator lhs;
+ private final BooleanEvaluator rhs;
+
+
+ public AndEvaluator(final BooleanEvaluator lhs, final BooleanEvaluator rhs) {
+ this.lhs = lhs;
+ this.rhs = rhs;
+ }
+
+ @Override
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ return lhs.evaluate(record) && rhs.evaluate(record);
+ }
+
+ @Override
+ public BooleanEvaluator negate() {
+ return new OrEvaluator(lhs.negate(), rhs.negate());
+ }
+
+ public BooleanEvaluator getLHS() {
+ return lhs;
+ }
+
+ public BooleanEvaluator getRHS() {
+ return rhs;
+ }
+
+ /**
+ * Converts this AND tree to Disjunctive Normal Form (OR's of AND's)
+ * @return
+ */
+ public BooleanEvaluator toDNF() {
+ final List<BooleanEvaluator> rhsEvaluators = new ArrayList<>();
+ final List<BooleanEvaluator> lhsEvaluators = new ArrayList<>();
+
+ if ( rhs instanceof OrEvaluator ) {
+ final OrEvaluator or = (OrEvaluator) rhs;
+ rhsEvaluators.add(or.getLHS());
+ rhsEvaluators.add(or.getRHS());
+ } else if ( rhs instanceof AndEvaluator ) {
+ rhsEvaluators.add( ((AndEvaluator) rhs).toDNF() );
+ } else {
+ rhsEvaluators.add(rhs);
+ }
+
+ if ( lhs instanceof OrEvaluator ) {
+ final OrEvaluator or = (OrEvaluator) lhs;
+ lhsEvaluators.add(or.getLHS());
+ lhsEvaluators.add(or.getRHS());
+ } else if ( lhs instanceof AndEvaluator ) {
+ lhsEvaluators.add( ((AndEvaluator) lhs).toDNF() );
+ } else {
+ lhsEvaluators.add(lhs);
+ }
+
+ if ( rhsEvaluators.size() == 1 && lhsEvaluators.size() == 1 ) {
+ return this;
+ }
+
+ final List<AndEvaluator> ands = new ArrayList<>();
+ for ( final BooleanEvaluator l : lhsEvaluators ) {
+ for ( final BooleanEvaluator r : rhsEvaluators ) {
+ final AndEvaluator and = new AndEvaluator(l, r);
+ ands.add(and);
+ }
+ }
+
+ final AndEvaluator and1 = ands.get(0);
+ final AndEvaluator and2 = ands.get(1);
+ OrEvaluator or = new OrEvaluator(and1, and2);
+
+ for (int i=2; i < ands.size(); i++) {
+ final AndEvaluator ae = ands.get(i);
+ or = new OrEvaluator(or, ae);
+ }
+
+ return or;
+ }
+
+ public String toString() {
+ final StringBuilder sb = new StringBuilder();
+ if ( lhs instanceof AndEvaluator || lhs instanceof OrEvaluator ) {
+ sb.append("(").append(lhs.toString()).append(")");
+ } else {
+ sb.append(lhs.toString());
+ }
+
+ sb.append(" & ");
+
+ if ( rhs instanceof AndEvaluator || rhs instanceof OrEvaluator ) {
+ sb.append("(").append(rhs.toString()).append(")");
+ } else {
+ sb.append(rhs.toString());
+ }
+
+ return sb.toString();
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.AND;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/logic/OrEvaluator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/logic/OrEvaluator.java
new file mode 100644
index 0000000..c93462a
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/logic/OrEvaluator.java
@@ -0,0 +1,58 @@
+package org.apache.nifi.pql.evaluation.logic;
+
+import org.apache.nifi.pql.evaluation.BooleanEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class OrEvaluator implements BooleanEvaluator {
+ private final BooleanEvaluator lhs;
+ private final BooleanEvaluator rhs;
+
+ public OrEvaluator(final BooleanEvaluator lhs, final BooleanEvaluator rhs) {
+ this.lhs = lhs;
+ this.rhs = rhs;
+ }
+
+ @Override
+ public Boolean evaluate(final ProvenanceEventRecord record) {
+ return lhs.evaluate(record) || rhs.evaluate(record);
+ }
+
+ @Override
+ public BooleanEvaluator negate() {
+ return new AndEvaluator(lhs.negate(), rhs.negate());
+ }
+
+ public BooleanEvaluator getLHS() {
+ return lhs;
+ }
+
+ public BooleanEvaluator getRHS() {
+ return rhs;
+ }
+
+
+ public String toString() {
+ final StringBuilder sb = new StringBuilder();
+ if ( lhs instanceof AndEvaluator || lhs instanceof OrEvaluator ) {
+ sb.append("(").append(lhs.toString()).append(")");
+ } else {
+ sb.append(lhs.toString());
+ }
+
+ sb.append(" | ");
+
+ if ( rhs instanceof AndEvaluator || rhs instanceof OrEvaluator ) {
+ sb.append("(").append(rhs.toString()).append(")");
+ } else {
+ sb.append(rhs.toString());
+ }
+
+ return sb.toString();
+ }
+
+ @Override
+ public int getEvaluatorType() {
+ return org.apache.nifi.pql.ProvenanceQueryParser.OR;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/CellValue.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/CellValue.java
new file mode 100644
index 0000000..724e340
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/CellValue.java
@@ -0,0 +1,69 @@
+package org.apache.nifi.pql.evaluation.order;
+
+import java.util.Comparator;
+
+public class CellValue<T> implements Comparable<CellValue<T>> {
+ private final T value;
+ private final int rowId;
+ private final Comparator<T> valueComparator;
+
+ public CellValue(final T value, final int rowId, final Comparator<T> valueComparator) {
+ this.value = value;
+ this.rowId = rowId;
+ this.valueComparator = valueComparator;
+ }
+
+ public T getValue() {
+ return value;
+ }
+
+ public int getRowId() {
+ return rowId;
+ }
+
+ @Override
+ public int compareTo(final CellValue<T> other) {
+ if ( other == null ) {
+ return 1;
+ }
+
+ if ( this == other ) {
+ return 0;
+ }
+
+ return valueComparator.compare(value, other.value);
+ }
+
+ @Override
+ public String toString() {
+ return value.toString();
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((value == null) ? 0 : value.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+
+ @SuppressWarnings("rawtypes")
+ CellValue other = (CellValue) obj;
+ if (value == null) {
+ if (other.value != null)
+ return false;
+ } else if (!value.equals(other.value))
+ return false;
+ return true;
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/FieldSorter.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/FieldSorter.java
new file mode 100644
index 0000000..b306b53
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/FieldSorter.java
@@ -0,0 +1,87 @@
+package org.apache.nifi.pql.evaluation.order;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.nifi.pql.evaluation.OperandEvaluator;
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class FieldSorter implements RowSorter {
+
+ private final SortedSet<CellValue<ProvenanceEventRecord>> values = new TreeSet<>();
+ private final MultiFieldComparator comparator;
+
+ public FieldSorter(final Map<OperandEvaluator<?>, SortDirection> fieldEvaluators) {
+ comparator = new MultiFieldComparator(fieldEvaluators);
+ }
+
+
+ @Override
+ public void add(final ProvenanceEventRecord record, final Group group, final int rowId) {
+ values.add(new CellValue<ProvenanceEventRecord>(record, rowId, comparator));
+ }
+
+ @Override
+ public List<Integer> sort() {
+ final List<Integer> rowIds = new ArrayList<>();
+ for ( final CellValue<?> value : values ) {
+ rowIds.add( value.getRowId() );
+ }
+ return rowIds;
+ }
+
+
+
+ private static class MultiFieldComparator implements Comparator<ProvenanceEventRecord> {
+ private final Map<OperandEvaluator<?>, SortDirection> evals;
+ private final Comparator<Number> numberComparator = Sorters.newNumberComparator();
+ private final Comparator<Object> objectComparator = Sorters.newObjectComparator();
+
+ public MultiFieldComparator(final Map<OperandEvaluator<?>, SortDirection> evals) {
+ this.evals = evals;
+ }
+
+ @Override
+ public int compare(final ProvenanceEventRecord r1, final ProvenanceEventRecord r2) {
+ if ( r1 == r2 ) {
+ return 0;
+ }
+ if (r1 == null && r2 == null) {
+ return 0;
+ }
+ if (r1 == null) {
+ return -1;
+ }
+ if (r2 == null) {
+ return 1;
+ }
+
+ for ( final Map.Entry<OperandEvaluator<?>, SortDirection> entry : evals.entrySet() ) {
+ final OperandEvaluator<?> eval = entry.getKey();
+ final SortDirection dir = entry.getValue();
+
+ int comparisonResult;
+
+ final Object v1 = eval.evaluate(r1);
+ final Object v2 = eval.evaluate(r2);
+
+ if ( Number.class.isAssignableFrom(eval.getType()) ) {
+ comparisonResult = numberComparator.compare((Number) v1, (Number) v2);
+ } else {
+ comparisonResult = objectComparator.compare(v1, v2);
+ }
+
+ if ( comparisonResult != 0 ) {
+ return dir == SortDirection.ASC ? comparisonResult : -comparisonResult;
+ }
+ }
+
+ return 0;
+ }
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/GroupedSorter.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/GroupedSorter.java
new file mode 100644
index 0000000..b54416a
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/GroupedSorter.java
@@ -0,0 +1,123 @@
+package org.apache.nifi.pql.evaluation.order;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class GroupedSorter implements RowSorter {
+ private final Map<Accumulator<?>, SortDirection> accumulators;
+ private final Map<Group, Integer> firstGroupOccurrence = new HashMap<>();
+ private final Comparator<Group> comparator;
+
+ private final Set<CellValue<Group>> values = new HashSet<>();
+
+ public GroupedSorter(final Map<Accumulator<?>, SortDirection> accumulators) {
+ this.accumulators = accumulators;
+ comparator = new GroupedComparator(accumulators);
+ }
+
+ @Override
+ public void add(final ProvenanceEventRecord record, final Group group, final int rowId) {
+ if ( !firstGroupOccurrence.containsKey(group) ) {
+ firstGroupOccurrence.put(group, firstGroupOccurrence.size());
+ }
+
+ for ( final Accumulator<?> accum : accumulators.keySet() ) {
+ accum.accumulate(record, group);
+ }
+
+ values.add(new CellValue<Group>(group, firstGroupOccurrence.get(group), comparator));
+ }
+
+ @Override
+ public List<Integer> sort() {
+ final List<CellValue<Group>> sortedGroups = new ArrayList<>();
+ for ( final CellValue<Group> value : values ) {
+ sortedGroups.add(value);
+ }
+
+ Collections.sort(sortedGroups);
+
+ final List<Integer> sorted = new ArrayList<>(values.size());
+
+ for ( final CellValue<Group> value : sortedGroups ) {
+ sorted.add( value.getRowId() );
+ }
+
+ return sorted;
+ }
+
+
+ private static class GroupedComparator implements Comparator<Group> {
+ private final Map<Accumulator<?>, SortDirection> map;
+
+ public GroupedComparator(final Map<Accumulator<?>, SortDirection> map) {
+ this.map = map;
+ }
+
+ @Override
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ public int compare(final Group r1, final Group r2) {
+ if ( r1 == r2 ) {
+ return 0;
+ }
+ if (r1 == null && r2 == null) {
+ return 0;
+ }
+ if (r1 == null) {
+ return -1;
+ }
+ if (r2 == null) {
+ return 1;
+ }
+ if ( r1.equals(r2) ) {
+ return 0;
+ }
+
+ for ( final Map.Entry<Accumulator<?>, SortDirection> entry : map.entrySet() ) {
+ final Accumulator<?> accumulator = entry.getKey();
+ final SortDirection dir = entry.getValue();
+
+ final List<Object> rowValues1 = (List<Object>) accumulator.getValues(r1);
+ final List<Object> rowValues2 = (List<Object>) accumulator.getValues(r2);
+
+ if ( rowValues1.size() > rowValues2.size() ) {
+ return -1;
+ } else if ( rowValues2.size() > rowValues1.size() ) {
+ return 1;
+ }
+
+ for (int i=0; i < rowValues1.size(); i++) {
+ final Object v1 = rowValues1.get(i);
+ final Object v2 = rowValues2.get(i);
+
+ int comparisonResult;
+
+ if ( Number.class.isAssignableFrom(v1.getClass()) ) {
+ final Comparator comparator = Sorters.newNumberComparator();
+ comparisonResult = comparator.compare((Number) v1, (Number) v2);
+ } else {
+ final Comparator comparator = Sorters.newObjectComparator();
+ comparisonResult = comparator.compare(v1, v2);
+ }
+
+ if ( comparisonResult != 0 ) {
+ return dir == SortDirection.ASC ? comparisonResult : -comparisonResult;
+ }
+ }
+ }
+
+ return 0;
+ }
+
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/RowSorter.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/RowSorter.java
new file mode 100644
index 0000000..a44d2bf
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/RowSorter.java
@@ -0,0 +1,13 @@
+package org.apache.nifi.pql.evaluation.order;
+
+import java.util.List;
+
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface RowSorter {
+
+ void add(ProvenanceEventRecord record, Group group, int rowId);
+
+ List<Integer> sort();
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/SortDirection.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/SortDirection.java
new file mode 100644
index 0000000..bf17bcc
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/SortDirection.java
@@ -0,0 +1,6 @@
+package org.apache.nifi.pql.evaluation.order;
+
+public enum SortDirection {
+ ASC,
+ DESC;
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/Sorters.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/Sorters.java
new file mode 100644
index 0000000..63cd6bc
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/order/Sorters.java
@@ -0,0 +1,74 @@
+package org.apache.nifi.pql.evaluation.order;
+
+import java.text.Collator;
+import java.util.Comparator;
+
+public class Sorters {
+
+ public static Comparator<Number> newNumberComparator() {
+ return new NumberComparator();
+ }
+
+ public static Comparator<Object> newObjectComparator() {
+ return new ObjectComparator();
+ }
+
+
+ private static class NumberComparator implements Comparator<Number> {
+ @Override
+ public int compare(Number o1, Number o2) {
+ if (o1 == o2) {
+ return 0;
+ }
+
+ if (o1 == null && o2 == null) {
+ return 0;
+ }
+
+ if (o1 == null) {
+ return -1;
+ }
+
+ if (o2 == null) {
+ return 1;
+ }
+
+ if (o1.doubleValue() < o2.doubleValue()) {
+ return -1;
+ }
+
+ if (o1.doubleValue() > o2.doubleValue()) {
+ return 1;
+ }
+
+ return 0;
+ }
+
+ }
+
+ private static class ObjectComparator implements Comparator<Object> {
+ private final Collator collator = Collator.getInstance();
+
+ @Override
+ public int compare(final Object o1, final Object o2) {
+ if ( o1 == o2 ) {
+ return 0;
+ }
+
+ if (o1 == null && o2 == null) {
+ return 0;
+ }
+
+ if (o1 == null) {
+ return -1;
+ }
+
+ if (o2 == null) {
+ return 1;
+ }
+
+ return collator.compare(o1.toString(), o2.toString());
+ }
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/repository/SelectAllRecords.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/repository/SelectAllRecords.java
new file mode 100644
index 0000000..e1f8df8
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/evaluation/repository/SelectAllRecords.java
@@ -0,0 +1,61 @@
+package org.apache.nifi.pql.evaluation.repository;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.nifi.pql.evaluation.RepositoryEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+
+public class SelectAllRecords implements RepositoryEvaluator {
+
+ public Iterator<StoredProvenanceEvent> evaluate(final ProvenanceEventRepository repository) throws IOException {
+ final int maxRecords = 10000;
+
+ return new Iterator<StoredProvenanceEvent>() {
+ long iterated = 0;
+ long fetched = 0;
+
+ List<StoredProvenanceEvent> records = null;
+ Iterator<StoredProvenanceEvent> listItr = null;
+
+ private void ensureIterator() {
+ if ( listItr == null || !listItr.hasNext() ) {
+ try {
+ records = repository.getEvents(fetched, maxRecords);
+ } catch (final IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+
+ listItr = records.iterator();
+ fetched += records.size();
+ }
+ }
+
+ public boolean hasNext() {
+ ensureIterator();
+ return listItr.hasNext();
+ }
+
+ public StoredProvenanceEvent next() {
+ if ( !hasNext() ) {
+ throw new NoSuchElementException();
+ }
+
+ if ( iterated++ == fetched ) {
+ records = null;
+ listItr = null;
+ }
+
+ return listItr.next();
+ }
+
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/exception/ProvenanceQueryLanguageException.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/exception/ProvenanceQueryLanguageException.java
new file mode 100644
index 0000000..87ca2c1
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/exception/ProvenanceQueryLanguageException.java
@@ -0,0 +1,23 @@
+package org.apache.nifi.pql.exception;
+
+public class ProvenanceQueryLanguageException extends RuntimeException {
+
+ private static final long serialVersionUID = 1L;
+
+ public ProvenanceQueryLanguageException() {
+ super();
+ }
+
+ public ProvenanceQueryLanguageException(final String message) {
+ super(message);
+ }
+
+ public ProvenanceQueryLanguageException(final Throwable cause) {
+ super(cause);
+ }
+
+ public ProvenanceQueryLanguageException(final String message, final Throwable cause) {
+ super(message, cause);
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/exception/ProvenanceQueryLanguageParsingException.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/exception/ProvenanceQueryLanguageParsingException.java
new file mode 100644
index 0000000..88426d3
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/exception/ProvenanceQueryLanguageParsingException.java
@@ -0,0 +1,23 @@
+package org.apache.nifi.pql.exception;
+
+public class ProvenanceQueryLanguageParsingException extends ProvenanceQueryLanguageException {
+
+ private static final long serialVersionUID = 1L;
+
+ public ProvenanceQueryLanguageParsingException() {
+ super();
+ }
+
+ public ProvenanceQueryLanguageParsingException(final String message) {
+ super(message);
+ }
+
+ public ProvenanceQueryLanguageParsingException(final Throwable cause) {
+ super(cause);
+ }
+
+ public ProvenanceQueryLanguageParsingException(final String message, final Throwable cause) {
+ super(message, cause);
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/groups/Group.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/groups/Group.java
new file mode 100644
index 0000000..35ee93b
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/groups/Group.java
@@ -0,0 +1,57 @@
+package org.apache.nifi.pql.groups;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class Group {
+
+ private final List<Object> values;
+ private final int hashCode;
+
+ public Group(final Object... values) {
+ this(Arrays.asList(values));
+ }
+
+ public Group(final List<Object> values) {
+ this.values = new ArrayList<>(values);
+
+ int prime = 23497;
+ int hc = 1;
+ for ( final Object o : values ) {
+ hc = prime * hc + (o == null ? 0 : o.hashCode());
+ }
+
+ this.hashCode = hc;
+ }
+
+ @Override
+ public int hashCode() {
+ return hashCode;
+ }
+
+ @Override
+ public boolean equals(final Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+
+ Group other = (Group) obj;
+ if (hashCode != other.hashCode)
+ return false;
+ if (values == null) {
+ if (other.values != null)
+ return false;
+ } else if (!values.equals(other.values))
+ return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return values == null ? "Default Group" : values.toString();
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/groups/Grouper.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/groups/Grouper.java
new file mode 100644
index 0000000..3ce667b
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/groups/Grouper.java
@@ -0,0 +1,20 @@
+package org.apache.nifi.pql.groups;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public class Grouper {
+
+ public static Group group(final ProvenanceEventRecord record, final List<RecordEvaluator<?>> evaluators) {
+ final List<Object> values = new ArrayList<>(evaluators.size());
+ for ( final RecordEvaluator<?> evaluator : evaluators ) {
+ values.add(evaluator.evaluate(record));
+ }
+
+ return new Group(values);
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/GroupingResultSet.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/GroupingResultSet.java
new file mode 100644
index 0000000..53c4dca
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/GroupingResultSet.java
@@ -0,0 +1,161 @@
+package org.apache.nifi.pql.results;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.pql.evaluation.order.RowSorter;
+import org.apache.nifi.pql.groups.Group;
+import org.apache.nifi.pql.groups.Grouper;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+
+public class GroupingResultSet implements ProvenanceResultSet {
+ private final List<String> labels;
+ private final List<Class<?>> returnTypes;
+ private final Iterator<? extends StoredProvenanceEvent> recordItr;
+ private final List<Accumulator<?>> selectAccumulators;
+ private final RecordEvaluator<Boolean> sourceEvaluator;
+ private final RecordEvaluator<Boolean> conditionEvaluator;
+ private final List<RecordEvaluator<?>> groupEvaluators;
+ private final RowSorter sorter;
+ private final Long limit;
+ private long recordsReturned = 0L;
+
+ private Iterator<List<Object>> rowItr;
+
+ public GroupingResultSet(
+ final Iterator<? extends StoredProvenanceEvent> recordItr,
+ final List<Accumulator<?>> selectAccumulators,
+ final RecordEvaluator<Boolean> sourceEvaluator,
+ final RecordEvaluator<Boolean> conditionEvaluator,
+ final List<String> labels, final List<Class<?>> returnTypes, final List<RecordEvaluator<?>> groupEvaluators,
+ final RowSorter sorter,
+ final Long limit) {
+
+ this.labels = labels;
+ this.returnTypes = returnTypes;
+ this.recordItr = recordItr;
+ this.selectAccumulators = selectAccumulators;
+ this.sourceEvaluator = sourceEvaluator;
+ this.conditionEvaluator = conditionEvaluator;
+ this.groupEvaluators = groupEvaluators;
+ this.sorter = sorter;
+ this.limit = limit;
+ }
+
+
+ @Override
+ public List<String> getLabels() {
+ return labels;
+ }
+
+ @Override
+ public List<Class<?>> getReturnType() {
+ return returnTypes;
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ private void createRowItr() {
+ int recordIdx = 0;
+ while (recordItr.hasNext()) {
+ final StoredProvenanceEvent record = recordItr.next();
+
+ if ( sourceEvaluator != null && !sourceEvaluator.evaluate(record) ) {
+ continue;
+ }
+
+ final boolean meetsConditions = conditionEvaluator == null ? true : conditionEvaluator.evaluate(record);
+ if ( meetsConditions ) {
+ final Group group = groupEvaluators == null ? null : Grouper.group(record, groupEvaluators);
+
+ for ( final Accumulator<?> accumulator : selectAccumulators ) {
+ accumulator.accumulate(record, group);
+ }
+
+ if ( sorter != null ) {
+ sorter.add(record, group, recordIdx++);
+ }
+ }
+ }
+
+ // Key = Group
+ // Value = Map
+ // Key = Accumulator
+ // Value = Column values for a row
+ final Map<Group, Map<Accumulator, List<Object>>> groupedMap = new LinkedHashMap<>();
+ for ( final Accumulator<?> accumulator : selectAccumulators ) {
+ final Map<Group, List<Object>> accumulatedValues = (Map) accumulator.getValues();
+
+ // for each row returned by this accumulator...
+ for ( final Map.Entry<Group, List<Object>> entry : accumulatedValues.entrySet() ) {
+ final Group group = entry.getKey();
+
+ Map<Accumulator, List<Object>> accumulatorRows = groupedMap.get(group);
+ if ( accumulatorRows == null ) {
+ accumulatorRows = new LinkedHashMap<>();
+ groupedMap.put(group, accumulatorRows);
+ }
+ accumulatorRows.put(accumulator, accumulatedValues.get(group));
+ }
+ }
+
+ final Collection<Map<Accumulator, List<Object>>> columnCollection = groupedMap.values();
+ final List<List<Object>> rows = new ArrayList<>();
+ for ( final Map<Accumulator, List<Object>> map : columnCollection ) {
+ final List<Object> columnValues = new ArrayList<>();
+
+ int rowIdx = 0;
+ for ( final List<Object> accumulatorRows : map.values() ) {
+ if (accumulatorRows.size() <= rowIdx) {
+ break;
+ }
+
+ final Object columnVal = accumulatorRows.get(rowIdx);
+ columnValues.add(columnVal);
+ }
+
+ rowIdx++;
+ rows.add(columnValues);
+ }
+
+ final List<List<Object>> sortedRows;
+ if ( sorter == null ) {
+ sortedRows = rows;
+ } else {
+ sortedRows = new ArrayList<>(rows.size());
+
+ final List<Integer> sortedRowIds = sorter.sort();
+ for (final Integer rowId : sortedRowIds) {
+ sortedRows.add( rows.get(rowId) );
+ }
+ }
+
+ rowItr = sortedRows.iterator();
+ }
+
+
+ @Override
+ public boolean hasNext() {
+ if ( rowItr == null ) {
+ createRowItr();
+ }
+ return (limit == null || recordsReturned <= limit ) && rowItr.hasNext();
+ }
+
+ @Override
+ public List<?> next() {
+ if ( hasNext() ) {
+ return rowItr.next();
+ }
+
+ throw new NoSuchElementException();
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/OrderedResultSet.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/OrderedResultSet.java
new file mode 100644
index 0000000..c5417c5
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/OrderedResultSet.java
@@ -0,0 +1,50 @@
+package org.apache.nifi.pql.results;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.nifi.pql.evaluation.order.RowSorter;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+
+public class OrderedResultSet implements ProvenanceResultSet {
+ private final Iterator<List<?>> sortedRowItr;
+ private final ProvenanceResultSet rs;
+
+ public OrderedResultSet(final ProvenanceResultSet rs, final RowSorter sorter) {
+ this.rs = rs;
+ final List<List<?>> rows = new ArrayList<>();
+
+ while (rs.hasNext()) {
+ final List<?> colVals = rs.next();
+ rows.add(colVals);
+ }
+
+ final List<List<?>> sortedRows = new ArrayList<>(rows.size());
+ for ( final Integer rowId : sorter.sort() ) {
+ sortedRows.add(rows.get(rowId.intValue()));
+ }
+
+ sortedRowItr = sortedRows.iterator();
+ }
+
+ @Override
+ public List<String> getLabels() {
+ return rs.getLabels();
+ }
+
+ @Override
+ public List<Class<?>> getReturnType() {
+ return rs.getReturnType();
+ }
+
+ @Override
+ public boolean hasNext() {
+ return sortedRowItr.hasNext();
+ }
+
+ @Override
+ public List<?> next() {
+ return sortedRowItr.next();
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/ResultRow.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/ResultRow.java
new file mode 100644
index 0000000..f02a992
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/ResultRow.java
@@ -0,0 +1,20 @@
+package org.apache.nifi.pql.results;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class ResultRow {
+ private final List<Object> values;
+
+ public ResultRow(final Object... values) {
+ this(Arrays.asList(values));
+ }
+
+ public ResultRow(final List<Object> values) {
+ this.values = values;
+ }
+
+ public List<Object> getValues() {
+ return values;
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/RowIterator.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/RowIterator.java
new file mode 100644
index 0000000..a9202dc
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/RowIterator.java
@@ -0,0 +1,69 @@
+package org.apache.nifi.pql.results;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class RowIterator implements Iterator<ResultRow> {
+ private final Iterator<Iterator<?>> itrs;
+
+ public RowIterator(final List<Iterator<?>> itrs) {
+ this.itrs = itrs.iterator();
+ }
+
+
+ @Override
+ public boolean hasNext() {
+ return itrs.hasNext();
+ }
+
+ public ResultRow next() {
+ final Iterator<?> columnValueItr = itrs.next();
+ final List<Object> colValues = new ArrayList<>();
+ while (columnValueItr.hasNext()) {
+ colValues.add(columnValueItr.next());
+ }
+ return new ResultRow(colValues);
+ }
+
+// @Override
+// public boolean hasNext() {
+// if ( curItr == null || !curItr.hasNext() ) {
+// while (itrs.hasNext()) {
+// curItr = itrs.next();
+// if ( curItr.hasNext() ) {
+// return true;
+// }
+// }
+//
+// return false;
+// }
+//
+// return true;
+// }
+//
+// @Override
+// public T next() {
+// if ( curItr.hasNext() ) {
+// return curItr.next();
+// }
+//
+// if ( curItr == null || !curItr.hasNext() ) {
+// while (itrs.hasNext()) {
+// curItr = itrs.next();
+// if ( curItr.hasNext() ) {
+// return curItr.next();
+// }
+// }
+//
+// throw new NoSuchElementException();
+// } else {
+// return curItr.next();
+// }
+// }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/StandardOrderedResultSet.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/StandardOrderedResultSet.java
new file mode 100644
index 0000000..4bec4c0
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/StandardOrderedResultSet.java
@@ -0,0 +1,111 @@
+package org.apache.nifi.pql.results;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.pql.evaluation.order.RowSorter;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+
+public class StandardOrderedResultSet implements ProvenanceResultSet {
+ private final List<String> labels;
+ private final List<Class<?>> returnTypes;
+
+ private final Iterator<? extends StoredProvenanceEvent> recordItr;
+ private final List<Accumulator<?>> selectAccumulators;
+ private final RecordEvaluator<Boolean> sourceEvaluator;
+ private final RecordEvaluator<Boolean> conditionEvaluator;
+ private final RowSorter sorter;
+ private final Long limit;
+
+ private Iterator<ResultRow> resultRowItr;
+
+ public StandardOrderedResultSet(final Iterator<? extends StoredProvenanceEvent> recordItr,
+ final List<Accumulator<?>> selectAccumulators,
+ final RecordEvaluator<Boolean> sourceEvaluator,
+ final RecordEvaluator<Boolean> conditionEvaluator,
+ final List<String> labels,
+ final List<Class<?>> returnTypes,
+ final RowSorter sorter,
+ final Long limit)
+ {
+ this.labels = labels;
+ this.returnTypes = returnTypes;
+
+ this.recordItr = recordItr;
+ this.selectAccumulators = selectAccumulators;
+ this.sourceEvaluator = sourceEvaluator;
+ this.conditionEvaluator = conditionEvaluator;
+ this.sorter = sorter;
+ this.limit = limit;
+ }
+
+
+ @Override
+ public List<String> getLabels() {
+ return labels;
+ }
+
+ @Override
+ public List<Class<?>> getReturnType() {
+ return returnTypes;
+ }
+
+ private void createResultRowItr() {
+ final List<ResultRow> rows = new ArrayList<>();
+ int idx = 0;
+ while (recordItr.hasNext()) {
+ final ProvenanceEventRecord record = recordItr.next();
+ if ( sourceEvaluator != null && !sourceEvaluator.evaluate(record) ) {
+ continue;
+ }
+
+ final boolean meetsConditions = conditionEvaluator == null ? true : conditionEvaluator.evaluate(record);
+ if ( meetsConditions ) {
+ final List<Object> values = new ArrayList<>(selectAccumulators.size());
+ for ( final Accumulator<?> accumulator : selectAccumulators ) {
+ final Object value = accumulator.accumulate(record, null);
+ accumulator.reset();
+ values.add(value);
+ }
+ rows.add(new ResultRow(values));
+ sorter.add(record, null, idx++);
+ }
+ }
+
+ final List<ResultRow> sortedRows = new ArrayList<>();
+ for ( final Integer unsortedIndex : sorter.sort() ) {
+ final ResultRow row = rows.get(unsortedIndex.intValue());
+ sortedRows.add(row);
+
+ if ( limit != null && sortedRows.size() >= limit.intValue() ) {
+ break;
+ }
+ }
+
+ resultRowItr = sortedRows.iterator();
+ }
+
+ @Override
+ public boolean hasNext() {
+ if ( resultRowItr == null ) {
+ createResultRowItr();
+ }
+
+ return resultRowItr.hasNext();
+ }
+
+ @Override
+ public List<?> next() {
+ if ( resultRowItr == null ) {
+ createResultRowItr();
+ }
+
+ return resultRowItr.next().getValues();
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/StandardUnorderedResultSet.java b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/StandardUnorderedResultSet.java
new file mode 100644
index 0000000..019ab5c
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/java/org/apache/nifi/pql/results/StandardUnorderedResultSet.java
@@ -0,0 +1,104 @@
+package org.apache.nifi.pql.results;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.nifi.pql.evaluation.Accumulator;
+import org.apache.nifi.pql.evaluation.RecordEvaluator;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+
+public class StandardUnorderedResultSet implements ProvenanceResultSet {
+
+ private final List<String> labels;
+ private final List<Class<?>> returnTypes;
+ private final Iterator<? extends StoredProvenanceEvent> recordItr;
+ private final RecordEvaluator<Boolean> sourceEvaluator;
+ private final RecordEvaluator<Boolean> conditionEvaluator;
+ private final List<Accumulator<?>> selectAccumulators;
+ private final Long limit;
+
+ private ResultRow nextRecord;
+ private long recordsReturned = 0L;
+
+ public StandardUnorderedResultSet(final Iterator<? extends StoredProvenanceEvent> recordItr,
+ final List<Accumulator<?>> selectAccumulators,
+ final RecordEvaluator<Boolean> sourceEvaluator,
+ final RecordEvaluator<Boolean> conditionEvaluator,
+ final List<String> labels,
+ final List<Class<?>> returnTypes,
+ final Long limit)
+ {
+ this.selectAccumulators = selectAccumulators;
+ this.labels = labels;
+ this.returnTypes = returnTypes;
+ this.recordItr = recordItr;
+ this.sourceEvaluator = sourceEvaluator;
+ this.conditionEvaluator = conditionEvaluator;
+ this.limit = limit;
+ }
+
+
+ @Override
+ public List<String> getLabels() {
+ return labels;
+ }
+
+ @Override
+ public List<Class<?>> getReturnType() {
+ return returnTypes;
+ }
+
+ private boolean findNextRecord() {
+ if ( limit != null && recordsReturned >= limit.longValue() ) {
+ return false;
+ }
+
+ while (recordItr.hasNext()) {
+ final ProvenanceEventRecord record = recordItr.next();
+
+ if ( sourceEvaluator != null && !sourceEvaluator.evaluate(record) ) {
+ continue;
+ }
+
+ final boolean meetsConditions = conditionEvaluator == null ? true : conditionEvaluator.evaluate(record);
+ if ( meetsConditions ) {
+ final List<Object> values = new ArrayList<>(selectAccumulators.size());
+ for ( final Accumulator<?> accumulator : selectAccumulators ) {
+ final Object value = accumulator.accumulate(record, null);
+ accumulator.reset();
+ values.add(value);
+ }
+ this.nextRecord = new ResultRow(values);
+ recordsReturned++;
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ @Override
+ public boolean hasNext() {
+ if ( nextRecord != null ) {
+ return true;
+ }
+
+ return findNextRecord();
+ }
+
+ @Override
+ public List<?> next() {
+ if ( hasNext() ) {
+ final List<?> value = nextRecord.getValues();
+ nextRecord = null;
+ return value;
+ }
+
+ throw new NoSuchElementException();
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/resources/docs/examples.pql b/nifi/nifi-commons/nifi-provenance-query-language/src/main/resources/docs/examples.pql
new file mode 100644
index 0000000..1f0b12c
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/resources/docs/examples.pql
@@ -0,0 +1,48 @@
+# Select all events that occurred in the last hour
+SELECT Event
+FROM *
+WHERE
+ Event.Time > 1 HOUR AGO
+
+
+# Count the number of RECEIVE events from myHost.myDomain in the last hour
+SELECT COUNT(*)
+FROM RECEIVE
+WHERE
+ Event.Time WITHIN 1 HOUR
+ AND
+ Event.TransitUri MATCHES '.*myHost\.myDomain.*'
+
+
+# Count the number of bytes received from myHost.myDomain in the last hour
+SELECT SUM(FileSize)
+FROM RECEIVE
+WHERE
+ Event.Time WITHIN 1 HOUR
+ AND
+ Event.TransitUri MATCHES '.*myHost\.MyDomain.*'
+
+
+# Count the number of objects and number of bytes sent to each host on August 1, 2014, but only for the 10 hosts that we send to the most
+SELECT TransitUri, COUNT(*) as numFiles, SUM(FileSize) as fileSize
+FROM SEND
+WHERE
+ Event.Time BETWEEN '08/01/2014 00:00:00.000' AND '08/01/2014 23:59:59.999'
+GROUP BY
+ Event.TransitUri
+ORDER BY
+ fileSize
+LIMIT 10
+
+
+
+
+# Select the filename and transit URI of any SEND event that occurred as the result of receiving a filename that ends with '.txt' in the last hour:
+DEFINE LINEAGE L:
+ RECEIVE R{
+ R['filename'] MATCHES '.*\.txt'
+ AND
+ R.Time > 1 HOUR AGO
+ } --> SEND S
+SELECT S.Time, S['filename'], S.TransitUri
+FROM LINEAGE L
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/main/resources/docs/implementation-notes.txt b/nifi/nifi-commons/nifi-provenance-query-language/src/main/resources/docs/implementation-notes.txt
new file mode 100644
index 0000000..e75786f
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/main/resources/docs/implementation-notes.txt
@@ -0,0 +1,52 @@
+ORDER BY
+ - Very Difficult.
+ - Does not have anything to do with what is selected
+ - Need to do an External Sort:
+ - Break records into groups of 10,000.
+ - Sort each group of 10,000 records.
+ - Write out those 10,000 records.
+ - Read in the first X number of records where X = 10,000 / number of files
+ - Merge the heads of the records.
+ - Algorithm is on Wikipedia. "External Sort"
+ - Similar to what we do in WALI when we order the Transactions by Transaction ID across multiple partitions.
+
+
+GROUP BY
+ - Implement in the Accumulator. May make sense to break Accumulator into two Interfaces:
+
+ GroupingAccumulator:
+ T accumulate(ProvenanceEventRecord record, Group group)
+ UngroupedAccumulator:
+ T accumulate(ProvenanceEventRecord record)
+
+ Then the GroupingAccumulator will simply map a group to the appropriate UngroupedAccumulator and then call #accumulate.
+ UngroupedAccumulator will never be used except for the GroupingAccumulator delegating to the appropriate UngroupedAccumulator.
+
+
+WHERE
+ - All functions must be able to be done in Lucene.
+
+EventAccumulator
+ - Should store provenance event location instead of event. Regardless of whether a field was selected or the entire event.
+
+
+
+
+
+
+
+
+Prov Repo:
+ - Allow ANDs and ORs in queries?
+ - ProvenanceEventRecord should return Location object. Location is a marker interface and the specific implementation will
+ to be used will depend on the repo. For example, VolatileProvenanceRepository would return something like "int getIndex()" and "long getId()"
+ so that we can get the event at the specified index and return null unless that event's id is equal to the result of calling 'getId()'.
+ Persistent Prov Repo would return a Location that includes filename & offset. Perhaps also a record index so that we can add multiple
+ records to a single repo update (byte offset of 'transaction' is 1000 and record offset into transaction is 4). This would be used
+ so that if we do an update with 100 records and all have similar fields (component type, component id, most previous attributes?), then we
+ can write that out once. This should probably be a new data structure that wraps a ProvenanceEventRecord:
+ StoredProvenanceEvent
+ ProvenanceEventRecord getEventRecord()
+ Location getLocation()
+ - Index all attributes and properties always? At least allow property value to be "*" to indicate all.
+
\ No newline at end of file
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/test/java/org/apache/nifi/pql/TestQuery.java b/nifi/nifi-commons/nifi-provenance-query-language/src/test/java/org/apache/nifi/pql/TestQuery.java
new file mode 100644
index 0000000..3317c21
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/test/java/org/apache/nifi/pql/TestQuery.java
@@ -0,0 +1,348 @@
+package org.apache.nifi.pql;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.VolatileProvenanceRepository;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestQuery {
+
+ private ProvenanceEventRepository repo;
+
+ @Before
+ public void setup() {
+ System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, "src/test/resources/nifi.properties");
+ repo = new VolatileProvenanceRepository();
+ }
+
+
+ private void createRecords() throws IOException {
+ final Map<String, String> previousAttributes = new HashMap<>();
+ previousAttributes.put("filename", "xyz");
+
+ final Map<String, String> updatedAttributes = new HashMap<>();
+ updatedAttributes.put("filename", "xyz.txt");
+ updatedAttributes.put("mime.type", "text/plain");
+
+ final StandardProvenanceEventRecord.Builder recordBuilder = new StandardProvenanceEventRecord.Builder();
+ recordBuilder.setAttributes(previousAttributes, Collections.<String, String>emptyMap())
+ .setComponentId("000")
+ .setComponentType("MyComponent")
+ .setEventType(ProvenanceEventType.RECEIVE)
+ .setFlowFileEntryDate(System.currentTimeMillis())
+ .setFlowFileUUID("1234")
+ .setTransitUri("https://localhost:80/nifi");
+
+
+ recordBuilder.setCurrentContentClaim("container", "section", "1", 0L, 100L);
+ repo.registerEvent(recordBuilder.build());
+
+
+ recordBuilder.setAttributes(previousAttributes, updatedAttributes);
+ recordBuilder.setCurrentContentClaim("container", "section", "2", 0L, 1024 * 1024L);
+ repo.registerEvent(recordBuilder.build());
+ }
+
+
+ private void createRecords(final int records, final ProvenanceEventType type, final long sleep) throws IOException {
+ final Map<String, String> previousAttributes = new HashMap<>();
+ previousAttributes.put("filename", "xyz");
+
+ final Map<String, String> updatedAttributes = new HashMap<>();
+ updatedAttributes.put("filename", "xyz.txt");
+ updatedAttributes.put("mime.type", "text/plain");
+
+ final StandardProvenanceEventRecord.Builder recordBuilder = new StandardProvenanceEventRecord.Builder();
+ recordBuilder.setAttributes(previousAttributes, Collections.<String, String>emptyMap())
+ .setComponentType("MyComponent")
+ .setEventType(type)
+ .setFlowFileEntryDate(System.currentTimeMillis())
+ .setFlowFileUUID("1234")
+ .setTransitUri("https://localhost:80/nifi");
+
+ final long now = System.currentTimeMillis();
+ for (int i=0; i < records; i++) {
+ recordBuilder.setCurrentContentClaim("container", "section", String.valueOf(i), 0L, 100L);
+ final Map<String, String> attr = new HashMap<>(updatedAttributes);
+ attr.put("i", String.valueOf(i));
+ recordBuilder.setAttributes(previousAttributes, attr);
+ recordBuilder.setFlowFileEntryDate(System.currentTimeMillis());
+ recordBuilder.setEventTime(now + (i * sleep));
+ recordBuilder.setComponentId(UUID.randomUUID().toString());
+
+ repo.registerEvent(recordBuilder.build());
+ }
+ }
+
+ @Test
+ public void testCompilationManually() {
+ System.out.println(ProvenanceQuery.compile("SELECT R.TransitUri FROM *", null, null));
+ System.out.println(ProvenanceQuery.compile("SELECT R['filename'] FROM RECEIVE, SEND;", null, null));
+ System.out.println(ProvenanceQuery.compile("SELECT Event FROM RECEIVE ORDER BY Event['filename'];", null, null));
+
+// System.out.println(Query.compile("SELECT Event FROM RECEIVE WHERE ((Event.TransitUri <> 'http') OR (Event['filename'] = '1.txt')) and (Event.Size > 1000 or Event.Size between 1 AND 4);"));
+
+ System.out.println(ProvenanceQuery.compile("SELECT SUM(Event.size) FROM RECEIVE", null, null));
+ }
+
+
+ @Test
+ public void testSumAverage() throws IOException {
+ createRecords();
+ dump(ProvenanceQuery.execute("SELECT Event", repo));
+
+ final ProvenanceQuery query = ProvenanceQuery.compile("SELECT SUM(Event.Size), AVG(Event.Size) FROM RECEIVE WHERE Event.TransitUri = 'https://localhost:80/nifi'", null, null);
+
+ final ProvenanceResultSet rs = query.execute(repo);
+ dump(rs);
+
+ dump(ProvenanceQuery.execute("SELECT Event.TransitUri", repo));
+ dump(ProvenanceQuery.execute("SELECT Event['mime.type'], Event['filename']", repo));
+ dump(ProvenanceQuery.execute("SELECT Event['filename'], SUM(Event.size) GROUP BY Event['filename']", repo));
+ }
+
+
+ @Test
+ public void testGroupBy() throws IOException {
+ createRecords(200000, ProvenanceEventType.RECEIVE, 0L);
+ createRecords(2, ProvenanceEventType.SEND, 0L);
+
+ ProvenanceResultSet rs = ProvenanceQuery.execute("SELECT Event['filename'], COUNT(Event), Event.Type GROUP BY Event['filename'], Event.Type", repo);
+ dump(rs);
+
+ rs = ProvenanceQuery.execute("SELECT Event['filename'], COUNT(Event), Event.Type GROUP BY Event['filename'], Event.Type", repo);
+
+ int receiveRows = 0;
+ int sendRows = 0;
+ while (rs.hasNext()) {
+ final List<?> cols = rs.next();
+ final ProvenanceEventType type = (ProvenanceEventType) cols.get(2);
+ if ( type == ProvenanceEventType.RECEIVE ) {
+ receiveRows++;
+ assertEquals("xyz.txt", cols.get(0));
+ assertEquals(200000L, cols.get(1));
+ } else if ( type == ProvenanceEventType.SEND ) {
+ sendRows++;
+ assertEquals("xyz.txt", cols.get(0));
+ assertEquals(2L, cols.get(1));
+ } else {
+ Assert.fail("Event type was " + type);
+ }
+ }
+
+ assertEquals(1, receiveRows);
+ assertEquals(1, sendRows);
+ }
+
+
+ @Test
+ public void testAverageGroupBy() throws IOException {
+ createRecords(200000, ProvenanceEventType.RECEIVE, 1L);
+ createRecords(5000, ProvenanceEventType.SEND, 1L);
+
+ dump(ProvenanceQuery.execute("SELECT AVG(Event.Size), Event.Type GROUP BY SECOND(Event.Time), Event.Type", repo));
+ }
+
+
+ @Test
+ public void testSelectSeveralRecords() throws IOException {
+ createRecords(2000, ProvenanceEventType.SEND, 1L);
+ createRecords(200, ProvenanceEventType.RECEIVE, 1L);
+ dump(ProvenanceQuery.execute(
+ "SELECT SECOND(Event.Time), Event.Type, SUM(Event.Size), COUNT(Event) "
+ + "FROM SEND, RECEIVE "
+ + "GROUP BY SECOND(Event.Time), Event.Type"
+ , repo));
+ }
+
+ @Test
+ public void testNot() throws IOException {
+ createRecords(2000, ProvenanceEventType.SEND, 0L);
+ createRecords(200, ProvenanceEventType.RECEIVE, 0L);
+
+ dump(ProvenanceQuery.execute("SELECT Event.Type, COUNT(Event) WHERE NOT(Event.Type = 'SEND')", repo));
+ dump(ProvenanceQuery.execute("SELECT Event.Type, COUNT(Event) WHERE NOT(Event.Type = 'RECEIVE')", repo));
+ dump(ProvenanceQuery.execute("SELECT Event.Type, COUNT(Event) WHERE NOT(NOT( Event.Type = 'SEND'))", repo));
+ }
+
+
+ @Test
+ public void testOrderByField() throws IOException {
+ createRecords(2000, ProvenanceEventType.SEND, 1L);
+
+ dump(ProvenanceQuery.execute("SELECT Event.Time, Event.ComponentId ORDER BY Event.ComponentId LIMIT 15", repo));
+ dump(ProvenanceQuery.execute("SELECT Event.Time, Event.ComponentId ORDER BY Event.Time DESC LIMIT 15", repo));
+ }
+
+
+ @Test
+ public void testOrderByGroupedField() throws IOException {
+ createRecords(2, ProvenanceEventType.SEND, 0L);
+ createRecords(5, ProvenanceEventType.RECEIVE, 0L);
+
+ dump(ProvenanceQuery.execute("SELECT Event.Type, SUM(Event.Size) GROUP BY Event.Type ORDER BY SUM(Event.Size) DESC", repo));
+
+ ProvenanceResultSet rs = ProvenanceQuery.execute("SELECT Event.Type, SUM(Event.Size) GROUP BY Event.Type ORDER BY SUM(Event.Size) DESC", repo);
+
+ assertTrue( rs.hasNext() );
+ List<?> values = rs.next();
+ assertEquals("RECEIVE", values.get(0).toString());
+ assertEquals(500L, values.get(1));
+
+ assertTrue( rs.hasNext() );
+ values = rs.next();
+ assertEquals("SEND", values.get(0).toString());
+ assertEquals(200L, values.get(1));
+
+ assertFalse( rs.hasNext() );
+
+
+ rs = ProvenanceQuery.execute("SELECT Event.Type, SUM(Event.Size) GROUP BY Event.Type ORDER BY SUM(Event.Size) ASC", repo);
+
+ assertTrue( rs.hasNext() );
+ values = rs.next();
+ assertEquals("SEND", values.get(0).toString());
+ assertEquals(200L, values.get(1));
+
+ assertTrue( rs.hasNext() );
+ values = rs.next();
+ assertEquals("RECEIVE", values.get(0).toString());
+ assertEquals(500L, values.get(1));
+
+ assertFalse( rs.hasNext() );
+ }
+
+
+ @Test
+ public void testOrderByFieldAndGroupedValue() throws IOException {
+ createRecords(3, ProvenanceEventType.SEND, 0L);
+ createRecords(5, ProvenanceEventType.RECEIVE, 0L);
+ createRecords(3, ProvenanceEventType.ATTRIBUTES_MODIFIED, 0L);
+
+ final String query = "SELECT Event.Type, SUM(Event.Size) GROUP BY Event.Type ORDER BY SUM(Event.Size) DESC, Event.Type";
+ dump(ProvenanceQuery.execute(query, repo));
+
+ ProvenanceResultSet rs = ProvenanceQuery.execute(query, repo);
+
+ assertTrue( rs.hasNext() );
+ List<?> vals = rs.next();
+ assertEquals(2, vals.size());
+ assertEquals("RECEIVE", vals.get(0).toString());
+ assertEquals(500L, vals.get(1));
+
+ assertTrue( rs.hasNext() );
+ vals = rs.next();
+ assertEquals(2, vals.size());
+ assertEquals("ATTRIBUTES_MODIFIED", vals.get(0).toString());
+ assertEquals(300L, vals.get(1));
+
+ assertTrue( rs.hasNext() );
+ vals = rs.next();
+ assertEquals(2, vals.size());
+ assertEquals("SEND", vals.get(0).toString());
+ assertEquals(300L, vals.get(1));
+
+ assertFalse( rs.hasNext() );
+ }
+
+
+ @Test
+ public void testAndsOrs() throws IOException {
+
+ final Map<String, String> previousAttributes = new HashMap<>();
+ previousAttributes.put("filename", "xyz");
+
+ final Map<String, String> updatedAttributes = new HashMap<>();
+ updatedAttributes.put("filename", "xyz.txt");
+ updatedAttributes.put("mime.type", "text/plain");
+ updatedAttributes.put("abc", "cba");
+ updatedAttributes.put("123", "321");
+
+ final StandardProvenanceEventRecord.Builder recordBuilder = new StandardProvenanceEventRecord.Builder();
+ recordBuilder.setAttributes(previousAttributes, Collections.<String, String>emptyMap())
+ .setComponentId("000")
+ .setComponentType("MyComponent")
+ .setEventType(ProvenanceEventType.SEND)
+ .setFlowFileEntryDate(System.currentTimeMillis())
+ .setFlowFileUUID("1234")
+ .setCurrentContentClaim("container", "section", "1", 0L, 100L)
+ .setAttributes(previousAttributes, updatedAttributes)
+ .setTransitUri("https://localhost:80/nifi");
+
+ repo.registerEvent(recordBuilder.build());
+
+ final String queryString = "SELECT Event "
+ + "WHERE "
+ + "( "
+ + " Event['filename'] = 'xyz.txt' "
+ + " OR "
+ + " Event['mime.type'] = 'ss' "
+ + ") "
+ + "AND "
+ + "( "
+ + " Event['abc'] = 'cba' "
+ + " OR "
+ + " Event['123'] = '123' "
+ + ")";
+ System.out.println(queryString);
+
+ final ProvenanceQuery query = ProvenanceQuery.compile(queryString, null, null);
+
+ System.out.println(query.getWhereClause());
+
+ ProvenanceResultSet rs = query.execute(repo);
+ assertTrue(rs.hasNext());
+ rs.next();
+ assertFalse(rs.hasNext());
+
+
+
+ updatedAttributes.put("filename", "xxyz");
+ repo = new VolatileProvenanceRepository();
+ recordBuilder.setAttributes(previousAttributes, updatedAttributes);
+ repo.registerEvent(recordBuilder.build());
+
+ rs = query.execute(repo);
+ assertFalse(rs.hasNext());
+
+
+
+ updatedAttributes.put("filename", "xyz.txt");
+ updatedAttributes.put("123", "123");
+ repo = new VolatileProvenanceRepository();
+ recordBuilder.setAttributes(previousAttributes, updatedAttributes);
+ repo.registerEvent(recordBuilder.build());
+
+ rs = query.execute(repo);
+ assertTrue(rs.hasNext());
+ rs.next();
+ assertFalse(rs.hasNext());
+
+ }
+
+ private void dump(final ProvenanceResultSet rs) {
+ System.out.println(rs.getLabels());
+ while (rs.hasNext()) {
+ System.out.println(rs.next());
+ }
+
+ System.out.println("\n\n\n");
+ }
+
+}
diff --git a/nifi/nifi-commons/nifi-provenance-query-language/src/test/resources/nifi.properties b/nifi/nifi-commons/nifi-provenance-query-language/src/test/resources/nifi.properties
new file mode 100644
index 0000000..9d0683d
--- /dev/null
+++ b/nifi/nifi-commons/nifi-provenance-query-language/src/test/resources/nifi.properties
@@ -0,0 +1,136 @@
+# Core Properties #
+nifi.version=nifi-0.0.1
+nifi.flow.configuration.file=./conf/flow.xml.gz
+nifi.flow.configuration.archive.dir=./conf/archive/
+nifi.flowcontroller.autoResumeState=false
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=2 sec
+nifi.administrative.yield.duration=30 sec
+
+nifi.authority.provider.configuration.file=./conf/authority-providers.xml
+nifi.reporting.task.configuration.file=./conf/reporting-tasks.xml
+nifi.controller.service.configuration.file=./conf/controller-services.xml
+nifi.templates.directory=./conf/templates
+nifi.ui.banner.text=
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./lib
+nifi.nar.working.directory=./work/nar/
+nifi.documentation.working.directory=./work/docs/components
+
+# H2 Settings
+nifi.database.directory=./database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.directory=./flowfile_repository
+nifi.flowfile.repository.partitions=256
+nifi.flowfile.repository.checkpoint.interval=2 mins
+nifi.queue.swap.threshold=20000
+nifi.swap.storage.directory=./flowfile_repository/swap
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.claim.max.appendable.size=10 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./content_repository
+
+
+# VOlatile prov properties
+nifi.provenance.repository.buffer.size=250000
+
+
+# Provenance Repository Properties
+nifi.provenance.repository.implementation=nifi.provenance.PersistentProvenanceRepository
+nifi.provenance.repository.directory.default=./provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=5 mins
+nifi.provenance.repository.rollover.size=100 MB
+nifi.provenance.repository.query.threads=2
+nifi.provenance.repository.compress.on.rollover=true
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, ContentType, Relationship, Details
+nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID
+# FlowFile Attributes that should be indexed and made searchable
+nifi.provenance.repository.indexed.attributes=
+# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
+# but should provide better performance
+nifi.provenance.repository.index.shard.size=500 MB
+
+# Component Status Repository
+nifi.components.status.repository.implementation=nifi.controller.status.history.VolatileComponentStatusRepository
+nifi.components.status.repository.buffer.size=288
+nifi.components.status.snapshot.frequency=5 mins
+
+# Site to Site properties
+nifi.remote.input.socket.port=
+nifi.remote.input.secure=true
+
+# web properties #
+nifi.web.war.directory=./lib
+nifi.web.http.interfaces=eth6
+nifi.web.http.host=
+nifi.web.http.port=8080
+nifi.web.https.interfaces=
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.jetty.working.directory=./work/jetty
+
+# security properties #
+nifi.sensitive.props.key=password
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+
+nifi.security.keystore=
+nifi.security.keystoreType=
+nifi.security.keystorePasswd=
+nifi.security.keyPasswd=
+nifi.security.truststore=
+nifi.security.truststoreType=
+nifi.security.truststorePasswd=
+nifi.security.needClientAuth=
+nifi.security.user.credential.cache.duration=24 hours
+nifi.security.user.authority.provider=file-provider
+nifi.security.support.new.account.requests=
+nifi.security.ocsp.responder.url=
+nifi.security.ocsp.responder.certificate=
+
+# cluster common properties (cluster manager and nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+nifi.cluster.protocol.socket.timeout=30 sec
+nifi.cluster.protocol.connection.handshake.timeout=45 sec
+# if multicast is used, then nifi.cluster.protocol.multicast.xxx properties must be configured #
+nifi.cluster.protocol.use.multicast=false
+nifi.cluster.protocol.multicast.address=
+nifi.cluster.protocol.multicast.port=
+nifi.cluster.protocol.multicast.service.broadcast.delay=500 ms
+nifi.cluster.protocol.multicast.service.locator.attempts=3
+nifi.cluster.protocol.multicast.service.locator.attempts.delay=1 sec
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=false
+nifi.cluster.node.address=
+nifi.cluster.node.interfaces=
+nifi.cluster.node.protocol.port=
+nifi.cluster.node.protocol.threads=2
+# if multicast is not used, nifi.cluster.node.unicast.xxx must have same values as nifi.cluster.manager.xxx #
+nifi.cluster.node.unicast.manager.address=
+nifi.cluster.node.unicast.manager.protocol.port=
+
+# cluster manager properties (only configure for cluster manager) #
+nifi.cluster.is.manager=false
+nifi.cluster.manager.address=
+nifi.cluster.manager.interfaces=eth6
+nifi.cluster.manager.protocol.port=
+nifi.cluster.manager.node.firewall.file=
+nifi.cluster.manager.node.event.history.size=10
+nifi.cluster.manager.node.api.connection.timeout=30 sec
+nifi.cluster.manager.node.api.read.timeout=30 sec
+nifi.cluster.manager.node.api.request.threads=10
+nifi.cluster.manager.flow.retrieval.delay=5 sec
+nifi.cluster.manager.protocol.threads=10
+nifi.cluster.manager.safemode.duration=0 sec
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
index 8294af3..3977a3d 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
@@ -28,7 +28,12 @@
private long bytesSinceMark = 0L;
public ByteCountingInputStream(final InputStream in) {
+ this(in, 0);
+ }
+
+ public ByteCountingInputStream(final InputStream in, final long initialCount) {
this.in = in;
+ this.bytesRead = initialCount;
}
@Override
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
index d8e1a42..dc8d197 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
@@ -25,7 +25,12 @@
private long bytesWritten = 0L;
public ByteCountingOutputStream(final OutputStream out) {
+ this(out, 0L);
+ }
+
+ public ByteCountingOutputStream(final OutputStream out, final long initialCount) {
this.out = out;
+ this.bytesWritten = initialCount;
}
@Override
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
index 41a0557..71dbc79 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
@@ -16,7 +16,6 @@
*/
package org.apache.nifi.util.file;
-import java.io.BufferedInputStream;
import java.io.Closeable;
import java.io.File;
import java.io.FileInputStream;
diff --git a/nifi/nifi-commons/pom.xml b/nifi/nifi-commons/pom.xml
index 768dfd9..2549649 100644
--- a/nifi/nifi-commons/pom.xml
+++ b/nifi/nifi-commons/pom.xml
@@ -35,6 +35,7 @@
<module>nifi-web-utils</module>
<module>nifi-processor-utilities</module>
<module>nifi-write-ahead-log</module>
+ <module>nifi-provenance-query-language</module>
<module>nifi-site-to-site-client</module>
</modules>
</project>
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java
index 241041a..cdff42d 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java
@@ -18,23 +18,25 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
import org.apache.nifi.provenance.search.Query;
import org.apache.nifi.provenance.search.QuerySubmission;
import org.apache.nifi.provenance.search.SearchableField;
public class MockProvenanceEventRepository implements ProvenanceEventRepository {
- private final List<ProvenanceEventRecord> records = new ArrayList<>();
+ private final List<StoredProvenanceEvent> records = new ArrayList<>();
private final AtomicLong idGenerator = new AtomicLong(0L);
@Override
- public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
+ public void registerEvents(final Collection<ProvenanceEventRecord> events) {
for (final ProvenanceEventRecord event : events) {
registerEvent(event);
}
@@ -50,7 +52,7 @@
}
newRecord.setEventId(idGenerator.getAndIncrement());
- records.add(newRecord);
+ records.add(new IdEnrichedProvenanceEvent(newRecord));
}
@Override
@@ -58,7 +60,7 @@
}
@Override
- public List<ProvenanceEventRecord> getEvents(long firstRecordId, int maxRecords) throws IOException {
+ public List<StoredProvenanceEvent> getEvents(long firstRecordId, int maxRecords) throws IOException {
if (firstRecordId > records.size()) {
return Collections.emptyList();
}
@@ -92,7 +94,7 @@
}
@Override
- public ProvenanceEventRecord getEvent(long id) throws IOException {
+ public StoredProvenanceEvent getEvent(long id) throws IOException {
if (id > records.size()) {
return null;
}
@@ -128,4 +130,44 @@
public ProvenanceEventBuilder eventBuilder() {
return new StandardProvenanceEventRecord.Builder();
}
+
+ @Override
+ public Long getEarliestEventTime() throws IOException {
+ final StoredProvenanceEvent event = getEvent(0);
+ if ( event == null ) {
+ return null;
+ }
+
+ return event.getEventTime();
+ }
+
+ @Override
+ public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+ if ( location instanceof EventIdLocation ) {
+ return getEvent( ((EventIdLocation) location).getId() );
+ }
+ throw new IllegalArgumentException("Invalid StorageLocation");
+ }
+
+ @Override
+ public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+ final List<StoredProvenanceEvent> events = new ArrayList<>(storageLocations.size());
+ for ( final StorageLocation location : storageLocations ) {
+ final StoredProvenanceEvent event = getEvent(location);
+ if ( event != null ) {
+ events.add(event);
+ }
+ }
+ return events;
+ }
+
+ @Override
+ public ProvenanceQuerySubmission submitQuery(final String query) {
+ return null;
+ }
+
+ @Override
+ public ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(final String queryIdentifier) {
+ return null;
+ }
}
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/.gitignore
index cd1a4e7..9afaccf 100755
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/.gitignore
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/.gitignore
@@ -4,3 +4,4 @@
/target
/target
/target
+/target/
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
index 2015530..398ba29 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
@@ -18,6 +18,8 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
import java.util.List;
import org.apache.nifi.controller.status.ProcessGroupStatus;
@@ -25,7 +27,10 @@
import org.apache.nifi.provenance.ProvenanceEventBuilder;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
import org.apache.nifi.provenance.search.Query;
import org.apache.nifi.provenance.search.QuerySubmission;
import org.apache.nifi.provenance.search.SearchableField;
@@ -59,12 +64,12 @@
}
@Override
- public ProvenanceEventRecord getEvent(long eventId) throws IOException {
+ public StoredProvenanceEvent getEvent(long eventId) throws IOException {
return null;
}
@Override
- public List<ProvenanceEventRecord> getEvents(long startEventId, int maxEvents) throws IOException {
+ public List<StoredProvenanceEvent> getEvents(long startEventId, int maxEvents) throws IOException {
return new ArrayList<>();
}
@@ -88,10 +93,6 @@
}
@Override
- public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
- }
-
- @Override
public ComputeLineageSubmission retrieveLineageSubmission(final String submissionId) {
return null;
}
@@ -130,6 +131,36 @@
public void initialize(EventReporter eventReporter) throws IOException {
}
+
+ @Override
+ public Long getEarliestEventTime() throws IOException {
+ return null;
+ }
+
+ @Override
+ public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+ return null;
+ }
+
+ @Override
+ public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public void registerEvents(final Collection<ProvenanceEventRecord> events) throws IOException {
+
+ }
+
+ @Override
+ public ProvenanceQuerySubmission submitQuery(String query) {
+ return null;
+ }
+
+ @Override
+ public ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(String queryIdentifier) {
+ return null;
+ }
};
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore
index 29546b5..d2d9c31 100755
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore
@@ -1,2 +1,3 @@
/target
/target/
+/target/
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index a04ae3a..b77e94d 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -2520,7 +2520,7 @@
} catch (final InitializationException ie) {
throw new ReportingTaskInstantiationException("Failed to initialize reporting task of type " + type, ie);
}
-
+
try (final NarCloseable x = NarCloseable.withNarLoader()) {
ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task);
} catch (final Exception e) {
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index 272c0ba..6b3cc95 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -48,7 +48,7 @@
private final ControllerServiceLookup serviceLookup;
private final AtomicReference<SchedulingStrategy> schedulingStrategy = new AtomicReference<>(SchedulingStrategy.TIMER_DRIVEN);
- private final AtomicReference<String> schedulingPeriod = new AtomicReference<>("5 mins");
+ private final AtomicReference<String> schedulingPeriod = new AtomicReference<>("1 mins");
private volatile String comment;
private volatile ScheduledState scheduledState = ScheduledState.STOPPED;
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 8d2e456..ca57b81 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -28,11 +28,9 @@
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
-import java.util.NoSuchElementException;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
@@ -53,9 +51,6 @@
import org.apache.nifi.controller.repository.io.LongHolder;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.NonCloseableInputStream;
-import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.FlowFileFilter;
import org.apache.nifi.processor.ProcessSession;
@@ -74,6 +69,9 @@
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.ProvenanceReporter;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.NonCloseableInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -299,7 +297,12 @@
resetReadClaim();
final long updateProvenanceStart = System.nanoTime();
- updateProvenanceRepo(checkpoint);
+ try {
+ updateProvenanceRepo(checkpoint);
+ } catch (final IOException ioe) {
+ rollback();
+ throw new ProcessException("Provenance Repository failed to update", ioe);
+ }
final long claimRemovalStart = System.nanoTime();
final long updateProvenanceNanos = claimRemovalStart - updateProvenanceStart;
@@ -497,7 +500,7 @@
eventTypes.add(eventType);
}
- private void updateProvenanceRepo(final Checkpoint checkpoint) {
+ private void updateProvenanceRepo(final Checkpoint checkpoint) throws IOException {
// Update Provenance Repository
final ProvenanceEventRepository provenanceRepo = context.getProvenanceRepository();
@@ -641,46 +644,16 @@
}
final List<ProvenanceEventRecord> autoTermEvents = checkpoint.autoTerminatedEvents;
- final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
- final Iterator<ProvenanceEventRecord> recordsToSubmitIterator = recordsToSubmit.iterator();
- final Iterator<ProvenanceEventRecord> autoTermIterator = autoTermEvents == null ? null : autoTermEvents.iterator();
-
- @Override
- public Iterator<ProvenanceEventRecord> iterator() {
- return new Iterator<ProvenanceEventRecord>() {
- @Override
- public boolean hasNext() {
- return recordsToSubmitIterator.hasNext() || (autoTermIterator != null && autoTermIterator.hasNext());
- }
-
- @Override
- public ProvenanceEventRecord next() {
- if (recordsToSubmitIterator.hasNext()) {
- final ProvenanceEventRecord rawEvent = recordsToSubmitIterator.next();
-
- // Update the Provenance Event Record with all of the info that we know about the event.
- // For SEND events, we do not want to update the FlowFile info on the Event, because the event should
- // reflect the FlowFile as it was sent to the remote system. However, for other events, we want to use
- // the representation of the FlowFile as it is committed, as this is the only way in which it really
- // exists in our system -- all other representations are volatile representations that have not been
- // exposed.
- return enrich(rawEvent, flowFileRecordMap, checkpoint.records, rawEvent.getEventType() != ProvenanceEventType.SEND);
- } else if (autoTermIterator != null && autoTermIterator.hasNext()) {
- return enrich(autoTermIterator.next(), flowFileRecordMap, checkpoint.records, true);
- }
-
- throw new NoSuchElementException();
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
- }
- };
-
- provenanceRepo.registerEvents(iterable);
+
+ final List<ProvenanceEventRecord> enrichedEvents = new ArrayList<>();
+ for ( final ProvenanceEventRecord record : recordsToSubmit ) {
+ enrichedEvents.add(enrich(record, flowFileRecordMap, checkpoint.records, record.getEventType() != ProvenanceEventType.SEND));
+ }
+ for ( final ProvenanceEventRecord record : autoTermEvents ) {
+ enrichedEvents.add(enrich(record, flowFileRecordMap, checkpoint.records, true));
+ }
+
+ provenanceRepo.registerEvents(enrichedEvents);
}
private void updateEventContentClaims(final ProvenanceEventBuilder builder, final FlowFile flowFile, final StandardRepositoryRecord repoRecord) {
@@ -1140,7 +1113,12 @@
final Connection conn = connections.get(context.getNextIncomingConnectionIndex() % connections.size());
final Set<FlowFileRecord> expired = new HashSet<>();
final FlowFileRecord flowFile = conn.getFlowFileQueue().poll(expired);
- removeExpired(expired, conn);
+
+ try {
+ removeExpired(expired, conn);
+ } catch (final IOException ioe) {
+ throw new ProcessException("Failed to update repositories to remove expired FlowFiles", ioe);
+ }
if (flowFile != null) {
registerDequeuedRecord(flowFile, conn);
@@ -1201,7 +1179,11 @@
for (final Connection conn : connections) {
final Set<FlowFileRecord> expired = new HashSet<>();
final List<FlowFileRecord> newlySelected = poller.poll(conn.getFlowFileQueue(), expired);
- removeExpired(expired, conn);
+ try {
+ removeExpired(expired, conn);
+ } catch (final IOException ioe) {
+ throw new ProcessException("Failed to update repositories to remove expired FlowFiles", ioe);
+ }
if (newlySelected.isEmpty() && expired.isEmpty()) {
continue;
@@ -1571,7 +1553,7 @@
}
}
- public void expireFlowFiles() {
+ public void expireFlowFiles() throws IOException {
final Set<FlowFileRecord> expired = new HashSet<>();
final FlowFileFilter filter = new FlowFileFilter() {
@Override
@@ -1589,7 +1571,7 @@
}
}
- private void removeExpired(final Set<FlowFileRecord> flowFiles, final Connection connection) {
+ private void removeExpired(final Set<FlowFileRecord> flowFiles, final Connection connection) throws IOException {
if (flowFiles.isEmpty()) {
return;
}
@@ -1612,7 +1594,31 @@
final Map<String, FlowFileRecord> recordIdMap = new HashMap<>();
for (final FlowFileRecord flowFile : flowFiles) {
recordIdMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile);
+ }
+
+ final Set<ProvenanceEventRecord> expiredEvents = expiredReporter.getEvents();
+ final List<ProvenanceEventRecord> events = new ArrayList<>(expiredEvents.size());
+ for ( final ProvenanceEventRecord event : expiredEvents ) {
+ final StandardProvenanceEventRecord.Builder enriched = new StandardProvenanceEventRecord.Builder().fromEvent(event);
+ final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
+ if (record == null) {
+ continue;
+ }
+ final ContentClaim claim = record.getContentClaim();
+ if (claim != null) {
+ enriched.setCurrentContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+ enriched.setPreviousContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+ }
+
+ enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
+ events.add(enriched.build());
+ }
+
+ context.getProvenanceRepository().registerEvents(events);
+ context.getFlowFileRepository().updateRepository(expiredRecords);
+
+ for ( final FlowFileRecord flowFile : flowFiles ) {
final StandardRepositoryRecord record = new StandardRepositoryRecord(connection.getFlowFileQueue(), flowFile);
record.markForDelete();
expiredRecords.add(record);
@@ -1623,53 +1629,6 @@
final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
LOG.info("{} terminated by {} due to FlowFile expiration; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
}
-
- try {
- final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
- @Override
- public Iterator<ProvenanceEventRecord> iterator() {
- final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator();
- final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<ProvenanceEventRecord>() {
- @Override
- public boolean hasNext() {
- return expiredEventIterator.hasNext();
- }
-
- @Override
- public ProvenanceEventRecord next() {
- final ProvenanceEventRecord event = expiredEventIterator.next();
- final StandardProvenanceEventRecord.Builder enriched = new StandardProvenanceEventRecord.Builder().fromEvent(event);
- final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
- if (record == null) {
- return null;
- }
-
- final ContentClaim claim = record.getContentClaim();
- if (claim != null) {
- enriched.setCurrentContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
- enriched.setPreviousContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
- }
-
- enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
- return enriched.build();
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
-
- return enrichingIterator;
- }
- };
-
- context.getProvenanceRepository().registerEvents(iterable);
- context.getFlowFileRepository().updateRepository(expiredRecords);
- } catch (final IOException e) {
- LOG.error("Failed to update FlowFile Repository to record expired records due to {}", e);
- }
-
}
private InputStream getInputStream(final FlowFile flowFile, final ContentClaim claim, final long offset) throws ContentNotFoundException {
@@ -2444,7 +2403,14 @@
final ProvenanceEventRecord dropEvent = provenanceReporter.drop(suspectRecord.getCurrent(), nfe.getMessage() == null ? "Content Not Found" : nfe.getMessage());
if (dropEvent != null) {
- context.getProvenanceRepository().registerEvent(dropEvent);
+ try {
+ context.getProvenanceRepository().registerEvent(dropEvent);
+ } catch (final IOException ioe) {
+ LOG.error("{} Failed to register DROP Provenance event for {} when handling ContentNotFound error due to {}", this, suspectRecord.getCurrent(), ioe.toString());
+ if ( LOG.isDebugEnabled() ) {
+ LOG.error("", ioe);
+ }
+ }
}
if (missingClaim == registeredClaim) {
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 087ec68..7347632 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -184,7 +184,7 @@
break;
} catch (final Exception e) {
final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e;
- final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask);
+ final ComponentLog componentLog = new SimpleProcessLogger(taskNode.getIdentifier(), reportingTask);
componentLog.error("Failed to invoke @OnEnabled method due to {}", cause);
LOG.error("Failed to invoke the On-Scheduled Lifecycle methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}",
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
index a351a68..d0020b5 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
@@ -16,9 +16,12 @@
*/
package org.apache.nifi.controller.tasks;
+import java.io.IOException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
+import javax.print.attribute.standard.Severity;
+
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.Funnel;
@@ -29,9 +32,12 @@
import org.apache.nifi.controller.repository.StandardProcessSession;
import org.apache.nifi.controller.repository.StandardProcessSessionFactory;
import org.apache.nifi.controller.scheduling.ProcessContextFactory;
+import org.apache.nifi.events.BulletinFactory;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* This task runs through all Connectable Components and goes through its
@@ -39,7 +45,8 @@
* desired side effect of expiring old FlowFiles.
*/
public class ExpireFlowFiles implements Runnable {
-
+ private static final Logger logger = LoggerFactory.getLogger(ExpireFlowFiles.class);
+
private final FlowController flowController;
private final ProcessContextFactory contextFactory;
@@ -51,7 +58,19 @@
@Override
public void run() {
final ProcessGroup rootGroup = flowController.getGroup(flowController.getRootGroupId());
- expireFlowFiles(rootGroup);
+
+ try {
+ expireFlowFiles(rootGroup);
+ } catch (final Exception e) {
+ logger.error("Failed to expire FlowFiles due to {}", e.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", e);
+ }
+
+ flowController.getBulletinRepository().addBulletin(BulletinFactory.createBulletin(
+ "FlowFile Expiration", Severity.ERROR.getName(), "Could not expire FlowFiles due to " + e));
+
+ }
}
private StandardProcessSession createSession(final Connectable connectable) {
@@ -60,7 +79,7 @@
return sessionFactory.createSession();
}
- private void expireFlowFiles(final Connectable connectable) {
+ private void expireFlowFiles(final Connectable connectable) throws IOException {
// determine if the incoming connections for this Connectable have Expiration configured.
boolean expirationConfigured = false;
for (final Connection incomingConn : connectable.getIncomingConnections()) {
@@ -80,7 +99,7 @@
session.commit();
}
- private void expireFlowFiles(final ProcessGroup group) {
+ private void expireFlowFiles(final ProcessGroup group) throws IOException {
for (final ProcessorNode procNode : group.getProcessors()) {
expireFlowFiles(procNode);
}
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
index 1ff63c5..7ae156c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
@@ -65,6 +65,7 @@
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventRepository;
import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -357,7 +358,7 @@
session.commit();
- final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 1000);
+ final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 1000);
// We should have a JOIN and 2 ATTRIBUTE_MODIFIED's
assertEquals(3, events.size());
@@ -412,7 +413,7 @@
session.remove(orig);
session.commit();
- final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 1000);
+ final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 1000);
assertEquals(2, events.size());
final ProvenanceEventRecord firstRecord = events.get(0);
@@ -838,7 +839,7 @@
session.transfer(newFlowFile, new Relationship.Builder().name("A").build());
session.commit();
- final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+ final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
assertFalse(events.isEmpty());
assertEquals(1, events.size());
@@ -857,7 +858,7 @@
session.transfer(newFlowFile, new Relationship.Builder().name("A").build());
session.commit();
- final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+ final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
assertFalse(events.isEmpty());
assertEquals(1, events.size());
@@ -883,7 +884,7 @@
session.transfer(existingFlowFile, new Relationship.Builder().name("A").build());
session.commit();
- final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+ final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
assertFalse(events.isEmpty());
assertEquals(1, events.size());
@@ -904,7 +905,7 @@
session.transfer(existingFlowFile, new Relationship.Builder().name("A").build());
session.commit();
- final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+ final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
assertFalse(events.isEmpty());
assertEquals(1, events.size());
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index fd16cb5..d7dfa1c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -64,8 +64,17 @@
nifi.content.viewer.url=${nifi.content.viewer.url}
# Provenance Repository Properties
+# Choose implementation of Provenance Repository
nifi.provenance.repository.implementation=${nifi.provenance.repository.implementation}
+# For "Persistent Provenance Repository"
+#nifi.provenance.repository.implementation=org.apache.nifi.provenance.PersistentProvenanceRepository
+# For in-memory provenance repository
+#nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository
+# For "Journaling Provenance Repository"
+#nifi.provenance.repository.implementation=org.apache.nifi.provenance.journaling.JournalingProvenanceRepository
+
+
# Persistent Provenance Repository Properties
nifi.provenance.repository.directory.default=${nifi.provenance.repository.directory.default}
nifi.provenance.repository.max.storage.time=${nifi.provenance.repository.max.storage.time}
@@ -85,9 +94,35 @@
# but should provide better performance
nifi.provenance.repository.index.shard.size=${nifi.provenance.repository.index.shard.size}
+
+
# Volatile Provenance Respository Properties
nifi.provenance.repository.buffer.size=${nifi.provenance.repository.buffer.size}
+
+
+# Journaling Provenance Repository Properties
+#nifi.provenance.repository.directory.default=${nifi.provenance.repository.directory.default}
+#nifi.provenance.repository.max.storage.time=${nifi.provenance.repository.max.storage.time}
+#nifi.provenance.repository.max.storage.size=${nifi.provenance.repository.max.storage.size}
+#nifi.provenance.repository.rollover.time=${nifi.provenance.repository.rollover.time}
+#nifi.provenance.repository.rollover.size=${nifi.provenance.repository.rollover.size}
+#nifi.provenance.repository.compress.on.rollover=${nifi.provenance.repository.compress.on.rollover}
+#nifi.provenance.repository.always.sync=${nifi.provenance.repository.always.sync}
+#nifi.provenance.repository.journal.count=${nifi.provenance.repository.journal.count}
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, ContentType, Relationship, Details
+#nifi.provenance.repository.indexed.fields=${nifi.provenance.repository.indexed.fields}
+# FlowFile Attributes that should be indexed and made searchable
+#nifi.provenance.repository.indexed.attributes=${nifi.provenance.repository.indexed.attributes}
+# Larger values for block size will result in better compression but slower read performance
+#nifi.provenance.repository.writer.block.size=5000
+#nifi.provenance.repository.worker.threads=2
+#nifi.provenance.repository.query.threads=${nifi.provenance.repository.query.threads}
+#nifi.provenance.repository.compression.threads=2
+
+
+
# Component Status Repository
nifi.components.status.repository.implementation=${nifi.components.status.repository.implementation}
nifi.components.status.repository.buffer.size=${nifi.components.status.repository.buffer.size}
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class
new file mode 100644
index 0000000..a6951d4
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class
new file mode 100644
index 0000000..2e868ea
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class
new file mode 100644
index 0000000..9c6e821
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class
new file mode 100644
index 0000000..bb3fc77
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class
new file mode 100644
index 0000000..a56b5ba
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class
new file mode 100644
index 0000000..9780f75
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class
new file mode 100644
index 0000000..f184d64
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class
new file mode 100644
index 0000000..0740bb4
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class
new file mode 100644
index 0000000..d517458
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class
new file mode 100644
index 0000000..1cf5ceb
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class
new file mode 100644
index 0000000..3ad7542
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class
new file mode 100644
index 0000000..4db4735
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class
new file mode 100644
index 0000000..e49ffe8
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class
new file mode 100644
index 0000000..39dd49a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class
new file mode 100644
index 0000000..b415421
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class
new file mode 100644
index 0000000..551097e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class
new file mode 100644
index 0000000..6913767
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class
new file mode 100644
index 0000000..f6e9f20
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class
new file mode 100644
index 0000000..41fe366
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class
new file mode 100644
index 0000000..9b9cdc0
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class
new file mode 100644
index 0000000..2ef1c39
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class
new file mode 100644
index 0000000..fad8245
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class
new file mode 100644
index 0000000..27ec8d3
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class
new file mode 100644
index 0000000..4673aec
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class
new file mode 100644
index 0000000..ac7e1b9
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class
new file mode 100644
index 0000000..933ef4d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class
new file mode 100644
index 0000000..0e77276
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class
Binary files differ
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index 7359c5a..cd3b794 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -664,9 +664,9 @@
resultsDto.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime()));
// get the oldest available event time
- final List<ProvenanceEventRecord> firstEvent = provenanceRepository.getEvents(0, 1);
- if (!firstEvent.isEmpty()) {
- resultsDto.setOldestEvent(new Date(firstEvent.get(0).getEventTime()));
+ final Long oldestEventTime = provenanceRepository.getEarliestEventTime();
+ if (oldestEventTime != null) {
+ resultsDto.setOldestEvent(new Date(oldestEventTime));
}
provenanceDto.setResults(resultsDto);
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
new file mode 100644
index 0000000..3d5163e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0"?>
+<project
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-provenance-repository-bundle</artifactId>
+ <version>0.0.2-incubating-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>nifi-journaling-provenance-repository</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-data-provenance-utils</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-properties</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-provenance-query-language</artifactId>
+ <version>0.0.2-incubating-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.lucene</groupId>
+ <artifactId>lucene-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.lucene</groupId>
+ <artifactId>lucene-analyzers-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.lucene</groupId>
+ <artifactId>lucene-queryparser</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-simple</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/IdEnrichedProvenanceEvent.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/IdEnrichedProvenanceEvent.java
new file mode 100644
index 0000000..0e070d8
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/IdEnrichedProvenanceEvent.java
@@ -0,0 +1,170 @@
+/*
+ * 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.provenance.journaling;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+
+/**
+ * Wraps a Provenance Event Record but provides a different Event ID for the record. This is done because we
+ * do not want to allow the ProvenanceEventBuilder to expose an identifier, because the identifier should be
+ * assigned only when the event has been persisted to the repository.
+ */
+public class IdEnrichedProvenanceEvent implements ProvenanceEventRecord {
+ private final ProvenanceEventRecord event;
+ private final long id;
+
+ public IdEnrichedProvenanceEvent(final ProvenanceEventRecord event, final long id) {
+ this.event = event;
+ this.id = id;
+ }
+
+ @Override
+ public long getEventId() {
+ return id;
+ }
+
+ public long getEventTime() {
+ return event.getEventTime();
+ }
+
+ public long getFlowFileEntryDate() {
+ return event.getFlowFileEntryDate();
+ }
+
+ public long getLineageStartDate() {
+ return event.getLineageStartDate();
+ }
+
+ public Set<String> getLineageIdentifiers() {
+ return event.getLineageIdentifiers();
+ }
+
+ public long getFileSize() {
+ return event.getFileSize();
+ }
+
+ public Long getPreviousFileSize() {
+ return event.getPreviousFileSize();
+ }
+
+ public long getEventDuration() {
+ return event.getEventDuration();
+ }
+
+ public ProvenanceEventType getEventType() {
+ return event.getEventType();
+ }
+
+ @Override
+ public String getAttribute(final String attributeName) {
+ return event.getAttribute(attributeName);
+ }
+
+ public Map<String, String> getAttributes() {
+ return event.getAttributes();
+ }
+
+ public Map<String, String> getPreviousAttributes() {
+ return event.getPreviousAttributes();
+ }
+
+ public Map<String, String> getUpdatedAttributes() {
+ return event.getUpdatedAttributes();
+ }
+
+ public String getComponentId() {
+ return event.getComponentId();
+ }
+
+ public String getComponentType() {
+ return event.getComponentType();
+ }
+
+ public String getTransitUri() {
+ return event.getTransitUri();
+ }
+
+ public String getSourceSystemFlowFileIdentifier() {
+ return event.getSourceSystemFlowFileIdentifier();
+ }
+
+ public String getFlowFileUuid() {
+ return event.getFlowFileUuid();
+ }
+
+ public List<String> getParentUuids() {
+ return event.getParentUuids();
+ }
+
+ public List<String> getChildUuids() {
+ return event.getChildUuids();
+ }
+
+ public String getAlternateIdentifierUri() {
+ return event.getAlternateIdentifierUri();
+ }
+
+ public String getDetails() {
+ return event.getDetails();
+ }
+
+ public String getRelationship() {
+ return event.getRelationship();
+ }
+
+ public String getSourceQueueIdentifier() {
+ return event.getSourceQueueIdentifier();
+ }
+
+ public String getContentClaimSection() {
+ return event.getContentClaimSection();
+ }
+
+ public String getPreviousContentClaimSection() {
+ return event.getPreviousContentClaimSection();
+ }
+
+ public String getContentClaimContainer() {
+ return event.getContentClaimContainer();
+ }
+
+ public String getPreviousContentClaimContainer() {
+ return event.getPreviousContentClaimContainer();
+ }
+
+ public String getContentClaimIdentifier() {
+ return event.getContentClaimIdentifier();
+ }
+
+ public String getPreviousContentClaimIdentifier() {
+ return event.getPreviousContentClaimIdentifier();
+ }
+
+ public Long getContentClaimOffset() {
+ return event.getContentClaimOffset();
+ }
+
+ public Long getPreviousContentClaimOffset() {
+ return event.getPreviousContentClaimOffset();
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledProvenanceEvent.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledProvenanceEvent.java
new file mode 100644
index 0000000..55749f9
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledProvenanceEvent.java
@@ -0,0 +1,184 @@
+/*
+ * 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.provenance.journaling;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+
+public class JournaledProvenanceEvent implements StoredProvenanceEvent {
+
+ private final ProvenanceEventRecord event;
+ private final JournaledStorageLocation location;
+
+ public JournaledProvenanceEvent(final ProvenanceEventRecord event, final JournaledStorageLocation location) {
+ this.event = event;
+ this.location = location;
+ }
+
+ @Override
+ public JournaledStorageLocation getStorageLocation() {
+ return location;
+ }
+
+ public long getEventId() {
+ return event.getEventId();
+ }
+
+ public long getEventTime() {
+ return event.getEventTime();
+ }
+
+ public long getFlowFileEntryDate() {
+ return event.getFlowFileEntryDate();
+ }
+
+ public long getLineageStartDate() {
+ return event.getLineageStartDate();
+ }
+
+ public Set<String> getLineageIdentifiers() {
+ return event.getLineageIdentifiers();
+ }
+
+ public long getFileSize() {
+ return event.getFileSize();
+ }
+
+ public Long getPreviousFileSize() {
+ return event.getPreviousFileSize();
+ }
+
+ public long getEventDuration() {
+ return event.getEventDuration();
+ }
+
+ public ProvenanceEventType getEventType() {
+ return event.getEventType();
+ }
+
+ public String getAttribute(final String attributeName) {
+ return event.getAttribute(attributeName);
+ }
+
+ public Map<String, String> getAttributes() {
+ return event.getAttributes();
+ }
+
+ public Map<String, String> getPreviousAttributes() {
+ return event.getPreviousAttributes();
+ }
+
+ public Map<String, String> getUpdatedAttributes() {
+ return event.getUpdatedAttributes();
+ }
+
+ public String getComponentId() {
+ return event.getComponentId();
+ }
+
+ public String getComponentType() {
+ return event.getComponentType();
+ }
+
+ public String getTransitUri() {
+ return event.getTransitUri();
+ }
+
+ public String getSourceSystemFlowFileIdentifier() {
+ return event.getSourceSystemFlowFileIdentifier();
+ }
+
+ public String getFlowFileUuid() {
+ return event.getFlowFileUuid();
+ }
+
+ public List<String> getParentUuids() {
+ return event.getParentUuids();
+ }
+
+ public List<String> getChildUuids() {
+ return event.getChildUuids();
+ }
+
+ public String getAlternateIdentifierUri() {
+ return event.getAlternateIdentifierUri();
+ }
+
+ public String getDetails() {
+ return event.getDetails();
+ }
+
+ public String getRelationship() {
+ return event.getRelationship();
+ }
+
+ public String getSourceQueueIdentifier() {
+ return event.getSourceQueueIdentifier();
+ }
+
+ public String getContentClaimSection() {
+ return event.getContentClaimSection();
+ }
+
+ public String getPreviousContentClaimSection() {
+ return event.getPreviousContentClaimSection();
+ }
+
+ public String getContentClaimContainer() {
+ return event.getContentClaimContainer();
+ }
+
+ public String getPreviousContentClaimContainer() {
+ return event.getPreviousContentClaimContainer();
+ }
+
+ public String getContentClaimIdentifier() {
+ return event.getContentClaimIdentifier();
+ }
+
+ public String getPreviousContentClaimIdentifier() {
+ return event.getPreviousContentClaimIdentifier();
+ }
+
+ public Long getContentClaimOffset() {
+ return event.getContentClaimOffset();
+ }
+
+ public Long getPreviousContentClaimOffset() {
+ return event.getPreviousContentClaimOffset();
+ }
+
+ public boolean equals(Object obj) {
+ return location.equals(obj);
+ }
+
+ public int hashCode() {
+ return location.hashCode();
+ }
+
+ public String toString() {
+ return location.toString();
+ }
+
+
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
new file mode 100644
index 0000000..d4f9e44
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
@@ -0,0 +1,92 @@
+/*
+ * 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.provenance.journaling;
+
+import java.util.Comparator;
+
+import org.apache.nifi.provenance.StorageLocation;
+
+public class JournaledStorageLocation implements StorageLocation, Comparable<JournaledStorageLocation>, Comparator<JournaledStorageLocation> {
+ private final String containerName;
+ private final String sectionName;
+ private final Long journalId;
+ private final int blockIndex;
+ private final long eventId;
+
+ public JournaledStorageLocation(final String containerName, final String sectionName, final Long journalId, final int blockIndex, final long eventId) {
+ this.containerName = containerName;
+ this.sectionName = sectionName;
+ this.journalId = journalId;
+ this.blockIndex = blockIndex;
+ this.eventId = eventId;
+ }
+
+ public String getContainerName() {
+ return containerName;
+ }
+
+ public String getSectionName() {
+ return sectionName;
+ }
+
+ public Long getJournalId() {
+ return journalId;
+ }
+
+ public int getBlockIndex() {
+ return blockIndex;
+ }
+
+ public long getEventId() {
+ return eventId;
+ }
+
+ @Override
+ public String toString() {
+ return "JournaledStorageLocation [containerName=" + containerName + ", sectionName=" + sectionName + ", journalId=" + journalId + ", eventId=" + eventId + "]";
+ }
+
+ @Override
+ public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+ final int containerVal = o1.getContainerName().compareTo(o2.getContainerName());
+ if ( containerVal != 0 ) {
+ return containerVal;
+ }
+
+ final int sectionVal = o1.getSectionName().compareTo(o2.getSectionName());
+ if ( sectionVal != 0 ) {
+ return sectionVal;
+ }
+
+ final int journalVal = o1.getJournalId().compareTo(o2.getJournalId());
+ if ( journalVal != 0 ) {
+ return journalVal;
+ }
+
+ final int blockVal = Integer.compare(o1.getBlockIndex(), o2.getBlockIndex());
+ if ( blockVal != 0 ) {
+ return blockVal;
+ }
+
+ return Long.compare(o1.getEventId(), o2.getEventId());
+ }
+
+ @Override
+ public int compareTo(final JournaledStorageLocation o) {
+ return compare(this, o);
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
new file mode 100644
index 0000000..12f2911
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
@@ -0,0 +1,748 @@
+/*
+ * 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.provenance.journaling;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.pql.ProvenanceQuery;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.provenance.ProvenanceEventBuilder;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.SearchableFieldParser;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.exception.EventNotFoundException;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.IndexAction;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
+import org.apache.nifi.provenance.journaling.index.LuceneIndexManager;
+import org.apache.nifi.provenance.journaling.index.QueryUtils;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.partition.Partition;
+import org.apache.nifi.provenance.journaling.partition.PartitionAction;
+import org.apache.nifi.provenance.journaling.partition.PartitionManager;
+import org.apache.nifi.provenance.journaling.partition.QueuingPartitionManager;
+import org.apache.nifi.provenance.journaling.partition.VoidPartitionAction;
+import org.apache.nifi.provenance.journaling.query.QueryManager;
+import org.apache.nifi.provenance.journaling.query.StandardQueryManager;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.query.ProvenanceQueryResult;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.search.SearchableField;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+// TODO: read-only is not checked everywhere!
+public class JournalingProvenanceRepository implements ProvenanceEventRepository {
+ public static final String WORKER_THREAD_POOL_SIZE = "nifi.provenance.repository.worker.threads";
+ public static final String BLOCK_SIZE = "nifi.provenance.repository.writer.block.size";
+
+ private static final Logger logger = LoggerFactory.getLogger(JournalingProvenanceRepository.class);
+
+ private final JournalingRepositoryConfig config;
+ private final AtomicLong idGenerator = new AtomicLong(0L);
+
+ // the follow member variables are effectively final. They are initialized
+ // in the initialize method rather than the constructor because we want to ensure
+ // that they are not created every time that the Java Service Loader instantiates the class.
+ private ScheduledExecutorService workerExecutor;
+ private ExecutorService queryExecutor;
+ private ExecutorService compressionExecutor;
+ private EventReporter eventReporter;
+ private PartitionManager partitionManager;
+ private QueryManager queryManager;
+ private IndexManager indexManager;
+
+ public JournalingProvenanceRepository() throws IOException {
+ this(createConfig());
+ }
+
+ public JournalingProvenanceRepository(final JournalingRepositoryConfig config) throws IOException {
+ this.config = config;
+ }
+
+ private static ThreadFactory createThreadFactory(final String namePrefix) {
+ final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+ final AtomicInteger counter = new AtomicInteger(0);
+
+ return new ThreadFactory() {
+ @Override
+ public Thread newThread(final Runnable r) {
+ final Thread thread = defaultFactory.newThread(r);
+ thread.setName(namePrefix + "-" + counter.incrementAndGet());
+ return thread;
+ }
+ };
+ }
+
+ private static JournalingRepositoryConfig createConfig() {
+ final NiFiProperties properties = NiFiProperties.getInstance();
+ final Map<String, Path> storageDirectories = properties.getProvenanceRepositoryPaths();
+ if (storageDirectories.isEmpty()) {
+ storageDirectories.put("provenance_repository", Paths.get("provenance_repository"));
+ }
+ final String storageTime = properties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_TIME, "24 hours");
+ final String storageSize = properties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_SIZE, "1 GB");
+ final String rolloverTime = properties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_TIME, "5 mins");
+ final String rolloverSize = properties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_SIZE, "100 MB");
+ final String shardSize = properties.getProperty(NiFiProperties.PROVENANCE_INDEX_SHARD_SIZE, "500 MB");
+ final int queryThreads = properties.getIntegerProperty(NiFiProperties.PROVENANCE_QUERY_THREAD_POOL_SIZE, 2);
+ final int workerThreads = properties.getIntegerProperty(WORKER_THREAD_POOL_SIZE, 4);
+ final int journalCount = properties.getIntegerProperty(NiFiProperties.PROVENANCE_JOURNAL_COUNT, 16);
+
+ final long storageMillis = FormatUtils.getTimeDuration(storageTime, TimeUnit.MILLISECONDS);
+ final long maxStorageBytes = DataUnit.parseDataSize(storageSize, DataUnit.B).longValue();
+ final long rolloverMillis = FormatUtils.getTimeDuration(rolloverTime, TimeUnit.MILLISECONDS);
+ final long rolloverBytes = DataUnit.parseDataSize(rolloverSize, DataUnit.B).longValue();
+
+ final boolean compressOnRollover = Boolean.parseBoolean(properties.getProperty(NiFiProperties.PROVENANCE_COMPRESS_ON_ROLLOVER));
+ final String indexedFieldString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_FIELDS);
+ final String indexedAttrString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_ATTRIBUTES);
+ final int blockSize = properties.getIntegerProperty(BLOCK_SIZE, 5000);
+
+ final Boolean alwaysSync = Boolean.parseBoolean(properties.getProperty("nifi.provenance.repository.always.sync", "false"));
+
+ final List<SearchableField> searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true);
+ final List<SearchableField> searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false);
+
+ // We always want to index the Event Time.
+ if (!searchableFields.contains(SearchableFields.EventTime)) {
+ searchableFields.add(SearchableFields.EventTime);
+ }
+
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+
+ final Map<String, File> containers = new HashMap<>(storageDirectories.size());
+ for ( final Map.Entry<String, Path> entry : storageDirectories.entrySet() ) {
+ containers.put(entry.getKey(), entry.getValue().toFile());
+ }
+ config.setContainers(containers);
+ config.setCompressOnRollover(compressOnRollover);
+ config.setSearchableFields(searchableFields);
+ config.setSearchableAttributes(searchableAttributes);
+ config.setJournalCapacity(rolloverBytes);
+ config.setJournalRolloverPeriod(rolloverMillis, TimeUnit.MILLISECONDS);
+ config.setEventExpiration(storageMillis, TimeUnit.MILLISECONDS);
+ config.setMaxStorageCapacity(maxStorageBytes);
+ config.setQueryThreadPoolSize(queryThreads);
+ config.setWorkerThreadPoolSize(workerThreads);
+ config.setPartitionCount(journalCount);
+ config.setBlockSize(blockSize);
+
+ if (shardSize != null) {
+ config.setDesiredIndexSize(DataUnit.parseDataSize(shardSize, DataUnit.B).longValue());
+ }
+
+ config.setAlwaysSync(alwaysSync);
+
+ return config;
+ }
+
+ @Override
+ public synchronized void initialize(final EventReporter eventReporter) throws IOException {
+ this.eventReporter = eventReporter;
+
+ // Ensure that the number of partitions specified by the config is at least as large as the
+ // number of sections that we have. If not, update the config to be equal to the number of
+ // sections that we have.
+ final Pattern numberPattern = Pattern.compile("\\d+");
+ int numSections = 0;
+ for ( final File container : config.getContainers().values() ) {
+ final String[] sections = container.list(new FilenameFilter() {
+ @Override
+ public boolean accept(final File dir, final String name) {
+ return numberPattern.matcher(name).matches();
+ }
+ });
+
+ if ( sections != null ) {
+ numSections += sections.length;
+ }
+ }
+
+ if ( config.getPartitionCount() < numSections ) {
+ logger.warn("Configured number of partitions for Provenance Repository is {}, but {} partitions already exist. Using {} partitions instead of {}.",
+ config.getPartitionCount(), numSections, numSections, config.getPartitionCount());
+ config.setPartitionCount(numSections);
+ }
+
+ // We use 3 different thread pools here because we don't want to threads from 1 pool to interfere with
+ // each other. This is because the worker threads can be long running, and they shouldn't tie up the
+ // compression threads. Likewise, there may be MANY compression tasks, which could delay the worker
+ // threads. And the query threads need to run immediately when a user submits a query - they cannot
+ // wait until we finish compressing data and sync'ing the repository!
+ final int workerThreadPoolSize = Math.max(2, config.getWorkerThreadPoolSize());
+ this.workerExecutor = Executors.newScheduledThreadPool(workerThreadPoolSize, createThreadFactory("Provenance Repository Worker Thread"));
+
+ final int queryThreadPoolSize = Math.max(2, config.getQueryThreadPoolSize());
+ this.queryExecutor = Executors.newScheduledThreadPool(queryThreadPoolSize, createThreadFactory("Provenance Repository Query Thread"));
+
+ final int compressionThreads = Math.max(1, config.getCompressionThreadPoolSize());
+ this.compressionExecutor = Executors.newFixedThreadPool(compressionThreads, createThreadFactory("Provenance Repository Compression Thread"));
+
+ this.indexManager = new LuceneIndexManager(this, config, workerExecutor, queryExecutor);
+ this.partitionManager = new QueuingPartitionManager(indexManager, idGenerator, config, workerExecutor, compressionExecutor);
+ this.queryManager = new StandardQueryManager(indexManager, queryExecutor, config, 10);
+
+ final Long maxEventId = getMaxEventId();
+ if ( maxEventId != null && maxEventId > 0 ) {
+ this.idGenerator.set(maxEventId); // maxEventId returns 1 greater than the last event id written
+ }
+
+ // the partition manager may have caused journals to be re-indexed. We will sync the
+ // index manager to make sure that we are completely in sync before allowing any new data
+ // to be written to the repo.
+ indexManager.sync();
+
+ final long expirationFrequencyNanos = config.getExpirationFrequency(TimeUnit.NANOSECONDS);
+ workerExecutor.scheduleWithFixedDelay(new ExpireOldEvents(), expirationFrequencyNanos, expirationFrequencyNanos, TimeUnit.NANOSECONDS);
+
+ workerExecutor.scheduleWithFixedDelay(new Runnable() {
+ @Override
+ public void run() {
+ partitionManager.deleteEventsBasedOnSize();
+ }
+ }, expirationFrequencyNanos, expirationFrequencyNanos, TimeUnit.NANOSECONDS);
+ }
+
+ @Override
+ public ProvenanceEventBuilder eventBuilder() {
+ return new StandardProvenanceEventRecord.Builder();
+ }
+
+ @Override
+ public void registerEvent(final ProvenanceEventRecord event) throws IOException {
+ registerEvents(Collections.singleton(event));
+ }
+
+ @Override
+ public void registerEvents(final Collection<ProvenanceEventRecord> events) throws IOException {
+ try {
+ partitionManager.withPartition(new VoidPartitionAction() {
+ @Override
+ public void perform(final Partition partition) throws IOException {
+ partition.registerEvents(events, idGenerator.getAndAdd(events.size()));
+ }
+ }, true);
+ } catch (final IOException ioe) {
+ if ( eventReporter != null ) {
+ eventReporter.reportEvent(Severity.ERROR, "Provenance Repository", "Failed to persist " + events.size() + " events to Provenance Repository due to " + ioe);
+ }
+ throw ioe;
+ }
+ }
+
+ @Override
+ public StoredProvenanceEvent getEvent(final long id) throws IOException {
+ final List<StoredProvenanceEvent> events = getEvents(id, 1);
+ if ( events.isEmpty() ) {
+ return null;
+ }
+
+ // We have to check the id of the event returned, because we are requesting up to 1 record
+ // starting with the given id. However, if that ID doesn't exist, we could get a record
+ // with a larger id.
+ final StoredProvenanceEvent event = events.get(0);
+ if ( event.getEventId() == id ) {
+ return event;
+ }
+
+ return null;
+ }
+
+ @Override
+ public List<StoredProvenanceEvent> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
+ // Must generate query to determine the appropriate StorageLocation objects and then call
+ // getEvent(List<StorageLocation>)
+ final Set<List<JournaledStorageLocation>> resultSet = indexManager.withEachIndex(
+ new IndexAction<List<JournaledStorageLocation>>() {
+ @Override
+ public List<JournaledStorageLocation> perform(final EventIndexSearcher searcher) throws IOException {
+ return searcher.getEvents(firstRecordId, maxRecords);
+ }
+ });
+
+ final ArrayList<JournaledStorageLocation> locations = new ArrayList<>(maxRecords);
+ for ( final List<JournaledStorageLocation> list : resultSet ) {
+ for ( final JournaledStorageLocation location : list ) {
+ locations.add(location);
+ }
+ }
+
+ Collections.sort(locations, new Comparator<JournaledStorageLocation>() {
+ @Override
+ public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+ return Long.compare(o1.getEventId(), o2.getEventId());
+ }
+ });
+
+ locations.trimToSize();
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ final List<StorageLocation> storageLocations = (List<StorageLocation>) ((List) locations);
+ return getEvents(storageLocations);
+ }
+
+ @Override
+ public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+ final List<StoredProvenanceEvent> storedEvents = getEvents(Collections.singletonList(location));
+ return (storedEvents == null || storedEvents.isEmpty()) ? null : storedEvents.get(0);
+ }
+
+
+
+ @Override
+ public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> locations) throws IOException {
+ // Group the locations by journal files because we want a single thread, at most, per journal file.
+ final Map<File, List<JournaledStorageLocation>> orderedLocations = QueryUtils.orderLocations(locations, config);
+
+ // Go through each journal file and create a callable that can lookup the records for that journal file.
+ final List<Future<List<StoredProvenanceEvent>>> futures = new ArrayList<>();
+ for ( final Map.Entry<File, List<JournaledStorageLocation>> entry : orderedLocations.entrySet() ) {
+ final File journalFile = entry.getKey();
+ final List<JournaledStorageLocation> locationsForFile = entry.getValue();
+
+ final Callable<List<StoredProvenanceEvent>> callable = new Callable<List<StoredProvenanceEvent>>() {
+ @Override
+ public List<StoredProvenanceEvent> call() throws Exception {
+ final File tocFile = QueryUtils.getTocFile(journalFile);
+ if ( !journalFile.exists() || !tocFile.exists() ) {
+ return Collections.emptyList();
+ }
+
+ try(final TocReader tocReader = new StandardTocReader(tocFile);
+ final JournalReader reader = new StandardJournalReader(journalFile))
+ {
+ final List<StoredProvenanceEvent> storedEvents = new ArrayList<>(locationsForFile.size());
+
+ for ( final JournaledStorageLocation location : locationsForFile ) {
+ final long blockOffset = tocReader.getBlockOffset(location.getBlockIndex());
+ final ProvenanceEventRecord event = reader.getEvent(blockOffset, location.getEventId());
+
+ storedEvents.add(new JournaledProvenanceEvent(event, location));
+ }
+
+ return storedEvents;
+ }
+ }
+ };
+
+ final Future<List<StoredProvenanceEvent>> future = queryExecutor.submit(callable);
+ futures.add(future);
+ }
+
+ // Get all of the events from the futures, waiting for them to finish.
+ final Map<StorageLocation, StoredProvenanceEvent> locationToEventMap = new HashMap<>(locations.size());
+ for ( final Future<List<StoredProvenanceEvent>> future : futures ) {
+ try {
+ final List<StoredProvenanceEvent> events = future.get();
+
+ // Map the location to the event, so that we can then re-order the events in the same order
+ // that the locations were passed to us.
+ for ( final StoredProvenanceEvent event : events ) {
+ locationToEventMap.put(event.getStorageLocation(), event);
+ }
+ } catch (final ExecutionException ee) {
+ final Throwable cause = ee.getCause();
+ if ( cause instanceof IOException ) {
+ throw (IOException) cause;
+ } else {
+ throw new RuntimeException(cause);
+ }
+ } catch (final InterruptedException ie) {
+ throw new RuntimeException(ie);
+ }
+ }
+
+ // Sort Events by the order of the provided locations.
+ final List<StoredProvenanceEvent> sortedEvents = new ArrayList<>(locations.size());
+ for ( final StorageLocation location : locations ) {
+ final StoredProvenanceEvent event = locationToEventMap.get(location);
+ if ( event != null ) {
+ sortedEvents.add(event);
+ }
+ }
+
+ return sortedEvents;
+ }
+
+
+ @Override
+ public Long getMaxEventId() throws IOException {
+ final Set<Long> maxIds = partitionManager.withEachPartitionSerially(new PartitionAction<Long>() {
+ @Override
+ public Long perform(final Partition partition) throws IOException {
+ return partition.getMaxEventId();
+ }
+ }, false);
+
+ Long maxId = null;
+ for ( final Long id : maxIds ) {
+ if ( id == null ) {
+ continue;
+ }
+
+ if ( maxId == null || id > maxId ) {
+ maxId = id;
+ }
+ }
+
+ return maxId;
+ }
+
+ ProgressAwareIterator<? extends StoredProvenanceEvent> selectMatchingEvents(final String query, final Set<String> referencedFields, final AtomicLong lastTimeProgressMade) throws IOException {
+ final Set<EventIndexSearcher> searchers = indexManager.getSearchers();
+ final Iterator<EventIndexSearcher> searchItr = searchers.iterator();
+
+ return new ProgressAwareIterator<StoredProvenanceEvent>() {
+ private Iterator<LazyInitializedProvenanceEvent> eventItr;
+ private int searchersComplete = 0;
+ private EventIndexSearcher currentSearcher;
+
+ @Override
+ public int getPercentComplete() {
+ return searchers.isEmpty() ? 100 : searchersComplete / searchers.size() * 100;
+ }
+
+ @Override
+ public boolean hasNext() {
+ // while the event iterator has no information...
+ while ( eventItr == null || !eventItr.hasNext() ) {
+ // if there's not another searcher then we're out of events.
+ if ( !searchItr.hasNext() ) {
+ return false;
+ }
+
+ // we're finished with this searcher. Close it.
+ if ( currentSearcher != null ) {
+ try {
+ currentSearcher.close();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to close {} due to {}", currentSearcher, ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+ }
+
+ // We have a searcher. get events from it. If there are no matches,
+ // then our while loop will keep going.
+ currentSearcher = searchItr.next();
+ searchersComplete++;
+
+ try {
+ eventItr = currentSearcher.select(query, referencedFields);
+ } catch (final IOException ioe) {
+ throw new EventNotFoundException("Could not find next event", ioe);
+ }
+ }
+
+ // the event iterator has no events, and the search iterator has no more
+ // searchers. There are no more events.
+ return eventItr != null && eventItr.hasNext();
+ }
+
+ @Override
+ public StoredProvenanceEvent next() {
+ lastTimeProgressMade.set(System.nanoTime());
+ return eventItr.next();
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+
+ public ProvenanceResultSet query(final String query) throws IOException {
+ final ProvenanceQuerySubmission submission = submitQuery(query);
+ return submission.getResult().getResultSet();
+ }
+
+
+ public ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(final String queryIdentifier) {
+ return queryManager.retrieveProvenanceQuerySubmission(queryIdentifier);
+ }
+
+ @Override
+ public ProvenanceQuerySubmission submitQuery(final String query) {
+ ProvenanceQuerySubmission submission;
+ final AtomicLong lastTimeProgressMade = new AtomicLong(System.nanoTime());
+ final long tenMinsInNanos = TimeUnit.MINUTES.toNanos(10);
+
+ try {
+ final ProvenanceQuery provenanceQuery = ProvenanceQuery.compile(query, getSearchableFields(), getSearchableAttributes());
+
+ final Set<String> referencedFields = provenanceQuery.getReferencedFields();
+// final Set<String> referencedFields = null;
+ final ProgressAwareIterator<? extends StoredProvenanceEvent> eventItr = selectMatchingEvents(query, referencedFields, lastTimeProgressMade);
+ final ProvenanceResultSet rs = provenanceQuery.evaluate(eventItr);
+
+ submission = new JournalingRepoQuerySubmission(query, new ProvenanceQueryResult() {
+ @Override
+ public ProvenanceResultSet getResultSet() {
+ return rs;
+ }
+
+ @Override
+ public Date getExpiration() {
+ return new Date(tenMinsInNanos + lastTimeProgressMade.get());
+ }
+
+ @Override
+ public String getError() {
+ return null;
+ }
+
+ @Override
+ public int getPercentComplete() {
+ return eventItr.getPercentComplete();
+ }
+
+ @Override
+ public boolean isFinished() {
+ return eventItr.getPercentComplete() >= 100;
+ }
+ });
+ } catch (final IOException ioe) {
+ logger.error("Failed to perform query {} due to {}", query, ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+
+ submission = new JournalingRepoQuerySubmission(query, new ProvenanceQueryResult() {
+ @Override
+ public ProvenanceResultSet getResultSet() {
+ return null;
+ }
+
+ @Override
+ public Date getExpiration() {
+ return new Date(tenMinsInNanos + lastTimeProgressMade.get());
+ }
+
+ @Override
+ public String getError() {
+ return "Failed to perform query due to " + ioe;
+ }
+
+ @Override
+ public int getPercentComplete() {
+ return 0;
+ }
+
+ @Override
+ public boolean isFinished() {
+ return true;
+ }
+ });
+ }
+
+ queryManager.registerSubmission(submission);
+ return submission;
+ }
+
+
+ @Override
+ public QuerySubmission submitQuery(final Query query) {
+ return queryManager.submitQuery(query);
+ }
+
+ @Override
+ public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
+ return queryManager.retrieveQuerySubmission(queryIdentifier);
+ }
+
+ @Override
+ public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) {
+ return queryManager.submitLineageComputation(flowFileUuid);
+ }
+
+ @Override
+ public ComputeLineageSubmission retrieveLineageSubmission(final String lineageIdentifier) {
+ return queryManager.retrieveLineageSubmission(lineageIdentifier);
+ }
+
+ @Override
+ public ComputeLineageSubmission submitExpandParents(final long eventId) {
+ return queryManager.submitExpandParents(this, eventId);
+ }
+
+ @Override
+ public ComputeLineageSubmission submitExpandChildren(final long eventId) {
+ return queryManager.submitExpandChildren(this, eventId);
+ }
+
+ @Override
+ public void close() throws IOException {
+ if ( partitionManager != null ) {
+ partitionManager.shutdown();
+ }
+
+ if ( indexManager != null ) {
+ try {
+ indexManager.close();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to shutdown Index Manager due to {}", ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+ }
+
+ if ( queryManager != null ) {
+ try {
+ queryManager.close();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to shutdown Query Manager due to {}", ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+ }
+
+ compressionExecutor.shutdown();
+ workerExecutor.shutdown();
+ queryExecutor.shutdown();
+ }
+
+ @Override
+ public List<SearchableField> getSearchableFields() {
+ final List<SearchableField> searchableFields = new ArrayList<>(config.getSearchableFields());
+ // we exclude the Event Time because it is always searchable and is a bit special in its handling
+ // because it dictates in some cases which index files we look at
+ searchableFields.remove(SearchableFields.EventTime);
+ return searchableFields;
+ }
+
+ @Override
+ public List<SearchableField> getSearchableAttributes() {
+ return config.getSearchableAttributes();
+ }
+
+ @Override
+ public Long getEarliestEventTime() throws IOException {
+ // Get the earliest event timestamp for each partition
+ final Set<Long> earliestTimes = partitionManager.withEachPartitionSerially(new PartitionAction<Long>() {
+ @Override
+ public Long perform(final Partition partition) throws IOException {
+ return partition.getEarliestEventTime();
+ }
+ }, false);
+
+ // Find the latest timestamp for each of the "earliest" timestamps.
+ // This is a bit odd, but we're doing it for a good reason:
+ // The UI is going to show the earliest time available. Because we have a partitioned write-ahead
+ // log, if we just return the timestamp of the earliest event available, we could end up returning
+ // a time for an event that exists but the next event in its lineage does not exist because it was
+ // already aged off of a different journal. To avoid this, we return the "latest of the earliest"
+ // timestamps. This way, we know that no event with a larger ID has been aged off from any of the
+ // partitions.
+ Long latest = null;
+ for ( final Long earliestTime : earliestTimes ) {
+ if ( earliestTime == null ) {
+ continue;
+ }
+
+ if ( latest == null || earliestTime > latest ) {
+ latest = earliestTime;
+ }
+ }
+
+ return latest;
+ }
+
+
+
+ private class ExpireOldEvents implements Runnable {
+ @Override
+ public void run() {
+ final long now = System.currentTimeMillis();
+ final long expirationThreshold = now - config.getEventExpiration(TimeUnit.MILLISECONDS);
+
+ try {
+ indexManager.deleteOldEvents(expirationThreshold);
+ } catch (final IOException ioe) {
+ logger.error("Failed to delete expired events from index due to {}", ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+ }
+
+ try {
+ partitionManager.withEachPartitionSerially(new VoidPartitionAction() {
+ @Override
+ public void perform(final Partition partition) throws IOException {
+ try {
+ partition.deleteOldEvents(expirationThreshold);
+ } catch (final IOException ioe) {
+ logger.error("Failed to delete expired events from Partition {} due to {}", partition, ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+ }
+ }
+ }, false);
+ } catch (IOException ioe) {
+ logger.error("Failed to delete expired events from journals due to {}", ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+ }
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingRepoQuerySubmission.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingRepoQuerySubmission.java
new file mode 100644
index 0000000..f755bc6
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingRepoQuerySubmission.java
@@ -0,0 +1,80 @@
+/*
+ * 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.provenance.journaling;
+
+import java.util.Date;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.query.ProvenanceQueryResult;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
+
+public class JournalingRepoQuerySubmission implements ProvenanceQuerySubmission {
+
+ private final Date submissionTime = new Date();
+ private final String identifier = UUID.randomUUID().toString();
+ private final ProvenanceQueryResult result;
+
+ private final String query;
+ private final Runnable cancelCallback;
+
+ private volatile boolean canceled = false;
+
+ public JournalingRepoQuerySubmission(final String query, final ProvenanceQueryResult queryResult) {
+ this(query, queryResult, null);
+ }
+
+ public JournalingRepoQuerySubmission(final String query, final ProvenanceQueryResult queryResult, final Runnable cancelCallback) {
+ this.query = query;
+ this.cancelCallback = cancelCallback;
+ this.result = queryResult;
+ }
+
+ @Override
+ public String getQuery() {
+ return query;
+ }
+
+ @Override
+ public ProvenanceQueryResult getResult() {
+ return result;
+ }
+
+ @Override
+ public Date getSubmissionTime() {
+ return submissionTime;
+ }
+
+ @Override
+ public String getQueryIdentifier() {
+ return identifier;
+ }
+
+ @Override
+ public void cancel() {
+ this.canceled = true;
+
+ if ( cancelCallback != null ) {
+ cancelCallback.run();
+ }
+ }
+
+ @Override
+ public boolean isCanceled() {
+ return canceled;
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/LazyInitializedProvenanceEvent.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/LazyInitializedProvenanceEvent.java
new file mode 100644
index 0000000..1abd5c9
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/LazyInitializedProvenanceEvent.java
@@ -0,0 +1,373 @@
+/*
+ * 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.provenance.journaling;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexableField;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.journaling.exception.EventNotFoundException;
+import org.apache.nifi.provenance.journaling.index.IndexedFieldNames;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LazyInitializedProvenanceEvent implements StoredProvenanceEvent {
+ private static final Logger logger = LoggerFactory.getLogger(LazyInitializedProvenanceEvent.class);
+
+ private final ProvenanceEventRepository repo;
+ private final StorageLocation storageLocation;
+ private final Document doc;
+ private ProvenanceEventRecord fullRecord;
+
+ public LazyInitializedProvenanceEvent(final ProvenanceEventRepository repo, final StorageLocation storageLocation, final Document document) {
+ this.repo = repo;
+ this.storageLocation = storageLocation;
+ this.doc = document;
+ }
+
+ @Override
+ public long getEventId() {
+ return doc.getField(IndexedFieldNames.EVENT_ID).numericValue().longValue();
+ }
+
+ @Override
+ public StorageLocation getStorageLocation() {
+ return storageLocation;
+ }
+
+ @Override
+ public long getEventTime() {
+ return doc.getField(SearchableFields.EventTime.getSearchableFieldName()).numericValue().longValue();
+ }
+
+ private void ensureFullyLoaded() {
+ if ( fullRecord != null ) {
+ return;
+ }
+
+ final long id = getEventId();
+ try {
+ fullRecord = repo.getEvent(id);
+ } catch (final IOException ioe) {
+ final String containerName = doc.get(IndexedFieldNames.CONTAINER_NAME);
+ final String sectionName = doc.get(IndexedFieldNames.SECTION_NAME);
+ final String journalId = doc.get(IndexedFieldNames.JOURNAL_ID);
+
+ final String error = "Failed to load event with ID " + id + " from container '" + containerName + "', section '" + sectionName + "', journal '" + journalId + "' due to " + ioe;
+ logger.error(error);
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+
+ throw new EventNotFoundException(error);
+ }
+ }
+
+ @Override
+ public long getFlowFileEntryDate() {
+ ensureFullyLoaded();
+ return fullRecord.getFlowFileEntryDate();
+ }
+
+ @Override
+ public long getLineageStartDate() {
+ final IndexableField field = doc.getField(SearchableFields.LineageStartDate.getSearchableFieldName());
+ if ( field != null ) {
+ return field.numericValue().longValue();
+ }
+
+ ensureFullyLoaded();
+ return fullRecord.getLineageStartDate();
+ }
+
+ @Override
+ public Set<String> getLineageIdentifiers() {
+ ensureFullyLoaded();
+ return fullRecord.getLineageIdentifiers();
+ }
+
+ @Override
+ public long getFileSize() {
+ final IndexableField field = doc.getField(SearchableFields.FileSize.getSearchableFieldName());
+ if ( field != null ) {
+ return field.numericValue().longValue();
+ }
+
+ ensureFullyLoaded();
+ return fullRecord.getFileSize();
+ }
+
+ @Override
+ public Long getPreviousFileSize() {
+ ensureFullyLoaded();
+ return fullRecord.getPreviousFileSize();
+ }
+
+ @Override
+ public long getEventDuration() {
+ // TODO: Allow Event Duration to be indexed; it could be interesting for reporting.
+ ensureFullyLoaded();
+ return fullRecord.getEventDuration();
+ }
+
+ @Override
+ public ProvenanceEventType getEventType() {
+ final String name = doc.get(SearchableFields.EventType.getSearchableFieldName());
+ return ProvenanceEventType.valueOf(name.toUpperCase());
+ }
+
+ @Override
+ public Map<String, String> getAttributes() {
+ ensureFullyLoaded();
+ return fullRecord.getAttributes();
+ }
+
+ @Override
+ public String getAttribute(final String attributeName) {
+ final String attr = doc.get(attributeName);
+ if ( attr == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getAttribute(attributeName);
+ } else {
+ return attr;
+ }
+ }
+
+ @Override
+ public Map<String, String> getPreviousAttributes() {
+ ensureFullyLoaded();
+ return fullRecord.getPreviousAttributes();
+ }
+
+ @Override
+ public Map<String, String> getUpdatedAttributes() {
+ ensureFullyLoaded();
+ return fullRecord.getUpdatedAttributes();
+ }
+
+ @Override
+ public String getComponentId() {
+ final String componentId = doc.get(SearchableFields.ComponentID.getSearchableFieldName());
+ if ( componentId == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getComponentId();
+ } else {
+ return componentId;
+ }
+ }
+
+ @Override
+ public String getComponentType() {
+ // TODO: Make indexable.
+ ensureFullyLoaded();
+ return fullRecord.getComponentType();
+ }
+
+ @Override
+ public String getTransitUri() {
+ final String transitUri = doc.get(SearchableFields.TransitURI.getSearchableFieldName());
+ if ( transitUri == null ) {
+ final ProvenanceEventType eventType = getEventType();
+ switch (eventType) {
+ case RECEIVE:
+ case SEND:
+ ensureFullyLoaded();
+ return fullRecord.getTransitUri();
+ default:
+ return null;
+ }
+ } else {
+ return transitUri;
+ }
+ }
+
+ @Override
+ public String getSourceSystemFlowFileIdentifier() {
+ ensureFullyLoaded();
+ return fullRecord.getSourceSystemFlowFileIdentifier();
+ }
+
+ @Override
+ public String getFlowFileUuid() {
+ String uuid = doc.get(SearchableFields.FlowFileUUID.getSearchableFieldName());
+ if ( uuid != null ) {
+ return uuid;
+ }
+
+ ensureFullyLoaded();
+ return fullRecord.getFlowFileUuid();
+ }
+
+ @Override
+ public List<String> getParentUuids() {
+ final IndexableField[] uuids = doc.getFields(SearchableFields.FlowFileUUID.getSearchableFieldName());
+ if ( uuids == null || uuids.length < 2 ) {
+ return Collections.emptyList();
+ }
+
+ switch (getEventType()) {
+ case JOIN: {
+ final List<String> parentUuids = new ArrayList<>(uuids.length - 1);
+ for (int i=1; i < uuids.length; i++) {
+ parentUuids.add(uuids[i].stringValue());
+ }
+ return parentUuids;
+ }
+ default:
+ return Collections.emptyList();
+ }
+ }
+
+ @Override
+ public List<String> getChildUuids() {
+ final IndexableField[] uuids = doc.getFields(SearchableFields.FlowFileUUID.getSearchableFieldName());
+ if ( uuids == null || uuids.length < 2 ) {
+ return Collections.emptyList();
+ }
+
+ switch (getEventType()) {
+ case REPLAY:
+ case CLONE:
+ case FORK: {
+ final List<String> childUuids = new ArrayList<>(uuids.length - 1);
+ for (int i=1; i < uuids.length; i++) {
+ childUuids.add(uuids[i].stringValue());
+ }
+ return childUuids;
+ }
+ default:
+ return Collections.emptyList();
+ }
+ }
+
+ @Override
+ public String getAlternateIdentifierUri() {
+ final String altId = doc.get(SearchableFields.AlternateIdentifierURI.getSearchableFieldName());
+ if ( altId == null && getEventType() == ProvenanceEventType.ADDINFO ) {
+ ensureFullyLoaded();
+ return fullRecord.getAlternateIdentifierUri();
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public String getDetails() {
+ final String details = doc.get(SearchableFields.Details.getSearchableFieldName());
+ if ( details == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getDetails();
+ }
+ return null;
+ }
+
+ @Override
+ public String getRelationship() {
+ final String relationship = doc.get(SearchableFields.Relationship.getSearchableFieldName());
+ if ( relationship == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getRelationship();
+ }
+ return null;
+ }
+
+ @Override
+ public String getSourceQueueIdentifier() {
+ final String queueId = doc.get(SearchableFields.SourceQueueIdentifier.getSearchableFieldName());
+ if ( queueId == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getSourceQueueIdentifier();
+ }
+ return null;
+ }
+
+ @Override
+ public String getContentClaimSection() {
+ final String claimSection = doc.get(SearchableFields.ContentClaimSection.getSearchableFieldName());
+ if ( claimSection == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getContentClaimSection();
+ }
+ return null;
+ }
+
+ @Override
+ public String getPreviousContentClaimSection() {
+ ensureFullyLoaded();
+ return fullRecord.getPreviousContentClaimSection();
+ }
+
+ @Override
+ public String getContentClaimContainer() {
+ final String claimContainer = doc.get(SearchableFields.ContentClaimContainer.getSearchableFieldName());
+ if ( claimContainer == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getContentClaimContainer();
+ }
+ return null;
+ }
+
+ @Override
+ public String getPreviousContentClaimContainer() {
+ ensureFullyLoaded();
+ return fullRecord.getPreviousContentClaimContainer();
+ }
+
+ @Override
+ public String getContentClaimIdentifier() {
+ final String claimIdentifier = doc.get(SearchableFields.ContentClaimIdentifier.getSearchableFieldName());
+ if ( claimIdentifier == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getContentClaimIdentifier();
+ }
+ return null;
+ }
+
+ @Override
+ public String getPreviousContentClaimIdentifier() {
+ ensureFullyLoaded();
+ return fullRecord.getPreviousContentClaimIdentifier();
+ }
+
+ @Override
+ public Long getContentClaimOffset() {
+ final String claimOffset = doc.get(SearchableFields.ContentClaimOffset.getSearchableFieldName());
+ if ( claimOffset == null ) {
+ ensureFullyLoaded();
+ return fullRecord.getContentClaimOffset();
+ }
+ return null;
+ }
+
+ @Override
+ public Long getPreviousContentClaimOffset() {
+ ensureFullyLoaded();
+ return fullRecord.getPreviousContentClaimOffset();
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/ProgressAwareIterator.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/ProgressAwareIterator.java
new file mode 100644
index 0000000..c473224
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/ProgressAwareIterator.java
@@ -0,0 +1,25 @@
+/*
+ * 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.provenance.journaling;
+
+import java.util.Iterator;
+
+public interface ProgressAwareIterator<T> extends Iterator<T> {
+
+ int getPercentComplete();
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
new file mode 100644
index 0000000..c1a7de8
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
@@ -0,0 +1,388 @@
+/*
+ * 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.provenance.journaling.config;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.provenance.search.SearchableField;
+
+public class JournalingRepositoryConfig {
+ private Map<String, File> containers = new HashMap<>();
+ private Map<String, Long> containerCapacities = new HashMap<>();
+
+ private long expirationMillis = TimeUnit.MILLISECONDS.convert(24, TimeUnit.HOURS);
+ private long storageCapacity = 1024L * 1024L * 1024L; // 1 GB
+ private long rolloverMillis = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES);
+ private long journalCapacity = 1024L * 1024L * 5L; // 5 MB
+ private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB
+ private int partitionCount = 16;
+ private int blockSize = 5000;
+ private int indexesPerContainer = 1;
+ private long expirationFrequency = TimeUnit.MINUTES.toNanos(2L);
+
+ private List<SearchableField> searchableFields = new ArrayList<>();
+ private List<SearchableField> searchableAttributes = new ArrayList<>();
+ private boolean compress = true;
+ private boolean alwaysSync = false;
+ private int workerThreadPoolSize = 2;
+ private int queryThreadPoolSize = 2;
+ private int compressionThreadPoolSize = 1;
+ private boolean readOnly = false;
+
+ public void setReadOnly(final boolean readOnly) {
+ this.readOnly = readOnly;
+ }
+
+ public boolean isReadOnly() {
+ return readOnly;
+ }
+
+ public int getIndexesPerContainer() {
+ return indexesPerContainer;
+ }
+
+ /**
+ * Specifies where the repository will store data
+ *
+ * @return
+ */
+ public Map<String, File> getContainers() {
+ return Collections.unmodifiableMap(containers);
+ }
+
+ /**
+ * Specifies where the repository should store data
+ *
+ * @param storageDirectory
+ */
+ public void setContainers(final Map<String, File> containers) {
+ this.containers = new HashMap<>(containers);
+ }
+
+
+ public long getMaxCapacity(final String containerName) {
+ final Long maxCapacity = containerCapacities.get(containerName);
+ if ( maxCapacity == null ) {
+ return getMaxStorageCapacity();
+ } else {
+ return maxCapacity;
+ }
+ }
+
+ public void setMaxContainerCapacities(final Map<String, Long> containerCapacities) {
+ this.containerCapacities = new HashMap<>(containerCapacities);
+ }
+
+ public void setMaxContainerCapacity(final String containerName, final long maxCapacity) {
+ if ( maxCapacity < 1 ) {
+ throw new IllegalArgumentException("Cannot set max container capacity for " + containerName + " to " + maxCapacity + " bytes");
+ }
+
+ this.containerCapacities.put(containerName, maxCapacity);
+ }
+ /**
+ * Returns the maximum amount of time that a given record will stay in the
+ * repository
+ *
+ * @param timeUnit
+ * @return
+ */
+ public long getEventExpiration(final TimeUnit timeUnit) {
+ return timeUnit.convert(expirationMillis, TimeUnit.MILLISECONDS);
+ }
+
+ /**
+ * Specifies how long a record should stay in the repository
+ *
+ * @param expiration
+ * @param timeUnit
+ */
+ public void setEventExpiration(final long expiration, final TimeUnit timeUnit) {
+ this.expirationMillis = TimeUnit.MILLISECONDS.convert(expiration, timeUnit);
+ }
+
+ /**
+ * Returns the maximum amount of data to store in the repository (in bytes)
+ *
+ * @return
+ */
+ public long getMaxStorageCapacity() {
+ return storageCapacity;
+ }
+
+ /**
+ * Sets the maximum amount of data to store in the repository (in bytes)
+ * @param maxStorageCapacity
+ */
+ public void setMaxStorageCapacity(final long maxStorageCapacity) {
+ this.storageCapacity = maxStorageCapacity;
+ }
+
+ /**
+ * Returns the maximum amount of time to write to a single event file
+ *
+ * @param timeUnit
+ * @return
+ */
+ public long getJournalRolloverPeriod(final TimeUnit timeUnit) {
+ return timeUnit.convert(rolloverMillis, TimeUnit.MILLISECONDS);
+ }
+
+ /**
+ * Sets the maximum amount of time to write to a single event file
+ *
+ * @param rolloverPeriod
+ * @param timeUnit
+ */
+ public void setJournalRolloverPeriod(final long rolloverPeriod, final TimeUnit timeUnit) {
+ this.rolloverMillis = TimeUnit.MILLISECONDS.convert(rolloverPeriod, timeUnit);
+ }
+
+ /**
+ * Returns the number of bytes (pre-compression) that will be
+ * written to a single journal file before the file is rolled over
+ *
+ * @return
+ */
+ public long getJournalCapacity() {
+ return journalCapacity;
+ }
+
+ /**
+ * Sets the number of bytes (pre-compression) that will be written
+ * to a single journal file before the file is rolled over
+ *
+ * @param journalCapacity
+ */
+ public void setJournalCapacity(final long journalCapacity) {
+ this.journalCapacity = journalCapacity;
+ }
+
+ /**
+ * Returns the fields that can be indexed
+ *
+ * @return
+ */
+ public List<SearchableField> getSearchableFields() {
+ return Collections.unmodifiableList(searchableFields);
+ }
+
+ /**
+ * Sets the fields to index
+ *
+ * @param searchableFields
+ */
+ public void setSearchableFields(final List<SearchableField> searchableFields) {
+ this.searchableFields = new ArrayList<>(searchableFields);
+ }
+
+ /**
+ * Returns the FlowFile attributes that can be indexed
+ *
+ * @return
+ */
+ public List<SearchableField> getSearchableAttributes() {
+ return Collections.unmodifiableList(searchableAttributes);
+ }
+
+ /**
+ * Sets the FlowFile attributes to index
+ *
+ * @param searchableAttributes
+ */
+ public void setSearchableAttributes(final List<SearchableField> searchableAttributes) {
+ this.searchableAttributes = new ArrayList<>(searchableAttributes);
+ }
+
+ /**
+ * Indicates whether or not event files will be compressed when they are
+ * rolled over
+ *
+ * @return
+ */
+ public boolean isCompressOnRollover() {
+ return compress;
+ }
+
+ /**
+ * Specifies whether or not to compress event files on rollover
+ *
+ * @param compress
+ */
+ public void setCompressOnRollover(final boolean compress) {
+ this.compress = compress;
+ }
+
+ public int getWorkerThreadPoolSize() {
+ return workerThreadPoolSize;
+ }
+
+ public void setWorkerThreadPoolSize(final int workerThreadPoolSize) {
+ if (workerThreadPoolSize < 1) {
+ throw new IllegalArgumentException();
+ }
+ this.workerThreadPoolSize = workerThreadPoolSize;
+ }
+
+
+
+ public int getQueryThreadPoolSize() {
+ return queryThreadPoolSize;
+ }
+
+ public void setQueryThreadPoolSize(int queryThreadPoolSize) {
+ if (queryThreadPoolSize < 1) {
+ throw new IllegalArgumentException();
+ }
+ this.queryThreadPoolSize = queryThreadPoolSize;
+ }
+
+ /**
+ * <p>
+ * Specifies the desired size of each Provenance Event index shard, in
+ * bytes. We shard the index for a few reasons:
+ * </p>
+ *
+ * <ol>
+ * <li>
+ * A very large index requires a significant amount of Java heap space to
+ * search. As the size of the shard increases, the required Java heap space
+ * also increases.
+ * </li>
+ * <li>
+ * By having multiple shards, we have the ability to use multiple concurrent
+ * threads to search the individual shards, resulting in far less latency
+ * when performing a search across millions or billions of records.
+ * </li>
+ * <li>
+ * We keep track of which time ranges each index shard spans. As a result,
+ * we are able to determine which shards need to be searched if a search
+ * provides a date range. This can greatly increase the speed of a search
+ * and reduce resource utilization.
+ * </li>
+ * </ol>
+ *
+ * @param bytes
+ */
+ public void setDesiredIndexSize(final long bytes) {
+ this.desiredIndexBytes = bytes;
+ }
+
+ /**
+ * Returns the desired size of each index shard. See the
+ * {@Link #setDesiredIndexSize} method for an explanation of why we choose
+ * to shard the index.
+ *
+ * @return
+ */
+ public long getDesiredIndexSize() {
+ return desiredIndexBytes;
+ }
+
+ /**
+ * Sets the number of Journal files to use when persisting records.
+ *
+ * @param numJournals
+ */
+ public void setPartitionCount(final int numJournals) {
+ if (numJournals < 1) {
+ throw new IllegalArgumentException();
+ }
+
+ this.partitionCount = numJournals;
+ }
+
+ /**
+ * Returns the number of Journal files that will be used when persisting
+ * records.
+ *
+ * @return
+ */
+ public int getPartitionCount() {
+ return partitionCount;
+ }
+
+ /**
+ * Specifies whether or not the Repository should sync all updates to disk.
+ *
+ * @return
+ */
+ public boolean isAlwaysSync() {
+ return alwaysSync;
+ }
+
+ /**
+ * Configures whether or not the Repository should sync all updates to disk.
+ * Setting this value to true means that updates are guaranteed to be
+ * persisted across restarted, even if there is a power failure or a sudden
+ * Operating System crash, but it can be very expensive.
+ *
+ * @param alwaysSync
+ */
+ public void setAlwaysSync(boolean alwaysSync) {
+ this.alwaysSync = alwaysSync;
+ }
+
+ /**
+ * Returns the minimum number of Provenance Events that should be written to a single Block.
+ * Events are written out in blocks, which are later optionally compressed. A larger block size
+ * will potentially result in better compression. However, a smaller block size will result
+ * in better performance when reading the data. The default value is 5000 events per block.
+ *
+ * @return
+ */
+ public int getBlockSize() {
+ return blockSize;
+ }
+
+ /**
+ * Sets the minimum number of Provenance Events that should be written to a single Block.
+ * Events are written out in blocks, which are later optionally compressed. A larger block size
+ * will potentially result in better compression. However, a smaller block size will result
+ * in better performance when reading the data. The default value is 5000 events per block.
+ *
+ * @return
+ */
+ public void setBlockSize(final int blockSize) {
+ if ( blockSize < 1 ) {
+ throw new IllegalArgumentException("Cannot set block size to " + blockSize + "; must be a positive integer");
+ }
+ this.blockSize = blockSize;
+ }
+
+ public int getCompressionThreadPoolSize() {
+ return compressionThreadPoolSize;
+ }
+
+ public void setCompressionThreadPoolSize(final int compressionThreadPoolSize) {
+ this.compressionThreadPoolSize = compressionThreadPoolSize;
+ }
+
+ public long getExpirationFrequency(final TimeUnit unit) {
+ return unit.convert(expirationFrequency, TimeUnit.NANOSECONDS);
+ }
+
+ public void setExpirationFrequency(final long period, final TimeUnit unit) {
+ this.expirationFrequency = unit.toNanos(period);
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/exception/EventNotFoundException.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/exception/EventNotFoundException.java
new file mode 100644
index 0000000..398cf8e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/exception/EventNotFoundException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.provenance.journaling.exception;
+
+public class EventNotFoundException extends RuntimeException {
+ private static final long serialVersionUID = -8490783814308479930L;
+
+ public EventNotFoundException(final String message) {
+ super(message);
+ }
+
+ public EventNotFoundException(final String message, final Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
new file mode 100644
index 0000000..fbc5746
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
@@ -0,0 +1,107 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.LazyInitializedProvenanceEvent;
+import org.apache.nifi.provenance.search.Query;
+
+public interface EventIndexSearcher extends Closeable {
+ /**
+ * Searches the repository for any events that match the provided query and returns the locations
+ * where those events are stored
+ * @param query
+ * @return
+ */
+ SearchResult search(Query query) throws IOException;
+
+ /**
+ * Returns the locations of all events for a FlowFile that has a FlowFile UUID in the collection of
+ * UUIDs provided, if the event time occurs between earliestTime and latestTime. The return value is
+ * ordered in the order in which the records should be read from the journals in order to obtain
+ * maximum efficiency
+ *
+ * @param flowFileUuids
+ * @param earliestTime
+ * @param latestTime
+ *
+ * @return
+ * @throws IOException
+ */
+ List<JournaledStorageLocation> getEventsForFlowFiles(Collection<String> flowFileUuids, long earliestTime, long latestTime) throws IOException;
+
+ /**
+ * Returns the locations of events that have Event ID's at least equal to minEventId, and returns
+ * up to the given number of results
+ *
+ * @param minEventId
+ * @param maxResults
+ * @return
+ * @throws IOException
+ */
+ List<JournaledStorageLocation> getEvents(long minEventId, int maxResults) throws IOException;
+
+ /**
+ * Returns the largest event id that is known by the index being searched
+ * @param container
+ * @param section
+ * @return
+ * @throws IOException
+ */
+ Long getMaxEventId(String container, String section) throws IOException;
+
+ /**
+ * Returns the locations of the latest events for the index being searched
+ * @param numEvents
+ * @return
+ * @throws IOException
+ */
+ List<JournaledStorageLocation> getLatestEvents(int numEvents) throws IOException;
+
+ /**
+ * Returns the total number of events that exist for the index being searched
+ * @return
+ * @throws IOException
+ */
+ long getNumberOfEvents() throws IOException;
+
+ /**
+ * Evaluates the given query against the index, returning an iterator of lazily initialized provenance events
+ *
+ * @param query
+ * @param referencedFields the set of fields that are referenced in the query
+ * @throws IOException
+ */
+ Iterator<LazyInitializedProvenanceEvent> select(String query, Set<String> referencedFields) throws IOException;
+
+ /**
+ * Evaluates the given query against the index, returning an iterator of locations from which the matching
+ * records can be retrieved
+ *
+ * @param query
+ * @return
+ * @throws IOException
+ */
+ Iterator<JournaledStorageLocation> selectLocations(String query) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
new file mode 100644
index 0000000..a151838
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
@@ -0,0 +1,66 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+
+public interface EventIndexWriter extends Closeable {
+
+ /**
+ * Adds all of the events to the index.
+ * @param events
+ * @throws IOException
+ */
+ void index(final Collection<JournaledProvenanceEvent> events) throws IOException;
+
+ /**
+ * Forces all updates to the index to be pushed to disk.
+ * @throws IOException
+ */
+ void sync() throws IOException;
+
+ /**
+ * Deletes any records that belong to the given container/section/journal
+ * @param containerName
+ * @param section
+ * @param journalId
+ * @throws IOException
+ */
+ void delete(String containerName, String section, Long journalId) throws IOException;
+
+ /**
+ * Deletes any records that belong to the given container and section but have a journal Id less
+ * than the specified value
+ * @param containerName
+ * @param section
+ * @param journalId
+ * @throws IOException
+ */
+ void deleteEventsBefore(String containerName, String section, Long journalId) throws IOException;
+
+
+ /**
+ * Removes all events from the index that occurred before the given time
+ * @param earliestEventTimeToDelete
+ * @throws IOException
+ */
+ void deleteOldEvents(long earliestEventTimeToDelete) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexAction.java
new file mode 100644
index 0000000..6486d56
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexAction.java
@@ -0,0 +1,23 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.IOException;
+
+public interface IndexAction<T> {
+ T perform(EventIndexSearcher searcher) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
new file mode 100644
index 0000000..9e1fc39
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.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.provenance.journaling.index;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Set;
+
+public interface IndexManager extends Closeable {
+
+ /**
+ * Returns an EventIndexWriter for the given container.
+ * @param container
+ * @return
+ */
+ EventIndexWriter getIndexWriter(final String container);
+
+ /**
+ * Returns the max event ID that has been indexed for the given container and section.
+ *
+ * @param container
+ * @param section
+ * @return
+ */
+ Long getMaxEventId(String container, String section) throws IOException;
+
+ /**
+ * Returns the total number of indices for all containers
+ * @return
+ */
+ int getNumberOfIndices();
+
+ /**
+ * Returns a new {@link EventIndexSearcher} that can be used to search the indices for
+ * the given container
+ *
+ * @param containerName the containerName to search
+ *
+ * @return
+ * @throws IOException
+ */
+ EventIndexSearcher newIndexSearcher(String containerName) throws IOException;
+
+ /**
+ * Executes the given action against each index and returns a Set of results,
+ * where each result is obtained from performing the given action against one index
+ *
+ * @param action the action to perform
+ * @return
+ * @throws IOException
+ */
+ <T> Set<T> withEachIndex(IndexAction<T> action) throws IOException;
+
+ /**
+ * Executes the given action against each index and returns a Set of results,
+ * where each result is obtained from performing the given action against one index
+ *
+ * @param action the action to perform
+ * @param autoClose whether or not the EventIndexSearcher should automatically be closed
+ *
+ * @return
+ * @throws IOException
+ */
+ <T> Set<T> withEachIndex(IndexAction<T> action, boolean autoClose) throws IOException;
+
+ /**
+ * Returns an Iterator of EventIndexSearchers.
+ * @return
+ */
+ Set<EventIndexSearcher> getSearchers() throws IOException;
+
+ /**
+ * Performs the given action against each index, waiting for the action to complete
+ * against each index before returning
+ *
+ * @param action the action to perform against each index
+ * @throws IOException
+ */
+ void withEachIndex(VoidIndexAction action) throws IOException;
+
+ /**
+ * Performs the given action against each index
+ *
+ * @param action the action to perform
+ *
+ * @param async if true, the method will return immediatley and the actions will occur
+ * in the background. If <code>false</code>, the method will not return until the action
+ * has been performed against all indices
+ *
+ * @throws IOException
+ */
+ void withEachIndex(VoidIndexAction action, boolean async) throws IOException;
+
+ /**
+ * Removes any events that have a Storage Location that includes the provided containerName, secitonIndex, and journalId,
+ * and then re-adds all of the events that are in the given journalFile.
+ * @param containerName
+ * @param sectionIndex
+ * @param journalId
+ * @param journalFile
+ * @throws IOException
+ */
+ void reindex(final String containerName, final int sectionIndex, final Long journalId, final File journalFile) throws IOException;
+
+ /**
+ * Syncs all indices
+ * @throws IOException
+ */
+ void sync() throws IOException;
+
+ /**
+ * Returns the total number of events in all indices
+ * @return
+ */
+ long getNumberOfEvents() throws IOException;
+
+ /**
+ * Removes all events from all indices that occurred before the given time
+ * @param earliestEventTimeToDelete
+ * @throws IOException
+ */
+ void deleteOldEvents(long earliestEventTimeToDelete) throws IOException;
+
+ /**
+ * Deletes any events from the index that belong to the given container, section, and journal id
+ *
+ * @param containerName
+ * @param sectionIndex
+ * @param journalId
+ *
+ * @throws IOException
+ */
+ void deleteEvents(String containerName, int sectionIndex, Long journalId) throws IOException;
+
+ /**
+ * Deletes any events from the index that belong to the given container and section but have
+ * a journal id before the value specified
+ *
+ * @param containerName
+ * @param sectionIndex
+ * @param journalId
+ *
+ * @throws IOException
+ */
+ void deleteEventsBefore(String containerName, int sectionIndex, Long journalId) throws IOException;
+
+ /**
+ * Returns the size (in bytes) of the index for the given container
+ * @param containerName
+ * @return
+ */
+ long getSize(String containerName);
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexedFieldNames.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexedFieldNames.java
new file mode 100644
index 0000000..977df9f
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexedFieldNames.java
@@ -0,0 +1,27 @@
+/*
+ * 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.provenance.journaling.index;
+
+public class IndexedFieldNames {
+
+ public static final String CONTAINER_NAME = "containerName";
+ public static final String SECTION_NAME = "sectionName";
+ public static final String JOURNAL_ID = "journalId";
+ public static final String BLOCK_INDEX = "blockIndex";
+ public static final String EVENT_ID = "eventId";
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/JournalingSearchableFields.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/JournalingSearchableFields.java
new file mode 100644
index 0000000..9ec9f5d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/JournalingSearchableFields.java
@@ -0,0 +1,29 @@
+/*
+ * 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.provenance.journaling.index;
+
+import org.apache.nifi.provenance.NamedSearchableField;
+import org.apache.nifi.provenance.search.SearchableField;
+
+public class JournalingSearchableFields {
+ public static SearchableField CONTAINER_NAME = new NamedSearchableField(IndexedFieldNames.CONTAINER_NAME, IndexedFieldNames.CONTAINER_NAME, "Container Name", false);
+ public static SearchableField SECTION_NAME = new NamedSearchableField(IndexedFieldNames.SECTION_NAME, IndexedFieldNames.SECTION_NAME, "Section Name", false);
+ public static SearchableField JOURNAL_ID = new NamedSearchableField(IndexedFieldNames.JOURNAL_ID, IndexedFieldNames.JOURNAL_ID, "Journal ID", false);
+ public static SearchableField BLOCK_INDEX = new NamedSearchableField(IndexedFieldNames.BLOCK_INDEX, IndexedFieldNames.BLOCK_INDEX, "Block Index", false);
+ public static SearchableField EVENT_ID = new NamedSearchableField(IndexedFieldNames.EVENT_ID, IndexedFieldNames.EVENT_ID, "Event ID", false);
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
new file mode 100644
index 0000000..1f53d27
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
@@ -0,0 +1,451 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.toc.TocJournalReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LuceneIndexManager implements IndexManager {
+ private static final Logger logger = LoggerFactory.getLogger(LuceneIndexManager.class);
+
+ private final ProvenanceEventRepository repo;
+ private final JournalingRepositoryConfig config;
+ private final ExecutorService queryExecutor;
+
+ private final Map<String, List<LuceneIndexWriter>> writers = new HashMap<>();
+ private final Map<String, AtomicLong> writerIndexes = new HashMap<>();
+ private final ConcurrentMap<String, IndexSize> indexSizes = new ConcurrentHashMap<>();
+
+ public LuceneIndexManager(final ProvenanceEventRepository repo, final JournalingRepositoryConfig config, final ScheduledExecutorService workerExecutor, final ExecutorService queryExecutor) throws IOException {
+ this.repo = repo;
+ this.config = config;
+ this.queryExecutor = queryExecutor;
+
+ final int rolloverSeconds = (int) config.getJournalRolloverPeriod(TimeUnit.SECONDS);
+ if ( !config.isReadOnly() ) {
+ for ( final Map.Entry<String, File> entry : config.getContainers().entrySet() ) {
+ final String containerName = entry.getKey();
+ final File container = entry.getValue();
+
+ final List<LuceneIndexWriter> writerList = new ArrayList<>(config.getIndexesPerContainer());
+ writers.put(containerName, writerList);
+ writerIndexes.put(containerName, new AtomicLong(0L));
+
+ for ( int i=0; i < config.getIndexesPerContainer(); i++ ){
+ final File indexDir = new File(container, "indices/" + i);
+ writerList.add(new LuceneIndexWriter(repo, indexDir, config));
+ }
+
+ workerExecutor.scheduleWithFixedDelay(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ sync(containerName);
+ } catch (final Throwable t) {
+ logger.error("Failed to sync Provenance Repository Container {} due to {}", containerName, t);
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", t);
+ }
+ }
+ }
+ }, rolloverSeconds, rolloverSeconds, TimeUnit.SECONDS);
+ }
+ }
+ }
+
+ @Override
+ public EventIndexSearcher newIndexSearcher(final String containerName) throws IOException {
+ final File containerDir = config.getContainers().get(containerName);
+ if ( containerDir == null ) {
+ throw new IllegalArgumentException();
+ }
+
+ final List<EventIndexSearcher> searchers = new ArrayList<>();
+
+ try {
+ if (config.isReadOnly()) {
+ for (int i=0; i < config.getIndexesPerContainer(); i++) {
+ final File indexDir = new File(containerName, "indices/" + i);
+ searchers.add(new LuceneIndexSearcher(repo, indexDir));
+ }
+ } else {
+ final List<LuceneIndexWriter> writerList = writers.get(containerName);
+ for ( final LuceneIndexWriter writer : writerList ) {
+ searchers.add(writer.newIndexSearcher());
+ }
+ }
+ } catch (final IOException ioe) {
+ // If we failed to create a searcher, we need to close all that we've already created.
+ for ( final EventIndexSearcher searcher : searchers ) {
+ try {
+ searcher.close();
+ } catch (final IOException ioe2) {
+ ioe.addSuppressed(ioe2);
+ }
+ }
+
+ throw ioe;
+ }
+
+ return new MultiIndexSearcher(searchers);
+ }
+
+ @Override
+ public LuceneIndexWriter getIndexWriter(final String container) {
+ if (config.isReadOnly() ) {
+ throw new IllegalStateException("Cannot obtain Index Writer because repository is read-only");
+ }
+
+ final AtomicLong index = writerIndexes.get(container);
+ if (index == null ) {
+ throw new IllegalArgumentException();
+ }
+
+ final long curVal = index.get();
+ final List<LuceneIndexWriter> writerList = writers.get(container);
+ return writerList.get((int) (curVal % writerList.size()));
+ }
+
+ @Override
+ public Long getMaxEventId(final String container, final String section) throws IOException {
+ final List<LuceneIndexWriter> writerList = writers.get(container);
+ if ( writerList == null ) {
+ return null;
+ }
+
+ Long max = null;
+ for ( final LuceneIndexWriter writer : writerList ) {
+ try (final EventIndexSearcher searcher = writer.newIndexSearcher()) {
+ final Long maxForWriter = searcher.getMaxEventId(container, section);
+ if ( maxForWriter != null ) {
+ if (max == null || maxForWriter.longValue() > max.longValue() ) {
+ max = maxForWriter;
+ }
+ }
+ }
+ }
+
+ return max;
+ }
+
+ @Override
+ public void sync() throws IOException {
+ for ( final List<LuceneIndexWriter> writerList : writers.values() ) {
+ for ( final LuceneIndexWriter writer : writerList ) {
+ writer.sync();
+ }
+ }
+ }
+
+
+ private void sync(final String containerName) throws IOException {
+ final AtomicLong index = writerIndexes.get(containerName);
+ final long curIndex = index.getAndIncrement();
+
+ final List<LuceneIndexWriter> writerList = writers.get(containerName);
+ final EventIndexWriter toSync = writerList.get((int) (curIndex % writerList.size()));
+ toSync.sync();
+ }
+
+ @Override
+ public void close() throws IOException {
+ for ( final List<LuceneIndexWriter> writerList : writers.values() ) {
+ for ( final LuceneIndexWriter writer : writerList ) {
+ try {
+ writer.close();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to close {} due to {}", writer, ioe);
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+ }
+ }
+ }
+
+
+ @Override
+ public Set<EventIndexSearcher> getSearchers() throws IOException {
+ final Set<EventIndexSearcher> searchers = new HashSet<>();
+
+ try {
+ final Set<String> containerNames = config.getContainers().keySet();
+ for (final String containerName : containerNames) {
+ final EventIndexSearcher searcher = newIndexSearcher(containerName);
+ searchers.add(searcher);
+ }
+
+ return searchers;
+ } catch (final Exception e) {
+ for ( final EventIndexSearcher searcher : searchers ) {
+ try {
+ searcher.close();
+ } catch (final IOException ioe) {
+ e.addSuppressed(ioe);
+ }
+ }
+
+ throw e;
+ }
+ }
+
+ @Override
+ public <T> Set<T> withEachIndex(final IndexAction<T> action) throws IOException {
+ return withEachIndex(action, true);
+ }
+
+ @Override
+ public <T> Set<T> withEachIndex(final IndexAction<T> action, final boolean autoClose) throws IOException {
+ final Set<T> results = new HashSet<>();
+ final Map<String, Future<T>> futures = new HashMap<>();
+ final Set<String> containerNames = config.getContainers().keySet();
+ for (final String containerName : containerNames) {
+ final Callable<T> callable = new Callable<T>() {
+ @Override
+ public T call() throws Exception {
+ final EventIndexSearcher searcher = newIndexSearcher(containerName);
+ try {
+ return action.perform(searcher);
+ } finally {
+ if ( autoClose ) {
+ searcher.close();
+ }
+ }
+ }
+ };
+
+ final Future<T> future = queryExecutor.submit(callable);
+ futures.put(containerName, future);
+ }
+
+ for ( final Map.Entry<String, Future<T>> entry : futures.entrySet() ) {
+ try {
+ final T result = entry.getValue().get();
+ results.add(result);
+ } catch (final ExecutionException ee) {
+ final Throwable cause = ee.getCause();
+ if ( cause instanceof IOException ) {
+ throw (IOException) cause;
+ } else {
+ throw new RuntimeException("Failed to query Container " + entry.getKey() + " due to " + cause, cause);
+ }
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ return results;
+ }
+
+ @Override
+ public void withEachIndex(final VoidIndexAction action) throws IOException {
+ withEachIndex(action, false);
+ }
+
+ @Override
+ public void withEachIndex(final VoidIndexAction action, final boolean async) throws IOException {
+ final Map<String, Future<?>> futures = new HashMap<>();
+ final Set<String> containerNames = config.getContainers().keySet();
+
+ for (final String containerName : containerNames) {
+ final Callable<Object> callable = new Callable<Object>() {
+ @Override
+ public Object call() throws IOException {
+ try (final EventIndexSearcher searcher = newIndexSearcher(containerName)) {
+ action.perform(searcher);
+ return null;
+ } catch (final Throwable t) {
+ if ( async ) {
+ logger.error("Failed to perform action against container " + containerName + " due to " + t, t);
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", t);
+ }
+
+ return null;
+ } else {
+ throw new IOException("Failed to perform action against container " + containerName + " due to " + t, t);
+ }
+ }
+ }
+ };
+
+ final Future<?> future = queryExecutor.submit(callable);
+ futures.put(containerName, future);
+ }
+
+ if ( !async ) {
+ for ( final Map.Entry<String, Future<?>> entry : futures.entrySet() ) {
+ try {
+ // throw any exception thrown by runnable
+ entry.getValue().get();
+ } catch (final ExecutionException ee) {
+ final Throwable cause = ee.getCause();
+ if ( cause instanceof IOException ) {
+ throw ((IOException) cause);
+ }
+
+ throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+ }
+
+ @Override
+ public int getNumberOfIndices() {
+ return config.getContainers().size();
+ }
+
+ @Override
+ public void deleteEvents(final String containerName, final int sectionIndex, final Long journalId) throws IOException {
+ final List<LuceneIndexWriter> writerList = writers.get(containerName);
+ for ( final LuceneIndexWriter writer : writerList ) {
+ writer.delete(containerName, String.valueOf(sectionIndex), journalId);
+ }
+ }
+
+ @Override
+ public void deleteEventsBefore(final String containerName, final int sectionIndex, final Long journalId) throws IOException {
+ final List<LuceneIndexWriter> writerList = writers.get(containerName);
+ for ( final LuceneIndexWriter writer : writerList ) {
+ writer.deleteEventsBefore(containerName, String.valueOf(sectionIndex), journalId);
+ }
+ }
+
+ @Override
+ public void reindex(final String containerName, final int sectionIndex, final Long journalId, final File journalFile) throws IOException {
+ deleteEvents(containerName, sectionIndex, journalId);
+
+ final LuceneIndexWriter writer = getIndexWriter(containerName);
+ try (final TocJournalReader reader = new TocJournalReader(containerName, String.valueOf(sectionIndex), journalId, journalFile)) {
+ final List<JournaledProvenanceEvent> events = new ArrayList<>(1000);
+ JournaledProvenanceEvent event;
+
+ while ((event = reader.nextJournaledEvent()) != null) {
+ events.add(event);
+ if ( events.size() >= 1000 ) {
+ writer.index(events);
+ events.clear();
+ }
+ }
+
+ if (!events.isEmpty() ) {
+ writer.index(events);
+ }
+ }
+ }
+
+ @Override
+ public long getNumberOfEvents() throws IOException {
+ final AtomicLong totalCount = new AtomicLong(0L);
+ withEachIndex(new VoidIndexAction() {
+ @Override
+ public void perform(final EventIndexSearcher searcher) throws IOException {
+ totalCount.addAndGet(searcher.getNumberOfEvents());
+ }
+ });
+
+ return totalCount.get();
+ }
+
+ @Override
+ public void deleteOldEvents(final long earliestEventTimeToDelete) throws IOException {
+ for ( final String containerName : config.getContainers().keySet() ) {
+ final List<LuceneIndexWriter> writerList = writers.get(containerName);
+ for ( final LuceneIndexWriter writer : writerList ) {
+ writer.deleteOldEvents(earliestEventTimeToDelete);
+ }
+ }
+ }
+
+
+ @Override
+ public long getSize(final String containerName) {
+ // Cache index sizes so that we don't have to continually calculate it, as calculating it requires
+ // disk accesses, which are quite expensive.
+ final IndexSize indexSize = indexSizes.get(containerName);
+ if ( indexSize != null && !indexSize.isExpired() ) {
+ return indexSize.getSize();
+ }
+
+ final File containerFile = config.getContainers().get(containerName);
+ final File indicesDir = new File(containerFile, "indices");
+
+ final long size = getSize(indicesDir);
+ indexSizes.put(containerName, new IndexSize(size));
+ return size;
+ }
+
+ private long getSize(final File file) {
+ if ( file.isDirectory() ) {
+ long totalSize = 0L;
+
+ final File[] children = file.listFiles();
+ if ( children != null ) {
+ for ( final File child : children ) {
+ totalSize += getSize(child);
+ }
+ }
+
+ return totalSize;
+ } else {
+ return file.length();
+ }
+ }
+
+
+ private static class IndexSize {
+ private final long size;
+ private final long expirationTime;
+
+ public IndexSize(final long size) {
+ this.size = size;
+ this.expirationTime = System.currentTimeMillis() + 5000L; // good for 5 seconds
+ }
+
+ public long getSize() {
+ return size;
+ }
+
+ public boolean isExpired() {
+ return System.currentTimeMillis() > expirationTime;
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
new file mode 100644
index 0000000..9bc123f
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
@@ -0,0 +1,333 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortField.Type;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopFieldDocs;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.nifi.pql.LuceneTranslator;
+import org.apache.nifi.pql.ProvenanceQuery;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.LazyInitializedProvenanceEvent;
+import org.apache.nifi.provenance.journaling.exception.EventNotFoundException;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.util.ObjectHolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LuceneIndexSearcher implements EventIndexSearcher {
+ private static final Logger logger = LoggerFactory.getLogger(LuceneIndexSearcher.class);
+
+ private final ProvenanceEventRepository repo;
+ private final DirectoryReader reader;
+ private final IndexSearcher searcher;
+ private final FSDirectory fsDirectory;
+
+ private final String description;
+
+ private static final Set<String> REQUIRED_FIELDS;
+
+ static {
+ final Set<String> fields = new HashSet<>();
+ fields.add(IndexedFieldNames.BLOCK_INDEX);
+ fields.add(IndexedFieldNames.CONTAINER_NAME);
+ fields.add(IndexedFieldNames.EVENT_ID);
+ fields.add(IndexedFieldNames.JOURNAL_ID);
+ fields.add(IndexedFieldNames.SECTION_NAME);
+ fields.add(SearchableFields.EventTime.getSearchableFieldName());
+ fields.add(SearchableFields.EventType.getSearchableFieldName());
+ REQUIRED_FIELDS = fields;
+ }
+
+ public LuceneIndexSearcher(final ProvenanceEventRepository repo, final File indexDirectory) throws IOException {
+ this.repo = repo;
+ this.fsDirectory = FSDirectory.open(indexDirectory);
+ this.reader = DirectoryReader.open(fsDirectory);
+ this.searcher = new IndexSearcher(reader);
+ this.description = "LuceneIndexSearcher[indexDirectory=" + indexDirectory + "]";
+ }
+
+ public LuceneIndexSearcher(final ProvenanceEventRepository repo, final DirectoryReader reader, final File indexDirectory) {
+ this.repo = repo;
+ this.reader = reader;
+ this.searcher = new IndexSearcher(reader);
+ this.fsDirectory = null;
+ this.description = "LuceneIndexSearcher[indexDirectory=" + indexDirectory + "]";
+ }
+
+ @Override
+ public void close() throws IOException {
+ IOException suppressed = null;
+ try {
+ reader.close();
+ } catch (final IOException ioe) {
+ suppressed = ioe;
+ }
+
+ if ( fsDirectory != null ) {
+ fsDirectory.close();
+ }
+
+ if ( suppressed != null ) {
+ throw suppressed;
+ }
+ }
+
+
+ private List<JournaledStorageLocation> getOrderedLocations(final TopDocs topDocs) throws IOException {
+ final ScoreDoc[] scoreDocs = topDocs.scoreDocs;
+ final List<JournaledStorageLocation> locations = new ArrayList<>(scoreDocs.length);
+ populateLocations(topDocs, locations);
+
+ return locations;
+ }
+
+
+ private void populateLocations(final TopDocs topDocs, final Collection<JournaledStorageLocation> locations) throws IOException {
+ for ( final ScoreDoc scoreDoc : topDocs.scoreDocs ) {
+ final Document document = reader.document(scoreDoc.doc);
+ locations.add(QueryUtils.createLocation(document));
+ }
+ }
+
+
+ @Override
+ public SearchResult search(final Query provenanceQuery) throws IOException {
+ final org.apache.lucene.search.Query luceneQuery = QueryUtils.convertQueryToLucene(provenanceQuery);
+ final TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults());
+ final List<JournaledStorageLocation> locations = getOrderedLocations(topDocs);
+
+ return new SearchResult(locations, topDocs.totalHits);
+ }
+
+ @Override
+ public List<JournaledStorageLocation> getEvents(final long minEventId, final int maxResults) throws IOException {
+ final BooleanQuery query = new BooleanQuery();
+ query.add(NumericRangeQuery.newLongRange(IndexedFieldNames.EVENT_ID, minEventId, null, true, true), Occur.MUST);
+
+ final TopDocs topDocs = searcher.search(query, maxResults, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG)));
+ return getOrderedLocations(topDocs);
+ }
+
+ @Override
+ public Long getMaxEventId(final String container, final String section) throws IOException {
+ final BooleanQuery query = new BooleanQuery();
+
+ if ( container != null ) {
+ query.add(new TermQuery(new Term(IndexedFieldNames.CONTAINER_NAME, container)), Occur.MUST);
+ }
+
+ if ( section != null ) {
+ query.add(new TermQuery(new Term(IndexedFieldNames.SECTION_NAME, section)), Occur.MUST);
+ }
+
+ final TopDocs topDocs = searcher.search(query, 1, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG, true)));
+ final List<JournaledStorageLocation> locations = getOrderedLocations(topDocs);
+ if ( locations.isEmpty() ) {
+ return null;
+ }
+
+ return locations.get(0).getEventId();
+ }
+
+ @Override
+ public List<JournaledStorageLocation> getEventsForFlowFiles(final Collection<String> flowFileUuids, final long earliestTime, final long latestTime) throws IOException {
+ // Create a query for all Events related to the FlowFiles of interest. We do this by adding all ID's as
+ // "SHOULD" clauses and then setting the minimum required to 1.
+ final BooleanQuery flowFileIdQuery;
+ if (flowFileUuids == null || flowFileUuids.isEmpty()) {
+ flowFileIdQuery = null;
+ } else {
+ flowFileIdQuery = new BooleanQuery();
+ for (final String flowFileUuid : flowFileUuids) {
+ flowFileIdQuery.add(new TermQuery(new Term(SearchableFields.FlowFileUUID.getSearchableFieldName(), flowFileUuid)), Occur.SHOULD);
+ }
+ flowFileIdQuery.setMinimumNumberShouldMatch(1);
+ }
+
+ flowFileIdQuery.add(NumericRangeQuery.newLongRange(SearchableFields.EventTime.getSearchableFieldName(),
+ earliestTime, latestTime, true, true), Occur.MUST);
+
+ final TopDocs topDocs = searcher.search(flowFileIdQuery, 1000);
+ return getOrderedLocations(topDocs);
+ }
+
+
+ @Override
+ public List<JournaledStorageLocation> getLatestEvents(final int numEvents) throws IOException {
+ final MatchAllDocsQuery query = new MatchAllDocsQuery();
+
+ final TopFieldDocs topDocs = searcher.search(query, numEvents, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG, true)));
+ final List<JournaledStorageLocation> locations = getOrderedLocations(topDocs);
+ return locations;
+ }
+
+ @Override
+ public String toString() {
+ return description;
+ }
+
+ @Override
+ public long getNumberOfEvents() {
+ return reader.numDocs();
+ }
+
+
+ private <T> Iterator<T> select(final String query, final Set<String> referencedFields, final DocumentTransformer<T> transformer) throws IOException {
+ final org.apache.lucene.search.Query luceneQuery = LuceneTranslator.toLuceneQuery(ProvenanceQuery.compile(query, repo.getSearchableFields(), repo.getSearchableAttributes()).getWhereClause());
+ final int batchSize = 1000000;
+
+ final Set<String> fieldsToLoad;
+ if ( referencedFields == null ) {
+ fieldsToLoad = null;
+ } else {
+ fieldsToLoad = new HashSet<>(REQUIRED_FIELDS);
+ fieldsToLoad.addAll(referencedFields);
+ }
+
+ final ObjectHolder<TopDocs> topDocsHolder = new ObjectHolder<>(null);
+ return new Iterator<T>() {
+ int fetched = 0;
+ int scoreDocIndex = 0;
+
+ @Override
+ public boolean hasNext() {
+ if ( topDocsHolder.get() == null ) {
+ try {
+ topDocsHolder.set(searcher.search(luceneQuery, batchSize));
+ } catch (final IOException ioe) {
+ throw new EventNotFoundException("Unable to obtain next record from " + LuceneIndexSearcher.this, ioe);
+ }
+ }
+
+ final boolean hasNext = fetched < topDocsHolder.get().totalHits;
+ if ( !hasNext ) {
+ try {
+ LuceneIndexSearcher.this.close();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to close {} due to {}", this, ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+ }
+ return hasNext;
+ }
+
+ @Override
+ public T next() {
+ if ( !hasNext() ) {
+ throw new NoSuchElementException();
+ }
+
+ TopDocs topDocs = topDocsHolder.get();
+ ScoreDoc[] scoreDocs = topDocs.scoreDocs;
+ if ( scoreDocIndex >= scoreDocs.length ) {
+ try {
+ topDocs = getTopDocs(scoreDocs[scoreDocs.length - 1], luceneQuery, batchSize);
+ topDocsHolder.set(topDocs);
+ scoreDocs = topDocs.scoreDocs;
+ scoreDocIndex = 0;
+ } catch (final IOException ioe) {
+ throw new EventNotFoundException("Unable to obtain next record from " + LuceneIndexSearcher.this, ioe);
+ }
+ }
+
+ final ScoreDoc scoreDoc = scoreDocs[scoreDocIndex++];
+ final Document document;
+ try {
+ document = getDocument(scoreDoc.doc, fieldsToLoad);
+ } catch (final IOException ioe) {
+ throw new EventNotFoundException("Unable to obtain next record from " + LuceneIndexSearcher.this, ioe);
+ }
+ fetched++;
+
+ return transformer.transform(document);
+ }
+
+ // this method exists solely for the use of a profiler so that I can see which methods are taking the longest when
+ // profiling only org.apache.nifi.*
+ private TopDocs getTopDocs(final ScoreDoc start, final org.apache.lucene.search.Query luceneQuery, final int batchSize) throws IOException {
+ return searcher.searchAfter(start, luceneQuery, batchSize);
+ }
+
+ // this method exists solely for the use of a profiler so that I can see which methods are taking the longest when
+ // profiling only org.apache.nifi.*
+ private Document getDocument(final int docId, final Set<String> referencedFields) throws IOException {
+ if ( referencedFields == null || referencedFields.isEmpty() ) {
+ return searcher.doc(docId);
+ } else {
+ return searcher.doc(docId, referencedFields);
+ }
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+
+ @Override
+ public Iterator<LazyInitializedProvenanceEvent> select(final String query, final Set<String> referencedFields) throws IOException {
+ return select(query, referencedFields, new DocumentTransformer<LazyInitializedProvenanceEvent>() {
+ @Override
+ public LazyInitializedProvenanceEvent transform(final Document document) {
+ return new LazyInitializedProvenanceEvent(repo, QueryUtils.createLocation(document), document);
+ }
+ });
+ }
+
+ @Override
+ public Iterator<JournaledStorageLocation> selectLocations(final String query) throws IOException {
+ return select(query, REQUIRED_FIELDS, new DocumentTransformer<JournaledStorageLocation>() {
+ @Override
+ public JournaledStorageLocation transform(final Document document) {
+ return QueryUtils.createLocation(document);
+ }
+ });
+ }
+
+ private static interface DocumentTransformer<T> {
+ T transform(Document document);
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
new file mode 100644
index 0000000..1f84891
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
@@ -0,0 +1,283 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.ConcurrentMergeScheduler;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.Version;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.search.SearchableField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LuceneIndexWriter implements EventIndexWriter {
+ private static final Store STORE_FIELDS = Store.YES;
+ private static final Logger logger = LoggerFactory.getLogger(LuceneIndexWriter.class);
+
+ private final Set<SearchableField> nonAttributeSearchableFields;
+ private final Set<SearchableField> attributeSearchableFields;
+ private final File indexDir;
+
+ private final ProvenanceEventRepository repo;
+ private final Directory directory;
+ private final Analyzer analyzer;
+ private final IndexWriter indexWriter;
+ private final AtomicLong indexMaxId = new AtomicLong(-1L);
+
+ public LuceneIndexWriter(final ProvenanceEventRepository repo, final File indexDir, final JournalingRepositoryConfig config) throws IOException {
+ this.repo = repo;
+ this.indexDir = indexDir;
+
+ attributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(config.getSearchableAttributes()));
+ nonAttributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(config.getSearchableFields()));
+
+ directory = FSDirectory.open(indexDir);
+ analyzer = new StandardAnalyzer();
+ final IndexWriterConfig writerConfig = new IndexWriterConfig(Version.LATEST, analyzer);
+ // Increase number of concurrent merges since we are on SSD:
+ final ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler();
+ writerConfig.setMergeScheduler(cms);
+ final int mergeThreads = Math.max(2, Math.min(4, config.getWorkerThreadPoolSize() / 2));
+ cms.setMaxMergesAndThreads(mergeThreads, mergeThreads);
+
+ indexWriter = new IndexWriter(directory, writerConfig);
+ }
+
+ public EventIndexSearcher newIndexSearcher() throws IOException {
+ logger.trace("Creating index searcher for {}", indexWriter);
+
+ final DirectoryReader reader = DirectoryReader.open(indexWriter, false);
+ return new LuceneIndexSearcher(repo, reader, indexDir);
+ }
+
+ @Override
+ public void close() throws IOException {
+ IOException suppressed = null;
+ try {
+ indexWriter.close();
+ } catch (final IOException ioe) {
+ suppressed = ioe;
+ }
+
+ analyzer.close();
+
+ try {
+ directory.close();
+ } catch (final IOException ioe) {
+ if ( suppressed != null ) {
+ ioe.addSuppressed(suppressed);
+ }
+
+ throw ioe;
+ }
+ }
+
+
+ private void addField(final Document doc, final SearchableField field, final String value, final Store store) {
+ if (value == null || (!nonAttributeSearchableFields.contains(field) && !field.isAttribute())) {
+ return;
+ }
+
+ doc.add(new StringField(field.getSearchableFieldName(), value.toLowerCase(), store));
+ }
+
+ @Override
+ public void index(final Collection<JournaledProvenanceEvent> events) throws IOException {
+ long maxId = this.indexMaxId.get();
+
+ final long startNanos = System.nanoTime();
+
+ final List<Document> documents = new ArrayList<>(events.size());
+ for ( final JournaledProvenanceEvent event : events ) {
+ maxId = event.getEventId();
+
+ final Map<String, String> attributes = event.getAttributes();
+
+ final Document doc = new Document();
+ addField(doc, SearchableFields.FlowFileUUID, event.getFlowFileUuid(), STORE_FIELDS);
+ addField(doc, SearchableFields.Filename, attributes.get(CoreAttributes.FILENAME.key()), STORE_FIELDS);
+ addField(doc, SearchableFields.ComponentID, event.getComponentId(), STORE_FIELDS);
+ addField(doc, SearchableFields.AlternateIdentifierURI, event.getAlternateIdentifierUri(), STORE_FIELDS);
+ addField(doc, SearchableFields.EventType, event.getEventType().name(), STORE_FIELDS);
+ addField(doc, SearchableFields.Relationship, event.getRelationship(), STORE_FIELDS);
+ addField(doc, SearchableFields.Details, event.getDetails(), STORE_FIELDS);
+ addField(doc, SearchableFields.ContentClaimSection, event.getContentClaimSection(), STORE_FIELDS);
+ addField(doc, SearchableFields.ContentClaimContainer, event.getContentClaimContainer(), STORE_FIELDS);
+ addField(doc, SearchableFields.ContentClaimIdentifier, event.getContentClaimIdentifier(), STORE_FIELDS);
+ addField(doc, SearchableFields.SourceQueueIdentifier, event.getSourceQueueIdentifier(), STORE_FIELDS);
+
+ if (nonAttributeSearchableFields.contains(SearchableFields.TransitURI)) {
+ addField(doc, SearchableFields.TransitURI, event.getTransitUri(), STORE_FIELDS);
+ }
+
+ for (final SearchableField searchableField : attributeSearchableFields) {
+ addField(doc, searchableField, attributes.get(searchableField.getSearchableFieldName()), STORE_FIELDS);
+ }
+
+ // Index the fields that we always index (unless there's nothing else to index at all)
+ doc.add(new LongField(SearchableFields.LineageStartDate.getSearchableFieldName(), event.getLineageStartDate(), STORE_FIELDS));
+ doc.add(new LongField(SearchableFields.EventTime.getSearchableFieldName(), event.getEventTime(), STORE_FIELDS));
+ doc.add(new LongField(SearchableFields.FileSize.getSearchableFieldName(), event.getFileSize(), STORE_FIELDS));
+
+ final JournaledStorageLocation location = event.getStorageLocation();
+ doc.add(new StringField(IndexedFieldNames.CONTAINER_NAME, location.getContainerName(), Store.YES));
+ doc.add(new StringField(IndexedFieldNames.SECTION_NAME, location.getSectionName(), Store.YES));
+ doc.add(new LongField(IndexedFieldNames.JOURNAL_ID, location.getJournalId(), Store.YES));
+ doc.add(new LongField(IndexedFieldNames.BLOCK_INDEX, location.getBlockIndex(), Store.YES));
+ doc.add(new LongField(IndexedFieldNames.EVENT_ID, location.getEventId(), Store.YES));
+
+ for (final String lineageIdentifier : event.getLineageIdentifiers()) {
+ addField(doc, SearchableFields.LineageIdentifier, lineageIdentifier, STORE_FIELDS);
+ }
+
+ // If it's event is a FORK, or JOIN, add the FlowFileUUID for all child/parent UUIDs.
+ if (event.getEventType() == ProvenanceEventType.FORK || event.getEventType() == ProvenanceEventType.CLONE || event.getEventType() == ProvenanceEventType.REPLAY) {
+ for (final String uuid : event.getChildUuids()) {
+ if (!uuid.equals(event.getFlowFileUuid())) {
+ addField(doc, SearchableFields.FlowFileUUID, uuid, STORE_FIELDS);
+ }
+ }
+ } else if (event.getEventType() == ProvenanceEventType.JOIN) {
+ for (final String uuid : event.getParentUuids()) {
+ if (!uuid.equals(event.getFlowFileUuid())) {
+ addField(doc, SearchableFields.FlowFileUUID, uuid, STORE_FIELDS);
+ }
+ }
+ } else if (event.getEventType() == ProvenanceEventType.RECEIVE && event.getSourceSystemFlowFileIdentifier() != null) {
+ // If we get a receive with a Source System FlowFile Identifier, we add another Document that shows the UUID
+ // that the Source System uses to refer to the data.
+ final String sourceIdentifier = event.getSourceSystemFlowFileIdentifier();
+ final String sourceFlowFileUUID;
+ final int lastColon = sourceIdentifier.lastIndexOf(":");
+ if (lastColon > -1 && lastColon < sourceIdentifier.length() - 2) {
+ sourceFlowFileUUID = sourceIdentifier.substring(lastColon + 1);
+ } else {
+ sourceFlowFileUUID = null;
+ }
+
+ if (sourceFlowFileUUID != null) {
+ addField(doc, SearchableFields.FlowFileUUID, sourceFlowFileUUID, STORE_FIELDS);
+ }
+ }
+
+ documents.add(doc);
+ }
+
+ indexWriter.addDocuments(documents);
+
+ // Update the index's max id
+ boolean updated = false;
+ do {
+ long curMax = indexMaxId.get();
+ if ( maxId > curMax ) {
+ updated = indexMaxId.compareAndSet(curMax, maxId);
+ } else {
+ updated = true;
+ }
+ } while (!updated);
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+ logger.debug("Indexed {} events in {} millis with {}", events.size(), millis, this);
+ }
+
+
+ @Override
+ public void delete(final String containerName, final String section, final Long journalId) throws IOException {
+ final BooleanQuery query = new BooleanQuery();
+ query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.CONTAINER_NAME, containerName)), Occur.MUST));
+ query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.SECTION_NAME, section)), Occur.MUST));
+ query.add(NumericRangeQuery.newLongRange(IndexedFieldNames.JOURNAL_ID, journalId, journalId, true, true), Occur.MUST);
+
+ final long start = System.nanoTime();
+ indexWriter.deleteDocuments(query);
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+ logger.info("Deleted events from {} that matched container={}, section={}, journal={} in {} millis", indexWriter, containerName, section, journalId, millis);
+ }
+
+ @Override
+ public void deleteEventsBefore(final String containerName, final String section, final Long journalId) throws IOException {
+ final BooleanQuery query = new BooleanQuery();
+ query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.CONTAINER_NAME, containerName)), Occur.MUST));
+ query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.SECTION_NAME, section)), Occur.MUST));
+ query.add(NumericRangeQuery.newLongRange(IndexedFieldNames.JOURNAL_ID, 0L, journalId, true, false), Occur.MUST);
+
+ final long start = System.nanoTime();
+ indexWriter.deleteDocuments(query);
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+ logger.info("Deleted events from {} that matched container={}, section={}, journal less than {} in {} millis", indexWriter, containerName, section, journalId, millis);
+ }
+
+
+ @Override
+ public void deleteOldEvents(final long earliestEventTimeToDelete) throws IOException {
+ final Query query = NumericRangeQuery.newLongRange(SearchableFields.EventTime.getSearchableFieldName(), 0L, earliestEventTimeToDelete, true, true);
+
+ final long start = System.nanoTime();
+ indexWriter.deleteDocuments(query);
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+ logger.info("Deleted events from {} that ocurred before {}; deletion took {} millis", this, new Date(earliestEventTimeToDelete), millis);
+ }
+
+ @Override
+ public void sync() throws IOException {
+ final long start = System.nanoTime();
+ indexWriter.commit();
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+ logger.info("Successfully sync'ed {} in {} millis", this, millis);
+ }
+
+ @Override
+ public String toString() {
+ return "LuceneIndexWriter[indexDir=" + indexDir + "]";
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
new file mode 100644
index 0000000..3d37379
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
@@ -0,0 +1,192 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.LazyInitializedProvenanceEvent;
+import org.apache.nifi.provenance.search.Query;
+
+import com.google.common.collect.Iterators;
+
+public class MultiIndexSearcher implements EventIndexSearcher {
+ private final List<EventIndexSearcher> searchers;
+
+ public MultiIndexSearcher(final List<EventIndexSearcher> searchers) {
+ this.searchers = searchers;
+ }
+
+ @Override
+ public void close() throws IOException {
+ IOException suppressed = null;
+
+ for ( final EventIndexSearcher searcher : searchers ) {
+ try {
+ searcher.close();
+ } catch (final IOException ioe) {
+ if ( suppressed == null ) {
+ suppressed = ioe;
+ } else {
+ suppressed.addSuppressed(ioe);
+ }
+ }
+ }
+
+ if ( suppressed != null ) {
+ throw suppressed;
+ }
+ }
+
+ @Override
+ public SearchResult search(final Query query) throws IOException {
+ int totalHitCount = 0;
+ final List<JournaledStorageLocation> locations = new ArrayList<>();
+
+ for ( final EventIndexSearcher searcher : searchers ) {
+ final SearchResult result = searcher.search(query);
+ totalHitCount += result.getTotalCount();
+ locations.addAll(result.getLocations());
+ }
+
+ Collections.sort(locations);
+ return new SearchResult(locations, totalHitCount);
+ }
+
+ @Override
+ public List<JournaledStorageLocation> getEvents(final long minEventId, final int maxResults) throws IOException {
+ final List<JournaledStorageLocation> locations = new ArrayList<>();
+ int results = 0;
+
+ // Perform search against all searchers and aggregate results.
+ for ( final EventIndexSearcher searcher : searchers ) {
+ final List<JournaledStorageLocation> searchLocations = searcher.getEvents(minEventId, maxResults);
+ locations.addAll(searchLocations);
+ if ( !searchLocations.isEmpty() ) {
+ results++;
+ }
+ }
+
+ // Results from this call are sorted. If we have only 0 or 1 searchers that had results, then
+ // we don't need to sort anything. Otherwise, we need to sort and return just the first X
+ // number of results.
+ if ( results > 1 ) {
+ Collections.sort(locations);
+ }
+
+ if ( locations.size() > maxResults ) {
+ return locations.subList(0, maxResults);
+ }
+
+ return locations;
+ }
+
+ @Override
+ public Long getMaxEventId(final String container, final String section) throws IOException {
+ Long max = null;
+ for ( final EventIndexSearcher searcher : searchers ) {
+ final Long maxForWriter = searcher.getMaxEventId(container, section);
+ if ( maxForWriter != null ) {
+ if (max == null || maxForWriter.longValue() > max.longValue() ) {
+ max = maxForWriter;
+ }
+ }
+ }
+
+ return max;
+ }
+
+ @Override
+ public List<JournaledStorageLocation> getEventsForFlowFiles(final Collection<String> flowFileUuids, final long earliestTime, final long latestTime) throws IOException {
+ final List<JournaledStorageLocation> locations = new ArrayList<>();
+ for ( final EventIndexSearcher searcher : searchers ) {
+ final List<JournaledStorageLocation> indexLocations = searcher.getEventsForFlowFiles(flowFileUuids, earliestTime, latestTime);
+ if ( indexLocations != null && !indexLocations.isEmpty() ) {
+ locations.addAll(indexLocations);
+ }
+ }
+
+ Collections.sort(locations);
+ return locations;
+ }
+
+ @Override
+ public List<JournaledStorageLocation> getLatestEvents(final int numEvents) throws IOException {
+ final List<JournaledStorageLocation> locations = new ArrayList<>();
+ for ( final EventIndexSearcher searcher : searchers ) {
+ final List<JournaledStorageLocation> indexLocations = searcher.getLatestEvents(numEvents);
+ if ( indexLocations != null && !indexLocations.isEmpty() ) {
+ locations.addAll(indexLocations);
+ }
+ }
+
+ Collections.sort(locations, new Comparator<JournaledStorageLocation>() {
+ @Override
+ public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+ return Long.compare(o1.getEventId(), o2.getEventId());
+ }
+ });
+ return locations;
+ }
+
+ @Override
+ public long getNumberOfEvents() throws IOException {
+ long totalCount = 0;
+
+ for ( final EventIndexSearcher searcher : searchers ) {
+ totalCount += searcher.getNumberOfEvents();
+ }
+
+ return totalCount;
+ }
+
+ @Override
+ public String toString() {
+ return searchers.toString();
+ }
+
+ @Override
+ public Iterator<LazyInitializedProvenanceEvent> select(final String query, final Set<String> referencedFields) throws IOException {
+ final List<Iterator<LazyInitializedProvenanceEvent>> iterators = new ArrayList<>(searchers.size());
+
+ for ( final EventIndexSearcher searcher : searchers ) {
+ final Iterator<LazyInitializedProvenanceEvent> itr = searcher.select(query, referencedFields);
+ iterators.add(itr);
+ }
+
+ return Iterators.concat(iterators.iterator());
+ }
+
+ @Override
+ public Iterator<JournaledStorageLocation> selectLocations(final String query) throws IOException {
+ final List<Iterator<JournaledStorageLocation>> iterators = new ArrayList<>(searchers.size());
+
+ for ( final EventIndexSearcher searcher : searchers ) {
+ final Iterator<JournaledStorageLocation> itr = searcher.selectLocations(query);
+ iterators.add(itr);
+ }
+
+ return Iterators.concat(iterators.iterator());
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
new file mode 100644
index 0000000..ada5317
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
@@ -0,0 +1,150 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.WildcardQuery;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.search.SearchTerm;
+
+public class QueryUtils {
+
+ public static JournaledStorageLocation createLocation(final Document document) {
+ final String containerName = document.get(IndexedFieldNames.CONTAINER_NAME);
+ if ( containerName == null ) {
+ return null;
+ }
+
+ final String sectionName = document.get(IndexedFieldNames.SECTION_NAME);
+ final long journalId = document.getField(IndexedFieldNames.JOURNAL_ID).numericValue().longValue();
+ final int blockIndex = document.getField(IndexedFieldNames.BLOCK_INDEX).numericValue().intValue();
+ final long eventId = document.getField(IndexedFieldNames.EVENT_ID).numericValue().longValue();
+
+ return new JournaledStorageLocation(containerName, sectionName, journalId, blockIndex, eventId);
+ }
+
+
+ public static org.apache.lucene.search.Query convertQueryToLucene(final org.apache.nifi.provenance.search.Query query) {
+ if (query.getStartDate() == null && query.getEndDate() == null && query.getSearchTerms().isEmpty()) {
+ return new MatchAllDocsQuery();
+ }
+
+ final BooleanQuery luceneQuery = new BooleanQuery();
+ for (final SearchTerm searchTerm : query.getSearchTerms()) {
+ final String searchValue = searchTerm.getValue();
+ if (searchValue == null) {
+ throw new IllegalArgumentException("Empty search value not allowed (for term '" + searchTerm.getSearchableField().getFriendlyName() + "')");
+ }
+
+ if (searchValue.contains("*") || searchValue.contains("?")) {
+ luceneQuery.add(new BooleanClause(new WildcardQuery(new Term(searchTerm.getSearchableField().getSearchableFieldName(), searchTerm.getValue().toLowerCase())), Occur.MUST));
+ } else {
+ luceneQuery.add(new BooleanClause(new TermQuery(new Term(searchTerm.getSearchableField().getSearchableFieldName(), searchTerm.getValue().toLowerCase())), Occur.MUST));
+ }
+ }
+
+ final Long minBytes = query.getMinFileSize() == null ? null : DataUnit.parseDataSize(query.getMinFileSize(), DataUnit.B).longValue();
+ final Long maxBytes = query.getMaxFileSize() == null ? null : DataUnit.parseDataSize(query.getMaxFileSize(), DataUnit.B).longValue();
+ if (minBytes != null || maxBytes != null) {
+ luceneQuery.add(NumericRangeQuery.newLongRange(SearchableFields.FileSize.getSearchableFieldName(), minBytes, maxBytes, true, true), Occur.MUST);
+ }
+
+ final Long minDateTime = query.getStartDate() == null ? null : query.getStartDate().getTime();
+ final Long maxDateTime = query.getEndDate() == null ? null : query.getEndDate().getTime();
+ if (maxDateTime != null || minDateTime != null) {
+ luceneQuery.add(NumericRangeQuery.newLongRange(SearchableFields.EventTime.getSearchableFieldName(), minDateTime, maxDateTime, true, true), Occur.MUST);
+ }
+
+ return luceneQuery;
+ }
+
+
+ private static File getJournalFile(final JournaledStorageLocation location, final JournalingRepositoryConfig config) throws FileNotFoundException {
+ final File containerDir = config.getContainers().get(location.getContainerName());
+ if ( containerDir == null ) {
+ throw new FileNotFoundException("Could not find Container with name " + location.getContainerName());
+ }
+
+ final String sectionName = location.getSectionName();
+ final File sectionFile = new File(containerDir, sectionName);
+ final File journalDir = new File(sectionFile, "journals");
+ final File journalFile = new File(journalDir, location.getJournalId() + ".journal");
+
+ return journalFile;
+ }
+
+
+ /**
+ * Orders the given StorageLocations so that we have a Map where the Key is a Journal file and the value is a List of JournaledStorageLocation in the order
+ * in which they should be read from the journal for optimal performance
+ * @param locations
+ * @param config
+ * @return
+ * @throws FileNotFoundException
+ * @throws IOException
+ */
+ public static Map<File, List<JournaledStorageLocation>> orderLocations(final List<StorageLocation> locations, final JournalingRepositoryConfig config) throws FileNotFoundException, IOException {
+ final Map<File, List<JournaledStorageLocation>> map = new HashMap<>();
+
+ for ( final StorageLocation location : locations ) {
+ if ( !(location instanceof JournaledStorageLocation) ) {
+ throw new IllegalArgumentException(location + " is not a valid StorageLocation for this repository");
+ }
+
+ final JournaledStorageLocation journaledLocation = (JournaledStorageLocation) location;
+ final File journalFile = getJournalFile(journaledLocation, config);
+ List<JournaledStorageLocation> list = map.get(journalFile);
+ if ( list == null ) {
+ list = new ArrayList<>();
+ map.put(journalFile, list);
+ }
+
+ list.add(journaledLocation);
+ }
+
+ for ( final List<JournaledStorageLocation> list : map.values() ) {
+ Collections.sort(list);
+ }
+
+ return map;
+ }
+
+ public static File getTocFile(final File journalFile) {
+ return new File(journalFile.getParentFile(), journalFile.getName() + ".toc");
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/SearchResult.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/SearchResult.java
new file mode 100644
index 0000000..ac82438
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/SearchResult.java
@@ -0,0 +1,40 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.util.List;
+
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+
+public class SearchResult {
+ private final int totalCount;
+ private final List<JournaledStorageLocation> locations;
+
+ public SearchResult(final List<JournaledStorageLocation> locations, final int totalCount) {
+ this.totalCount = totalCount;
+ this.locations = locations;
+ }
+
+ public int getTotalCount() {
+ return totalCount;
+ }
+
+ public List<JournaledStorageLocation> getLocations() {
+ return locations;
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/VoidIndexAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/VoidIndexAction.java
new file mode 100644
index 0000000..a1ae22f
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/VoidIndexAction.java
@@ -0,0 +1,23 @@
+/*
+ * 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.provenance.journaling.index;
+
+import java.io.IOException;
+
+public interface VoidIndexAction {
+ void perform(EventIndexSearcher searcher) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializer.java
new file mode 100644
index 0000000..67b1cb6
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializer.java
@@ -0,0 +1,30 @@
+/*
+ * 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.provenance.journaling.io;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface Deserializer {
+
+ String getCodecName();
+
+ ProvenanceEventRecord deserialize(DataInputStream in, int serializationVersion) throws IOException;
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializers.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializers.java
new file mode 100644
index 0000000..4be87e3
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializers.java
@@ -0,0 +1,30 @@
+/*
+ * 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.provenance.journaling.io;
+
+public class Deserializers {
+
+ public static Deserializer getDeserializer(final String codecName) {
+ switch (codecName) {
+ case StandardEventDeserializer.CODEC_NAME:
+ return new StandardEventDeserializer();
+ default:
+ throw new IllegalArgumentException("Unknown Provenance Serialization Codec: " + codecName);
+ }
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Serializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Serializer.java
new file mode 100644
index 0000000..8219b4c
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Serializer.java
@@ -0,0 +1,49 @@
+/*
+ * 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.provenance.journaling.io;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface Serializer {
+
+ /**
+ * Returns the serialization version that is used to serialize records
+ * @return
+ */
+ int getVersion();
+
+ /**
+ * Returns the name of the codec used to serialize the records
+ * @return
+ */
+ String getCodecName();
+
+ /**
+ * Serializes the given even to the given DataOutputStream.
+ * This method should NOT serialize the ID, as the ID is not yet known. The ID will instead by
+ * serialized to the stream appropriately by the JournalWriter.
+ *
+ * @param event
+ * @param out
+ * @throws IOException
+ */
+ void serialize(ProvenanceEventRecord event, DataOutputStream out) throws IOException;
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventDeserializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventDeserializer.java
new file mode 100644
index 0000000..fb537ee
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventDeserializer.java
@@ -0,0 +1,162 @@
+/*
+ * 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.provenance.journaling.io;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.IdEnrichedProvenanceEvent;
+import org.apache.nifi.stream.io.StreamUtils;
+
+public class StandardEventDeserializer implements Deserializer {
+ public static final String CODEC_NAME = StandardEventSerializer.CODEC_NAME;
+
+ @Override
+ public String getCodecName() {
+ return CODEC_NAME;
+ }
+
+ @Override
+ public ProvenanceEventRecord deserialize(final DataInputStream in, final int serializationVersion) throws IOException {
+ final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder();
+
+ final long eventId = in.readLong();
+ final String eventTypeName = in.readUTF();
+ final ProvenanceEventType eventType = ProvenanceEventType.valueOf(eventTypeName);
+ builder.setEventType(eventType);
+ builder.setEventTime(in.readLong());
+
+ final Long flowFileEntryDate = in.readLong();
+ builder.setEventDuration(in.readLong());
+
+ final Set<String> lineageIdentifiers = new HashSet<>();
+ final int numLineageIdentifiers = in.readInt();
+ for (int i = 0; i < numLineageIdentifiers; i++) {
+ lineageIdentifiers.add(readUUID(in));
+ }
+
+ final long lineageStartDate = in.readLong();
+
+ builder.setComponentId(readNullableString(in));
+ builder.setComponentType(readNullableString(in));
+
+ final String uuid = readUUID(in);
+ builder.setFlowFileUUID(uuid);
+ builder.setDetails(readNullableString(in));
+
+ // Read in the FlowFile Attributes
+ final Map<String, String> previousAttrs = readAttributes(in, false);
+ final Map<String, String> attrUpdates = readAttributes(in, true);
+ builder.setAttributes(previousAttrs, attrUpdates);
+
+ final boolean hasContentClaim = in.readBoolean();
+ if (hasContentClaim) {
+ builder.setCurrentContentClaim(in.readUTF(), in.readUTF(), in.readUTF(), in.readLong(), in.readLong());
+ } else {
+ builder.setCurrentContentClaim(null, null, null, null, 0L);
+ }
+
+ final boolean hasPreviousClaim = in.readBoolean();
+ if (hasPreviousClaim) {
+ builder.setPreviousContentClaim(in.readUTF(), in.readUTF(), in.readUTF(), in.readLong(), in.readLong());
+ }
+
+ builder.setSourceQueueIdentifier(readNullableString(in));
+
+ // Read Event-Type specific fields.
+ if (eventType == ProvenanceEventType.FORK || eventType == ProvenanceEventType.JOIN || eventType == ProvenanceEventType.CLONE || eventType == ProvenanceEventType.REPLAY) {
+ final int numParents = in.readInt();
+ for (int i = 0; i < numParents; i++) {
+ builder.addParentUuid(readUUID(in));
+ }
+
+ final int numChildren = in.readInt();
+ for (int i = 0; i < numChildren; i++) {
+ builder.addChildUuid(readUUID(in));
+ }
+ } else if (eventType == ProvenanceEventType.RECEIVE) {
+ builder.setTransitUri(readNullableString(in));
+ builder.setSourceSystemFlowFileIdentifier(readNullableString(in));
+ } else if (eventType == ProvenanceEventType.SEND) {
+ builder.setTransitUri(readNullableString(in));
+ } else if (eventType == ProvenanceEventType.ADDINFO) {
+ builder.setAlternateIdentifierUri(readNullableString(in));
+ } else if (eventType == ProvenanceEventType.ROUTE) {
+ builder.setRelationship(readNullableString(in));
+ }
+
+ builder.setFlowFileEntryDate(flowFileEntryDate);
+ builder.setLineageIdentifiers(lineageIdentifiers);
+ builder.setLineageStartDate(lineageStartDate);
+ final ProvenanceEventRecord event = builder.build();
+
+ return new IdEnrichedProvenanceEvent(event, eventId);
+ }
+
+
+ private static Map<String, String> readAttributes(final DataInputStream dis, final boolean valueNullable) throws IOException {
+ final int numAttributes = dis.readInt();
+ final Map<String, String> attrs = new HashMap<>();
+ for (int i = 0; i < numAttributes; i++) {
+ final String key = readLongString(dis);
+ final String value = valueNullable ? readLongNullableString(dis) : readLongString(dis);
+ attrs.put(key, value);
+ }
+
+ return attrs;
+ }
+
+ private static String readUUID(final DataInputStream in) throws IOException {
+ final long msb = in.readLong();
+ final long lsb = in.readLong();
+ return new UUID(msb, lsb).toString();
+ }
+
+ private static String readNullableString(final DataInputStream in) throws IOException {
+ final boolean valueExists = in.readBoolean();
+ if (valueExists) {
+ return in.readUTF();
+ } else {
+ return null;
+ }
+ }
+
+ private static String readLongNullableString(final DataInputStream in) throws IOException {
+ final boolean valueExists = in.readBoolean();
+ if (valueExists) {
+ return readLongString(in);
+ } else {
+ return null;
+ }
+ }
+
+ private static String readLongString(final DataInputStream in) throws IOException {
+ final int length = in.readInt();
+ final byte[] strBytes = new byte[length];
+ StreamUtils.fillBuffer(in, strBytes);
+ return new String(strBytes, StandardCharsets.UTF_8);
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventSerializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventSerializer.java
new file mode 100644
index 0000000..fae427e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventSerializer.java
@@ -0,0 +1,170 @@
+/*
+ * 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.provenance.journaling.io;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+public class StandardEventSerializer implements Serializer {
+ public static final String CODEC_NAME = "StandardProvCodec";
+
+ @Override
+ public int getVersion() {
+ return 1;
+ }
+
+ @Override
+ public String getCodecName() {
+ return CODEC_NAME;
+ }
+
+ @Override
+ public void serialize(final ProvenanceEventRecord event, final DataOutputStream out) throws IOException {
+ final ProvenanceEventType recordType = event.getEventType();
+
+ out.writeUTF(event.getEventType().name());
+ out.writeLong(event.getEventTime());
+ out.writeLong(event.getFlowFileEntryDate());
+ out.writeLong(event.getEventDuration());
+
+ writeUUIDs(out, event.getLineageIdentifiers());
+ out.writeLong(event.getLineageStartDate());
+
+ writeNullableString(out, event.getComponentId());
+ writeNullableString(out, event.getComponentType());
+ writeUUID(out, event.getFlowFileUuid());
+ writeNullableString(out, event.getDetails());
+
+ // Write FlowFile attributes
+ final Map<String, String> attrs = event.getPreviousAttributes();
+ out.writeInt(attrs.size());
+ for (final Map.Entry<String, String> entry : attrs.entrySet()) {
+ writeLongString(out, entry.getKey());
+ writeLongString(out, entry.getValue());
+ }
+
+ final Map<String, String> attrUpdates = event.getUpdatedAttributes();
+ out.writeInt(attrUpdates.size());
+ for (final Map.Entry<String, String> entry : attrUpdates.entrySet()) {
+ writeLongString(out, entry.getKey());
+ writeLongNullableString(out, entry.getValue());
+ }
+
+ // If Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'.
+ if (event.getContentClaimSection() != null && event.getContentClaimContainer() != null && event.getContentClaimIdentifier() != null) {
+ out.writeBoolean(true);
+ out.writeUTF(event.getContentClaimContainer());
+ out.writeUTF(event.getContentClaimSection());
+ out.writeUTF(event.getContentClaimIdentifier());
+ if (event.getContentClaimOffset() == null) {
+ out.writeLong(0L);
+ } else {
+ out.writeLong(event.getContentClaimOffset());
+ }
+ out.writeLong(event.getFileSize());
+ } else {
+ out.writeBoolean(false);
+ }
+
+ // If Previous Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'.
+ if (event.getPreviousContentClaimSection() != null && event.getPreviousContentClaimContainer() != null && event.getPreviousContentClaimIdentifier() != null) {
+ out.writeBoolean(true);
+ out.writeUTF(event.getPreviousContentClaimContainer());
+ out.writeUTF(event.getPreviousContentClaimSection());
+ out.writeUTF(event.getPreviousContentClaimIdentifier());
+ if (event.getPreviousContentClaimOffset() == null) {
+ out.writeLong(0L);
+ } else {
+ out.writeLong(event.getPreviousContentClaimOffset());
+ }
+
+ if (event.getPreviousFileSize() == null) {
+ out.writeLong(0L);
+ } else {
+ out.writeLong(event.getPreviousFileSize());
+ }
+ } else {
+ out.writeBoolean(false);
+ }
+
+ // write out the identifier of the destination queue.
+ writeNullableString(out, event.getSourceQueueIdentifier());
+
+ // Write type-specific info
+ if (recordType == ProvenanceEventType.FORK || recordType == ProvenanceEventType.JOIN || recordType == ProvenanceEventType.CLONE || recordType == ProvenanceEventType.REPLAY) {
+ writeUUIDs(out, event.getParentUuids());
+ writeUUIDs(out, event.getChildUuids());
+ } else if (recordType == ProvenanceEventType.RECEIVE) {
+ writeNullableString(out, event.getTransitUri());
+ writeNullableString(out, event.getSourceSystemFlowFileIdentifier());
+ } else if (recordType == ProvenanceEventType.SEND) {
+ writeNullableString(out, event.getTransitUri());
+ } else if (recordType == ProvenanceEventType.ADDINFO) {
+ writeNullableString(out, event.getAlternateIdentifierUri());
+ } else if (recordType == ProvenanceEventType.ROUTE) {
+ writeNullableString(out, event.getRelationship());
+ }
+ }
+
+ private void writeNullableString(final DataOutputStream out, final String toWrite) throws IOException {
+ if (toWrite == null) {
+ out.writeBoolean(false);
+ } else {
+ out.writeBoolean(true);
+ out.writeUTF(toWrite);
+ }
+ }
+
+ private void writeLongNullableString(final DataOutputStream out, final String toWrite) throws IOException {
+ if (toWrite == null) {
+ out.writeBoolean(false);
+ } else {
+ out.writeBoolean(true);
+ writeLongString(out, toWrite);
+ }
+ }
+
+ private void writeLongString(final DataOutputStream out, final String value) throws IOException {
+ final byte[] bytes = value.getBytes("UTF-8");
+ out.writeInt(bytes.length);
+ out.write(bytes);
+ }
+
+ static void writeUUID(final DataOutputStream out, final String uuid) throws IOException {
+ final UUID uuidObj = UUID.fromString(uuid);
+ out.writeLong(uuidObj.getMostSignificantBits());
+ out.writeLong(uuidObj.getLeastSignificantBits());
+ }
+
+ static void writeUUIDs(final DataOutputStream out, final Collection<String> list) throws IOException {
+ if (list == null) {
+ out.writeInt(0);
+ } else {
+ out.writeInt(list.size());
+ for (final String value : list) {
+ writeUUID(out, value);
+ }
+ }
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/CompressedOutputStream.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/CompressedOutputStream.java
new file mode 100644
index 0000000..618dd88
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/CompressedOutputStream.java
@@ -0,0 +1,36 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+public abstract class CompressedOutputStream extends OutputStream {
+
+ /**
+ * Begins a new compression block
+ * @throws IOException
+ */
+ public abstract void beginNewBlock() throws IOException;
+
+ /**
+ * Ends the current compression block
+ * @throws IOException
+ */
+ public abstract void finishBlock() throws IOException;
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/CompressionCodec.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/CompressionCodec.java
new file mode 100644
index 0000000..f6e856e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/CompressionCodec.java
@@ -0,0 +1,45 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public interface CompressionCodec {
+ /**
+ * Returns the name of the compression codec
+ * @return
+ */
+ String getName();
+
+ /**
+ * Wraps the given OutputStream so that data written will be compressed
+ * @param out
+ * @return
+ * @throws IOException
+ */
+ CompressedOutputStream newCompressionOutputStream(OutputStream out) throws IOException;
+
+ /**
+ * Wraps the given InputStream so that data read will be decompressed
+ * @param in
+ * @return
+ * @throws IOException
+ */
+ InputStream newCompressionInputStream(InputStream in) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/DeflatorCompressionCodec.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/DeflatorCompressionCodec.java
new file mode 100644
index 0000000..b9f2959
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/DeflatorCompressionCodec.java
@@ -0,0 +1,113 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.nifi.remote.io.CompressionInputStream;
+import org.apache.nifi.remote.io.CompressionOutputStream;
+
+public class DeflatorCompressionCodec implements CompressionCodec {
+ public static final String DEFLATOR_COMPRESSION_CODEC = "deflator-compression-codec";
+
+ @Override
+ public String getName() {
+ return DEFLATOR_COMPRESSION_CODEC;
+ }
+
+ @Override
+ public CompressedOutputStream newCompressionOutputStream(final OutputStream out) throws IOException {
+ return new DeflatorOutputStream(out);
+ }
+
+ @Override
+ public InputStream newCompressionInputStream(final InputStream in) throws IOException {
+ return new CompressionInputStream(in);
+ }
+
+
+ private static class DeflatorOutputStream extends CompressedOutputStream {
+ private final OutputStream originalOut;
+ private CompressionOutputStream compressionOutput;
+
+ public DeflatorOutputStream(final OutputStream out) {
+ this.originalOut = out;
+ }
+
+ private void verifyState() {
+ if ( compressionOutput == null ) {
+ throw new IllegalStateException("No Compression Block has been created");
+ }
+ }
+
+ @Override
+ public void write(final int b) throws IOException {
+ verifyState();
+ compressionOutput.write(b);
+ }
+
+ @Override
+ public void write(final byte[] b) throws IOException {
+ verifyState();
+ compressionOutput.write(b);
+ }
+
+ @Override
+ public void write(final byte[] b, final int off, final int len) throws IOException {
+ verifyState();
+ compressionOutput.write(b, off, len);
+ }
+
+ @Override
+ public void flush() throws IOException {
+ if ( compressionOutput != null ) {
+ compressionOutput.flush();
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ if ( compressionOutput != null ) {
+ compressionOutput.close();
+ }
+
+ originalOut.close();
+ }
+
+ @Override
+ public void beginNewBlock() throws IOException {
+ compressionOutput = new CompressionOutputStream(originalOut);
+ }
+
+ @Override
+ public void finishBlock() throws IOException {
+ // Calling close() on CompressionOutputStream doesn't close the underlying stream -- it is designed
+ // such that calling close() will write out the Compression footer and become unusable but not
+ // close the underlying stream because the whole point of CompressionOutputStream as opposed to
+ // GZIPOutputStream is that with CompressionOutputStream we can concatenate many together on a single
+ // stream.
+ if ( compressionOutput == null ) {
+ return;
+ } else {
+ compressionOutput.close();
+ }
+ }
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
new file mode 100644
index 0000000..3345f50
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
@@ -0,0 +1,55 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface JournalReader extends Closeable {
+
+ /**
+ * Retrieve a specific event from the journal, given the offset of the Block and the ID of the event
+ * @param blockOffset
+ * @param eventId
+ * @return
+ * @throws IOException
+ */
+ ProvenanceEventRecord getEvent(long blockOffset, long eventId) throws IOException;
+
+ /**
+ * Retrieve the next event in the journal, or <code>null</code> if no more events exist
+ * @return
+ * @throws IOException
+ */
+ ProvenanceEventRecord nextEvent() throws IOException;
+
+ /**
+ * Returns the current byte offset into the Journal from which the next event (if any) will be read
+ * @return
+ */
+ long getPosition();
+
+ /**
+ * Retrieves the last event from the journal, given the offset of the last Block in the journal
+ * @param blockOffset
+ * @return
+ * @throws IOException
+ */
+ ProvenanceEventRecord getLastEvent(long blockOffset) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalWriter.java
new file mode 100644
index 0000000..5108f49
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalWriter.java
@@ -0,0 +1,96 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+/**
+ * Responsible for writing events to an append-only journal, or write-ahead-log. Events are written in "Blocks."
+ * These Blocks are used so that if we are compressing data, we can compress individual Blocks. This allows us
+ * to store a "Block Index" so that we can quickly lookup the start of a Block when reading the data to quickly
+ * obtain the data that we need.
+ */
+public interface JournalWriter extends Closeable {
+
+ /**
+ * Returns the identifier of this journal. The identifier is unique per 'section' of the repository
+ * @return
+ */
+ long getJournalId();
+
+ /**
+ * Writes the given events to the journal and assigns the events sequential ID's starting with the
+ * ID given
+ *
+ * @param records
+ * @param firstRecordId
+ * @return
+ * @throws IOException
+ */
+ void write(Collection<ProvenanceEventRecord> events, long firstEventId) throws IOException;
+
+ /**
+ * Returns the File that the Journal is writing to
+ */
+ File getJournalFile();
+
+ /**
+ * Synchronizes changes to the underlying file system
+ * @throws IOException
+ */
+ void sync() throws IOException;
+
+ /**
+ * Returns the size of the journal
+ * @return
+ */
+ long getSize();
+
+ /**
+ * Returns the number of events that have been written to this journal
+ * @return
+ */
+ int getEventCount();
+
+ /**
+ * Returns the amount of time that has elapsed since the point at which the writer was created.
+ * @param timeUnit
+ * @return
+ */
+ long getAge(TimeUnit timeUnit);
+
+ /**
+ * Marks the end of a Block in the output file. If the previous Block has been finished and no new
+ * Block has been started, this method will return silently without doing anything.
+ * @throws IOException
+ */
+ void finishBlock() throws IOException;
+
+ /**
+ * Starts a new Block in the output file. If a Block has already been started, this method throws
+ * an IllegalStateException
+ *
+ * @throws IOException
+ */
+ void beginNewBlock() throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalMagicHeader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalMagicHeader.java
new file mode 100644
index 0000000..5803100
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalMagicHeader.java
@@ -0,0 +1,50 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.apache.nifi.stream.io.StreamUtils;
+
+public class StandardJournalMagicHeader {
+ private static final byte[] MAGIC_HEADER = "NiFiProvJournal_1".getBytes();
+
+ /**
+ * Writes the magic header to the output stream
+ * @param out
+ * @throws IOException
+ */
+ static void write(final OutputStream out) throws IOException {
+ out.write(MAGIC_HEADER);
+ }
+
+ /**
+ * Verifies that the magic header is present on the InputStream
+ * @param in
+ * @throws IOException
+ */
+ static void read(final InputStream in) throws IOException {
+ final byte[] magicHeaderBuffer = new byte[MAGIC_HEADER.length];
+ StreamUtils.fillBuffer(in, magicHeaderBuffer);
+ if ( !Arrays.equals(MAGIC_HEADER, magicHeaderBuffer) ) {
+ throw new IOException("File is not a Journaling Provenance Repository journal file");
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
new file mode 100644
index 0000000..9a937b4
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
@@ -0,0 +1,229 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.Deserializer;
+import org.apache.nifi.provenance.journaling.io.Deserializers;
+import org.apache.nifi.stream.io.ByteCountingInputStream;
+import org.apache.nifi.stream.io.LimitingInputStream;
+import org.apache.nifi.stream.io.MinimumLengthInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Standard implementation of {@link JournalReader}. This reader reads data that is written
+ * in the format specified by {@link StandardJournalWriter}
+ */
+public class StandardJournalReader implements JournalReader {
+ private static final Logger logger = LoggerFactory.getLogger(StandardJournalReader.class);
+
+ private final File file;
+
+ private ByteCountingInputStream compressedStream;
+ private ByteCountingInputStream decompressedStream;
+
+ private Deserializer deserializer;
+ private int serializationVersion;
+ private CompressionCodec compressionCodec = null;
+
+ private long lastEventIdRead = -1L;
+
+
+ public StandardJournalReader(final File file) throws IOException {
+ this.file = file;
+ resetStreams();
+ }
+
+ private void resetStreams() throws IOException {
+ final InputStream bufferedIn = new BufferedInputStream(new FileInputStream(file));
+ compressedStream = new ByteCountingInputStream(bufferedIn);
+ try {
+ final DataInputStream dis = new DataInputStream(compressedStream);
+
+ StandardJournalMagicHeader.read(dis);
+ final String codecName = dis.readUTF();
+ serializationVersion = dis.readInt();
+ final boolean compressed = dis.readBoolean();
+ if ( compressed ) {
+ final String compressionCodecName = dis.readUTF();
+ if ( DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC.equals(compressionCodecName) ) {
+ compressionCodec = new DeflatorCompressionCodec();
+ } else {
+ throw new IOException(file + " is compressed using unknown Compression Codec " + compressionCodecName);
+ }
+ }
+ deserializer = Deserializers.getDeserializer(codecName);
+
+ resetDecompressedStream();
+ } catch (final Exception e) {
+ try {
+ compressedStream.close();
+ } catch (final IOException ignore) {}
+
+ throw new IOException("Failed to reset data stream when reading" + file, e);
+ }
+ }
+
+
+ private void resetDecompressedStream() throws IOException {
+ if ( compressionCodec == null ) {
+ decompressedStream = compressedStream;
+ } else {
+ decompressedStream = new ByteCountingInputStream(new BufferedInputStream(compressionCodec.newCompressionInputStream(compressedStream)), compressedStream.getBytesConsumed());
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ compressedStream.close();
+
+ if ( compressedStream != decompressedStream ) {
+ decompressedStream.close();
+ }
+ }
+
+ @Override
+ public ProvenanceEventRecord nextEvent() throws IOException {
+ return nextEvent(true);
+ }
+
+ @Override
+ public long getPosition() {
+ return decompressedStream.getBytesConsumed();
+ }
+
+ private boolean isData(final InputStream in) throws IOException {
+ in.mark(1);
+ final int b = in.read();
+ if ( b < 0 ) {
+ return false;
+ }
+ in.reset();
+
+ return true;
+ }
+
+ ProvenanceEventRecord nextEvent(final boolean spanBlocks) throws IOException {
+ boolean isData = isData(decompressedStream);
+ if ( !isData ) {
+ if ( !spanBlocks ) {
+ return null;
+ }
+
+ // we are allowed to span blocks. We're out of data but if we are compressed, it could
+ // just mean that the block has ended.
+ if ( compressionCodec == null ) {
+ return null;
+ }
+
+ isData = isData(compressedStream);
+ if ( !isData ) {
+ return null;
+ }
+
+ // There is no data in the compressed InputStream but there is in the underlying stream.
+ // This means we've hit the end of our block. We will create a new CompressionInputStream
+ // so that we can continue reading.
+ resetDecompressedStream();
+ }
+
+ try {
+ final DataInputStream dis = new DataInputStream(decompressedStream);
+ final int eventLength = dis.readInt();
+
+ final LimitingInputStream limitingInputStream = new LimitingInputStream(dis, eventLength);
+ final MinimumLengthInputStream minStream = new MinimumLengthInputStream(limitingInputStream, eventLength);
+ final ProvenanceEventRecord event = deserializer.deserialize(new DataInputStream(minStream), serializationVersion);
+ lastEventIdRead = event.getEventId();
+ return event;
+ } catch (final EOFException eof) {
+ logger.warn("{} Found unexpected End-of-File when reading from journal", this);
+ return null;
+ }
+ }
+
+ @Override
+ public ProvenanceEventRecord getEvent(final long blockOffset, final long eventId) throws IOException {
+ // If the requested event ID is less than the last event that we read, we need to reset to the beginning
+ // of the file. We do this because we know that the ID's are always increasing, so if we need an ID less
+ // than the previous ID, we have to go backward in the file. We can't do this with streams, so start the
+ // stream over.
+ if ( eventId <= lastEventIdRead ) {
+ close();
+ resetStreams();
+ }
+
+ final long bytesToSkip = blockOffset - compressedStream.getBytesConsumed();
+ if ( bytesToSkip > 0 ) {
+ StreamUtils.skip(compressedStream, bytesToSkip);
+ resetDecompressedStream();
+ }
+
+ ProvenanceEventRecord event;
+ while ((event = nextEvent()) != null) {
+ if ( event.getEventId() == eventId ) {
+ return event;
+ }
+ }
+
+ throw new IOException("Could not find event with ID " + eventId + " in " + this);
+ }
+
+ @Override
+ public String toString() {
+ return "StandardJournalReader[" + file + "]";
+ }
+
+ @Override
+ public ProvenanceEventRecord getLastEvent(final long blockOffset) throws IOException {
+ if ( blockOffset > compressedStream.getBytesConsumed() ) {
+ close();
+ resetStreams();
+ }
+
+ final long bytesToSkip = blockOffset - compressedStream.getBytesConsumed();
+ if ( bytesToSkip > 0 ) {
+ StreamUtils.skip(compressedStream, bytesToSkip);
+ resetDecompressedStream();
+ }
+
+ ProvenanceEventRecord lastReadRecord = null;
+ ProvenanceEventRecord event;
+ while ((event = nextEvent()) != null) {
+ lastReadRecord = event;
+ }
+
+ // If we weren't able to read anything and the block offset was given, just start over
+ // and read the entire thing, returning the last event.
+ if ( lastReadRecord == null && blockOffset > 0L ) {
+ return getLastEvent(0L);
+ }
+
+ // return the last even that we read, whether or not it was null
+ return lastReadRecord;
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
new file mode 100644
index 0000000..d18b05b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
@@ -0,0 +1,302 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.FileAlreadyExistsException;
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.Serializer;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.ByteArrayOutputStream;
+import org.apache.nifi.stream.io.ByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * <p>
+ * Standard implementation of {@link JournalWriter}.
+ * </p>
+ *
+ * <p>
+ * Writes out to a journal file using the format:
+ *
+ * <pre>
+ * <header>
+ * <begin block 1>
+ * <record 1>
+ * <record 2>
+ * <record 3>
+ * <end block 1>
+ * <begin block 2>
+ * <record 4>
+ * <record 5>
+ * <end block 2>
+ * ...
+ * <begin block N>
+ * <record N>
+ * <end block N>
+ * </pre>
+ *
+ * Where <header> is defined as:
+ * <pre>
+ * magic header "NiFiProvJournal_1"
+ * String: serialization codec name (retrieved from serializer)
+ * --> 2 bytes for length of string
+ * --> N bytes for actual serialization codec name
+ * int: serialization version
+ * boolean: compressed: 1 -> compressed, 0 -> not compressed
+ * String : if compressed, name of compression codec; otherwise, not present
+ * </pre>
+ *
+ * And <record> is defined as:
+ * <pre>
+ * bytes 0-3: int: record length
+ * bytes 4-11: long: record id
+ * bytes 12-N: serialized event according to the applied {@link Serializer}
+ * </pre>
+ * </p>
+ *
+ * <p>
+ * The structure of the <begin block> and <end block> element depend on whether or not
+ * compression is enabled. If the journal is not compressed, these elements are 0 bytes.
+ * If the journal is compressed, these are the compression header and compression footer, respectively.
+ * </p>
+ *
+ */
+public class StandardJournalWriter implements JournalWriter {
+ private static final Logger logger = LoggerFactory.getLogger(StandardJournalWriter.class);
+
+ private final long journalId;
+ private final File journalFile;
+ private final CompressionCodec compressionCodec;
+ private final Serializer serializer;
+ private final long creationTime = System.nanoTime();
+ private final String description;
+
+ private int eventCount;
+ private boolean blockStarted = false;
+
+ private final FileOutputStream fos;
+ private ByteCountingOutputStream uncompressedStream;
+ private OutputStream compressedStream;
+ private ByteCountingOutputStream out;
+
+ private long recordBytes = 256L;
+ private long recordCount = 1L;
+
+
+ public StandardJournalWriter(final long journalId, final File journalFile, final CompressionCodec compressionCodec, final Serializer serializer) throws IOException {
+ if ( journalFile.exists() ) {
+ // Check if there is actually any data here.
+ try (final InputStream fis = new FileInputStream(journalFile);
+ final InputStream bufferedIn = new BufferedInputStream(fis);
+ final DataInputStream dis = new DataInputStream(bufferedIn) ) {
+
+ StandardJournalMagicHeader.read(dis);
+ dis.readUTF();
+ dis.readInt();
+ dis.readBoolean();
+ final int nextByte = dis.read();
+ if ( nextByte > -1 ) {
+ throw new FileAlreadyExistsException(journalFile.getAbsolutePath());
+ }
+ } catch (final EOFException eof) {
+ // If we catch an EOF, there's no real data here, so we can overwrite the file.
+ }
+ }
+
+ this.journalId = journalId;
+ this.journalFile = journalFile;
+ this.compressionCodec = compressionCodec;
+ this.serializer = serializer;
+ this.description = "Journal Writer for " + journalFile;
+ this.fos = new FileOutputStream(journalFile);
+
+ uncompressedStream = new ByteCountingOutputStream(fos);
+ writeHeader(uncompressedStream);
+
+ if (compressionCodec != null) {
+ final CompressedOutputStream cos = compressionCodec.newCompressionOutputStream(uncompressedStream);
+ cos.beginNewBlock();
+ compressedStream = cos;
+ } else {
+ compressedStream = fos;
+ }
+
+ this.out = new ByteCountingOutputStream(compressedStream, uncompressedStream.getBytesWritten());
+ }
+
+ private void writeHeader(final OutputStream out) throws IOException {
+ final DataOutputStream dos = new DataOutputStream(out);
+ StandardJournalMagicHeader.write(out);
+ dos.writeUTF(serializer.getCodecName());
+ dos.writeInt(serializer.getVersion());
+
+ final boolean compressed = compressionCodec != null;
+ dos.writeBoolean(compressed);
+ if ( compressed ) {
+ dos.writeUTF(compressionCodec.getName());
+ }
+
+ dos.flush();
+ }
+
+ @Override
+ public long getJournalId() {
+ return journalId;
+ }
+
+ @Override
+ public void close() throws IOException {
+ finishBlock();
+
+ IOException suppressed = null;
+ try {
+ compressedStream.flush();
+ compressedStream.close();
+ } catch (final IOException ioe) {
+ suppressed = ioe;
+ }
+
+ try {
+ try {
+ uncompressedStream.flush();
+ } finally {
+ uncompressedStream.close();
+ }
+ } catch (final IOException ioe) {
+ if ( suppressed != null ) {
+ ioe.addSuppressed(suppressed);
+ }
+ throw ioe;
+ }
+
+ if ( suppressed != null ) {
+ throw suppressed;
+ }
+ }
+
+ @Override
+ public void write(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
+ final long start = System.nanoTime();
+ final int avgRecordSize = (int) (recordBytes / recordCount);
+
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream(avgRecordSize);
+ final DataOutputStream serializerDos = new DataOutputStream(baos);
+
+ final BufferedOutputStream bos = new BufferedOutputStream(out);
+ final DataOutputStream outDos = new DataOutputStream(bos);
+
+ try {
+ long id = firstEventId;
+ for ( final ProvenanceEventRecord event : events ) {
+ serializer.serialize(event, serializerDos);
+ serializerDos.flush();
+
+ final int serializedLength = baos.size();
+ final int recordLength = 8 + serializedLength; // record length is length of ID (8 bytes) plus length of serialized record
+ outDos.writeInt(recordLength);
+ outDos.writeLong(id++);
+ baos.writeTo(outDos);
+ recordBytes += recordLength;
+ recordCount++;
+ baos.reset();
+
+ eventCount++;
+ }
+ } finally {
+ outDos.flush();
+ }
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+ logger.debug("Finished writing {} events to {} in {} millis", events.size(), this, millis);
+ }
+
+
+ @Override
+ public File getJournalFile() {
+ return journalFile;
+ }
+
+ @Override
+ public void sync() throws IOException {
+ final long start = System.nanoTime();
+ fos.getFD().sync();
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()- start);
+ logger.debug("Successfully sync'ed {} in {} millis", this, millis);
+ }
+
+ @Override
+ public long getSize() {
+ return out.getBytesWritten();
+ }
+
+ @Override
+ public int getEventCount() {
+ return eventCount;
+ }
+
+ @Override
+ public long getAge(final TimeUnit timeUnit) {
+ return timeUnit.convert(System.nanoTime() - creationTime, TimeUnit.NANOSECONDS);
+ }
+
+
+ @Override
+ public void finishBlock() throws IOException {
+ if ( !blockStarted ) {
+ return;
+ }
+
+ blockStarted = false;
+
+ if ( compressedStream instanceof CompressedOutputStream ) {
+ ((CompressedOutputStream) compressedStream).finishBlock();
+ }
+ }
+
+ @Override
+ public void beginNewBlock() throws IOException {
+ if ( blockStarted ) {
+ throw new IllegalStateException("Block is already started");
+ }
+ blockStarted = true;
+
+ if ( compressedStream instanceof CompressedOutputStream ) {
+ ((CompressedOutputStream) compressedStream).beginNewBlock();
+ this.out = new ByteCountingOutputStream(compressedStream, uncompressedStream.getBytesWritten());
+ }
+ }
+
+ @Override
+ public String toString() {
+ return description;
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
new file mode 100644
index 0000000..31371af
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
@@ -0,0 +1,645 @@
+/*
+ * 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.provenance.journaling.partition;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.EventIndexWriter;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
+import org.apache.nifi.provenance.journaling.index.QueryUtils;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.JournalWriter;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalWriter;
+import org.apache.nifi.provenance.journaling.tasks.CompressionTask;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
+import org.apache.nifi.provenance.journaling.toc.StandardTocWriter;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
+import org.apache.nifi.provenance.journaling.toc.TocWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JournalingPartition implements Partition {
+ private static final Logger logger = LoggerFactory.getLogger(JournalingPartition.class);
+ private static final String JOURNAL_FILE_EXTENSION = ".journal";
+
+ private final String containerName;
+ private final int sectionIndex;
+
+ private final File section;
+ private final File journalsDir;
+ private final IndexManager indexManager;
+ private final AtomicLong containerSize;
+ private final ExecutorService executor;
+ private final JournalingRepositoryConfig config;
+
+ private JournalWriter journalWriter;
+ private TocWriter tocWriter;
+ private int numEventsAtEndOfLastBlock = 0;
+ private volatile long maxEventId = -1L;
+ private volatile Long earliestEventTime = null;
+
+ private final Lock lock = new ReentrantLock();
+ private boolean writable = true; // guarded by lock
+ private final List<File> timeOrderedJournalFiles = Collections.synchronizedList(new ArrayList<File>());
+ private final AtomicLong partitionSize = new AtomicLong(0L);
+
+ public JournalingPartition(final IndexManager indexManager, final String containerName, final int sectionIndex, final File sectionDir,
+ final JournalingRepositoryConfig config, final AtomicLong containerSize, final ExecutorService compressionExecutor) throws IOException {
+ this.indexManager = indexManager;
+ this.containerSize = containerSize;
+ this.containerName = containerName;
+ this.sectionIndex = sectionIndex;
+ this.section = sectionDir;
+ this.journalsDir = new File(section, "journals");
+ this.config = config;
+ this.executor = compressionExecutor;
+
+ if (!journalsDir.exists() && !journalsDir.mkdirs()) {
+ throw new IOException("Could not create directory " + section);
+ }
+
+ if ( journalsDir.exists() && journalsDir.isFile() ) {
+ throw new IOException("Could not create directory " + section + " because a file already exists with this name");
+ }
+ }
+
+
+ public EventIndexSearcher newIndexSearcher() throws IOException {
+ return indexManager.newIndexSearcher(containerName);
+ }
+
+ protected JournalWriter getJournalWriter(final long firstEventId) throws IOException {
+ if ( config.isReadOnly() ) {
+ throw new IllegalStateException("Cannot update repository because it is read-only");
+ }
+
+ if (isRolloverNecessary()) {
+ rollover(firstEventId);
+ }
+
+ return journalWriter;
+ }
+
+ // MUST be called with writeLock or readLock held.
+ private EventIndexWriter getIndexWriter() {
+ return indexManager.getIndexWriter(containerName);
+ }
+
+ @Override
+ public List<JournaledProvenanceEvent> registerEvents(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
+ if ( events.isEmpty() ) {
+ return Collections.emptyList();
+ }
+
+ lock.lock();
+ try {
+ if ( !writable ) {
+ throw new IOException("Cannot write to partition " + this + " because there was previously a write failure. The partition will fix itself in time if I/O problems are resolved");
+ }
+
+ final JournalWriter writer = getJournalWriter(firstEventId);
+
+ final int eventsWritten = writer.getEventCount();
+ if ( eventsWritten - numEventsAtEndOfLastBlock > config.getBlockSize() ) {
+ writer.finishBlock();
+ tocWriter.addBlockOffset(writer.getSize());
+ numEventsAtEndOfLastBlock = eventsWritten;
+ writer.beginNewBlock();
+ }
+
+ writer.write(events, firstEventId);
+
+ final List<JournaledProvenanceEvent> storedEvents = new ArrayList<>(events.size());
+ long id = firstEventId;
+ for (final ProvenanceEventRecord event : events) {
+ final JournaledStorageLocation location = new JournaledStorageLocation(containerName, String.valueOf(sectionIndex),
+ writer.getJournalId(), tocWriter.getCurrentBlockIndex(), id++);
+ final JournaledProvenanceEvent storedEvent = new JournaledProvenanceEvent(event, location);
+ storedEvents.add(storedEvent);
+ }
+
+ final EventIndexWriter indexWriter = getIndexWriter();
+ indexWriter.index(storedEvents);
+
+ if ( config.isAlwaysSync() ) {
+ writer.sync();
+ }
+
+ // update the maxEventId; we don't need a compareAndSet because the AtomicLong is modified
+ // only within a write lock. But we use AtomicLong so that we
+ if ( id > maxEventId ) {
+ maxEventId = id;
+ }
+
+ if ( earliestEventTime == null ) {
+ Long earliest = null;
+ for ( final ProvenanceEventRecord event : events ) {
+ if ( earliest == null || event.getEventTime() < earliest ) {
+ earliest = event.getEventTime();
+ }
+ }
+
+ earliestEventTime = earliest;
+ }
+
+ return storedEvents;
+ } catch (final IOException ioe) {
+ writable = false;
+ throw ioe;
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ // MUST be called with either the read lock or write lock held.
+ // determines whether or not we need to roll over the journal writer and toc writer.
+ private boolean isRolloverNecessary() {
+ if ( journalWriter == null ) {
+ return true;
+ }
+
+ final long ageSeconds = journalWriter.getAge(TimeUnit.SECONDS);
+ final long rolloverSeconds = config.getJournalRolloverPeriod(TimeUnit.SECONDS);
+ if ( ageSeconds >= rolloverSeconds ) {
+ return true;
+ }
+
+ if ( journalWriter.getSize() > config.getJournalCapacity() ) {
+ return true;
+ }
+
+ return false;
+ }
+
+ private void updateSize(final long delta) {
+ partitionSize.addAndGet(delta);
+ containerSize.addAndGet(delta);
+ }
+
+ // MUST be called with write lock held.
+ /**
+ * Rolls over the current journal (if any) and begins writing top a new journal.
+ *
+ * <p>
+ * <b>NOTE:</b> This method MUST be called with the write lock held!!
+ * </p>
+ *
+ * @param firstEventId the ID of the first event to add to this journal
+ * @throws IOException
+ */
+ private void rollover(final long firstEventId) throws IOException {
+ // if we have a writer already, close it and initiate rollover actions
+ final File finishedFile = journalWriter == null ? null : journalWriter.getJournalFile();
+ if ( journalWriter != null ) {
+ journalWriter.finishBlock();
+ journalWriter.close();
+ tocWriter.close();
+
+ final File finishedTocFile = tocWriter.getFile();
+ updateSize(finishedFile.length());
+
+ executor.submit(new Runnable() {
+ @Override
+ public void run() {
+ if ( config.isCompressOnRollover() ) {
+ final long originalSize = finishedFile.length();
+ final long compressedFileSize = new CompressionTask(finishedFile, journalWriter.getJournalId(), finishedTocFile).call();
+ final long sizeAdded = compressedFileSize - originalSize;
+ updateSize(sizeAdded);
+ }
+ }
+ });
+
+ timeOrderedJournalFiles.add(finishedFile);
+ }
+
+ // create new writers and reset state.
+ final File journalFile = new File(journalsDir, firstEventId + JOURNAL_FILE_EXTENSION);
+ journalWriter = new StandardJournalWriter(firstEventId, journalFile, null, new StandardEventSerializer());
+ try {
+ tocWriter = new StandardTocWriter(QueryUtils.getTocFile(journalFile), false, config.isAlwaysSync());
+ tocWriter.addBlockOffset(journalWriter.getSize());
+ numEventsAtEndOfLastBlock = 0;
+ } catch (final Exception e) {
+ try {
+ journalWriter.close();
+ } catch (final IOException ioe) {}
+
+ journalWriter = null;
+
+ throw e;
+ }
+
+ logger.debug("Rolling over {} from {} to {}", this, finishedFile, journalFile);
+ }
+
+
+ private Long getJournalId(final File file) {
+ long journalId;
+ final int dotIndex = file.getName().indexOf(".");
+ if ( dotIndex < 0 ) {
+ journalId = 0L;
+ } else {
+ try {
+ journalId = Long.parseLong(file.getName().substring(0, dotIndex));
+ } catch (final NumberFormatException nfe) {
+ return null;
+ }
+ }
+
+ return journalId;
+ }
+
+ @Override
+ public void restore() throws IOException {
+ lock.lock();
+ try {
+ // delete or rename files if stopped during rollover; compress any files that haven't been compressed
+ if ( !config.isReadOnly() ) {
+ final File[] children = journalsDir.listFiles();
+ if ( children != null ) {
+ final List<File> journalFiles = new ArrayList<>();
+
+ // find any journal files that either haven't been compressed or were partially compressed when
+ // we last shutdown and then restart compression.
+ for ( final File file : children ) {
+ final String filename = file.getName();
+ if ( !filename.endsWith(JOURNAL_FILE_EXTENSION) ) {
+ continue;
+ }
+
+ journalFiles.add(file);
+ updateSize(file.length());
+
+ if ( !config.isCompressOnRollover() ) {
+ continue;
+ }
+
+ if ( filename.endsWith(CompressionTask.FILE_EXTENSION) ) {
+ final File uncompressedFile = new File(journalsDir, filename.replace(CompressionTask.FILE_EXTENSION, ""));
+ if ( uncompressedFile.exists() ) {
+ // both the compressed and uncompressed version of this journal exist. The Compression Task was
+ // not complete when we shutdown. Delete the compressed journal and toc and re-start the Compression Task.
+ final File tocFile = QueryUtils.getTocFile(uncompressedFile);
+ executor.submit(new Runnable() {
+ @Override
+ public void run() {
+ final long originalSize = uncompressedFile.length();
+ final long compressedSize = new CompressionTask(uncompressedFile, getJournalId(uncompressedFile), tocFile).call();
+ final long sizeAdded = compressedSize - originalSize;
+ updateSize(sizeAdded);
+ }
+ });
+ } else {
+ // The compressed file exists but the uncompressed file does not. This means that we have finished
+ // writing the compressed file and deleted the original journal file but then shutdown before
+ // renaming the compressed file to the original filename. We can simply rename the compressed file
+ // to the original file and then address the TOC file.
+ final boolean rename = CompressionTask.rename(file, uncompressedFile);
+ if ( !rename ) {
+ logger.warn("{} During recovery, failed to rename {} to {}", this, file, uncompressedFile);
+ continue;
+ }
+
+ // Check if the compressed TOC file exists. If not, we are finished.
+ // If it does exist, then we know that it is complete, as described above, so we will go
+ // ahead and replace the uncompressed version.
+ final File tocFile = QueryUtils.getTocFile(uncompressedFile);
+ final File compressedTocFile = new File(tocFile.getParentFile(), tocFile.getName() + CompressionTask.FILE_EXTENSION);
+ if ( !compressedTocFile.exists() ) {
+ continue;
+ }
+
+ tocFile.delete();
+
+ final boolean renamedTocFile = CompressionTask.rename(compressedTocFile, tocFile);
+ if ( !renamedTocFile ) {
+ logger.warn("{} During recovery, failed to rename {} to {}", this, compressedTocFile, tocFile);
+ }
+ }
+ }
+ }
+
+ // we want to sort the list of all journal files.
+ // we need to create a map of file to last mod time, rather than comparing
+ // by using File.lastModified() because the File.lastModified() value could potentially
+ // change while running the comparator, which violates the comparator's contract.
+ timeOrderedJournalFiles.addAll(journalFiles);
+ final Map<File, Long> lastModTimes = new HashMap<>();
+ for ( final File journalFile : journalFiles ) {
+ lastModTimes.put(journalFile, journalFile.lastModified());
+ }
+ Collections.sort(timeOrderedJournalFiles, new Comparator<File>() {
+ @Override
+ public int compare(final File o1, final File o2) {
+ return lastModTimes.get(o1).compareTo(lastModTimes.get(o2));
+ }
+ });
+
+ // Get the first event in the earliest journal file so that we know what the earliest time available is
+ Collections.sort(journalFiles, new Comparator<File>() {
+ @Override
+ public int compare(final File o1, final File o2) {
+ return Long.compare(getJournalId(o1), getJournalId(o2));
+ }
+ });
+
+ for ( final File journal : journalFiles ) {
+ try (final JournalReader reader = new StandardJournalReader(journal)) {
+ final ProvenanceEventRecord record = reader.nextEvent();
+ if ( record != null ) {
+ this.earliestEventTime = record.getEventTime();
+ break;
+ }
+ } catch (final IOException ioe) {
+ }
+ }
+
+ // order such that latest journal file is first.
+ Collections.reverse(journalFiles);
+ for ( final File journal : journalFiles ) {
+ try (final JournalReader reader = new StandardJournalReader(journal);
+ final TocReader tocReader = new StandardTocReader(QueryUtils.getTocFile(journal))) {
+
+ final long lastBlockOffset = tocReader.getLastBlockOffset();
+ final ProvenanceEventRecord lastEvent = reader.getLastEvent(lastBlockOffset);
+ if ( lastEvent != null ) {
+ maxEventId = lastEvent.getEventId() + 1;
+ break;
+ }
+ } catch (final EOFException eof) {}
+ }
+
+ // We need to re-index all of the journal files that have not been indexed. We can do this by determining
+ // what is the largest event id that has been indexed for this container and section, and then re-indexing
+ // any file that has an event with an id larger than that.
+ // In order to do that, we iterate over the journal files in the order of newest (largest id) to oldest
+ // (smallest id). If the first event id in a file is greater than the max indexed, we re-index the file.
+ // Beyond that, we need to re-index one additional journal file because it's possible that if the first id
+ // is 10 and the max index id is 15, the file containing 10 could also go up to 20. So we re-index one
+ // file that has a min id less than what has been indexed; then we are done.
+ final Long maxIndexedId = indexManager.getMaxEventId(containerName, String.valueOf(sectionIndex));
+ final List<File> reindexJournals = new ArrayList<>();
+ for ( final File journalFile : journalFiles ) {
+ final Long firstEventId;
+ try {
+ firstEventId = getJournalId(journalFile);
+ } catch (final NumberFormatException nfe) {
+ // not a journal; skip this file
+ continue;
+ }
+
+ if ( maxIndexedId == null || firstEventId > maxIndexedId ) {
+ reindexJournals.add(journalFile);
+ } else {
+ reindexJournals.add(journalFile);
+ break;
+ }
+ }
+
+ // Make sure that the indexes are not pointing to events that no longer exist.
+ if ( journalFiles.isEmpty() ) {
+ indexManager.deleteEventsBefore(containerName, sectionIndex, Long.MAX_VALUE);
+ } else {
+ final File firstJournalFile = journalFiles.get(0);
+ indexManager.deleteEventsBefore(containerName, sectionIndex, getJournalId(firstJournalFile));
+ }
+
+ // The reindexJournals list is currently in order of newest to oldest. We need to re-index
+ // in order of oldest to newest, so reverse the list.
+ Collections.reverse(reindexJournals);
+
+ logger.info("Reindexing {} journal files that were not found in index for container {} and section {}", reindexJournals.size(), containerName, sectionIndex);
+ final long reindexStart = System.nanoTime();
+ for ( final File journalFile : reindexJournals ) {
+ indexManager.reindex(containerName, sectionIndex, getJournalId(journalFile), journalFile);
+ }
+ final long reindexMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - reindexStart);
+ logger.info("Finished reindexing {} journal files for container {} and section {}; reindex took {} millis",
+ reindexJournals.size(), containerName, sectionIndex, reindexMillis);
+ }
+ }
+ } finally {
+ lock.unlock();
+ }
+ }
+
+
+
+ @Override
+ public List<JournaledStorageLocation> getEvents(final long minEventId, final int maxRecords) throws IOException {
+ try (final EventIndexSearcher searcher = newIndexSearcher()) {
+ return searcher.getEvents(minEventId, maxRecords);
+ }
+ }
+
+ @Override
+ public void shutdown() {
+ if ( journalWriter != null ) {
+ try {
+ journalWriter.finishBlock();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to finish writing Block to {} due to {}", journalWriter, ioe);
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+
+ try {
+ journalWriter.close();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to close {} due to {}", journalWriter, ioe);
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+
+ try {
+ tocWriter.close();
+ } catch (final IOException ioe) {
+ logger.warn("Failed to close {} due to {}", tocWriter, ioe);
+ if ( logger.isDebugEnabled() ) {
+ logger.warn("", ioe);
+ }
+ }
+ }
+
+ }
+
+ @Override
+ public long getMaxEventId() {
+ return maxEventId;
+ }
+
+ @Override
+ public Long getEarliestEventTime() throws IOException {
+ return earliestEventTime;
+ }
+
+ @Override
+ public String toString() {
+ return "Partition[section=" + sectionIndex + "]";
+ }
+
+ @Override
+ public void verifyWritable(final long nextId) throws IOException {
+ final long freeSpace = section.getFreeSpace();
+ final long freeMegs = freeSpace / 1024 / 1024;
+ if (freeMegs < 10) {
+ // if not at least 10 MB, don't even try to write
+ throw new IOException("Not Enough Disk Space: partition housing " + section + " has only " + freeMegs + " MB of storage available");
+ }
+
+ rollover(nextId);
+ writable = true;
+ }
+
+ private boolean delete(final File journalFile) {
+ for (int i=0; i < 10; i++) {
+ if ( journalFile.delete() || !journalFile.exists() ) {
+ return true;
+ } else {
+ try {
+ Thread.sleep(100L);
+ } catch (final InterruptedException ie) {}
+ }
+ }
+
+ return false;
+ }
+
+ @Override
+ public void deleteOldEvents(final long earliestEventTimeToDelete) throws IOException {
+ final Set<File> removeFromTimeOrdered = new HashSet<>();
+
+ final long start = System.nanoTime();
+ try {
+ for ( final File journalFile : timeOrderedJournalFiles ) {
+ // since these are time-ordered, if we find one that we don't want to delete, we're done.
+ if ( journalFile.lastModified() < earliestEventTimeToDelete ) {
+ return;
+ }
+
+ final long journalSize;
+ if ( journalFile.exists() ) {
+ journalSize = journalFile.length();
+ } else {
+ continue;
+ }
+
+ if ( delete(journalFile) ) {
+ removeFromTimeOrdered.add(journalFile);
+ } else {
+ logger.warn("Failed to remove expired journal file {}; will attempt to delete again later", journalFile);
+ }
+
+ updateSize(-journalSize);
+ final File tocFile = QueryUtils.getTocFile(journalFile);
+ if ( !delete(tocFile) ) {
+ logger.warn("Failed to remove TOC file for expired journal file {}; will attempt to delete again later", journalFile);
+ }
+ }
+ } finally {
+ timeOrderedJournalFiles.removeAll(removeFromTimeOrdered);
+ }
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+ logger.info("Removed {} expired journal files from container {}, section {}; total time for deletion was {} millis",
+ removeFromTimeOrdered.size(), containerName, sectionIndex, millis);
+ }
+
+
+ @Override
+ public void deleteOldest() throws IOException {
+ File removeFromTimeOrdered = null;
+
+ final long start = System.nanoTime();
+ try {
+ for ( final File journalFile : timeOrderedJournalFiles ) {
+ final long journalSize;
+ if ( journalFile.exists() ) {
+ journalSize = journalFile.length();
+ } else {
+ continue;
+ }
+
+ if ( delete(journalFile) ) {
+ removeFromTimeOrdered = journalFile;
+ } else {
+ throw new IOException("Cannot delete oldest event file " + journalFile);
+ }
+
+ final File tocFile = QueryUtils.getTocFile(journalFile);
+ if ( !delete(tocFile) ) {
+ logger.warn("Failed to remove TOC file for expired journal file {}; will attempt to delete again later", journalFile);
+ }
+
+ updateSize(-journalSize);
+ indexManager.deleteEvents(containerName, sectionIndex, getJournalId(journalFile));
+ }
+ } finally {
+ if ( removeFromTimeOrdered != null ) {
+ timeOrderedJournalFiles.remove(removeFromTimeOrdered);
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+ logger.info("Removed oldest event file {} from container {}, section {}; total time for deletion was {} millis",
+ removeFromTimeOrdered, containerName, sectionIndex, millis);
+ } else {
+ logger.debug("No journals to remove for {}", this);
+ }
+ }
+ }
+
+
+ @Override
+ public long getPartitionSize() {
+ return partitionSize.get();
+ }
+
+ @Override
+ public long getContainerSize() {
+ return containerSize.get();
+ }
+
+ @Override
+ public String getContainerName() {
+ return containerName;
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
new file mode 100644
index 0000000..9efae04
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
@@ -0,0 +1,126 @@
+/*
+ * 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.provenance.journaling.partition;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+
+
+/**
+ * Represents a single Partition of the Journaling Provenance Repository. The repository is split into multiple
+ * partitions in order to provide higher throughput.
+ *
+ * Implementations of this interface MUST be threadsafe.
+ */
+public interface Partition {
+
+ /**
+ * Returns a new EventIndexSearcher that can be used to search the events in this partition
+ * @return
+ * @throws IOException
+ */
+ EventIndexSearcher newIndexSearcher() throws IOException;
+
+ /**
+ * Registers the given events with this partition. This includes persisting the events and indexing
+ * them so that they are searchable.
+ * @param events
+ * @return
+ */
+ List<JournaledProvenanceEvent> registerEvents(Collection<ProvenanceEventRecord> events, long firstEventId) throws IOException;
+
+ /**
+ * Restore state after a restart of NiFi
+ */
+ void restore() throws IOException;
+
+ /**
+ * Shuts down the Partition so that it can no longer be used
+ */
+ void shutdown();
+
+ /**
+ * Returns the largest event ID stored in this partition
+ * @return
+ */
+ long getMaxEventId();
+
+ /**
+ * Returns the name of the container that this Partition operates on
+ * @return
+ */
+ String getContainerName();
+
+ /**
+ * Returns the locations of events that have an id at least equal to minEventId, returning the events
+ * with the smallest ID's possible that are greater than minEventId
+ *
+ * @param minEventId
+ * @param maxRecords
+ * @return
+ */
+ List<JournaledStorageLocation> getEvents(long minEventId, int maxRecords) throws IOException;
+
+ /**
+ * Returns the timestamp of the earliest event in this Partition, or <code>null</code> if the Partition
+ * contains no events
+ * @return
+ * @throws IOException
+ */
+ Long getEarliestEventTime() throws IOException;
+
+ /**
+ * Verifies that the partition is able to be written to. A Partition may need to create a new journal
+ * in order to verify. In this case, the nextId is provided so that the Partition knows the minimum event id
+ * that will be written to the partition
+ *
+ * @throws IOException
+ */
+ void verifyWritable(long nextId) throws IOException;
+
+
+ /**
+ * Deletes any journal for this partition that occurred before the given time
+ * @param earliestEventTimeToDelete
+ * @throws IOException
+ */
+ void deleteOldEvents(long earliestEventTimeToDelete) throws IOException;
+
+ /**
+ * Returns the size of this partition in bytes
+ * @return
+ */
+ long getPartitionSize();
+
+ /**
+ * Returns the size of the journals in the entire container
+ * @return
+ */
+ long getContainerSize();
+
+ /**
+ * Deletes the oldest journal from this partition
+ * @throws IOException
+ */
+ void deleteOldest() throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionAction.java
new file mode 100644
index 0000000..8c680f5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionAction.java
@@ -0,0 +1,23 @@
+/*
+ * 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.provenance.journaling.partition;
+
+import java.io.IOException;
+
+public interface PartitionAction<T> {
+ T perform(Partition partition) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
new file mode 100644
index 0000000..e751543
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
@@ -0,0 +1,82 @@
+/*
+ * 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.provenance.journaling.partition;
+
+import java.io.IOException;
+import java.util.Set;
+
+
+/**
+ * The PartitionManager is responsible for accessing and maintaining the Partitions so that they are
+ * are written to efficiently and in a thread-safe manner.
+ */
+public interface PartitionManager {
+
+ /**
+ * Performs the given action against one of the partitions
+ *
+ * @param action the action to perform
+ * @param writeAction specifies whether or not the action writes to the repository
+ * @return
+ * @throws IOException
+ */
+ <T> T withPartition(PartitionAction<T> action, boolean writeAction) throws IOException;
+
+ /**
+ * Performs the given action against one of the partitions
+ *
+ * @param action the action to perform
+ * @param writeAction specifies whether or not the action writes to the repository
+ * @throws IOException
+ */
+ void withPartition(VoidPartitionAction action, boolean writeAction) throws IOException;
+
+
+ /**
+ * Performs the given Action on each partition and returns the set of results. This method does
+ * not use the thread pool in order to perform the request in parallel. This is desirable for
+ * very quick functions, as the thread pool can be fully utilized, resulting in a quick function
+ * taking far longer than it should.
+ *
+ * @param action the action to perform
+ * @param writeAction specifies whether or not the action writes to the repository
+ * @return
+ */
+ <T> Set<T> withEachPartitionSerially(PartitionAction<T> action, boolean writeAction) throws IOException;
+
+
+ /**
+ * Performs the given Action on each partition. Unlike
+ * {@link #withEachPartition(PartitionAction))}, this method does not use the thread pool
+ * in order to perform the request in parallel. This is desirable for very quick functions,
+ * as the thread pool can be fully utilized, resulting in a quick function taking far longer
+ * than it should.
+ *
+ * @param action the action to perform
+ * @param writeAction specifies whether or not the action writes to the repository
+ * @return
+ */
+ void withEachPartitionSerially(VoidPartitionAction action, boolean writeAction) throws IOException;
+
+ void shutdown();
+
+ /**
+ * Triggers the Partition Manager to delete events from journals and indices based on the sizes of the containers
+ * and overall size of the repository
+ */
+ void deleteEventsBasedOnSize();
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
new file mode 100644
index 0000000..0b68f00
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
@@ -0,0 +1,351 @@
+/*
+ * 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.provenance.journaling.partition;
+
+import java.io.File;
+import java.io.IOException;
+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.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
+import org.apache.nifi.util.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class QueuingPartitionManager implements PartitionManager {
+
+ private static final Logger logger = LoggerFactory.getLogger(QueuingPartitionManager.class);
+
+ private final IndexManager indexManager;
+ private final JournalingRepositoryConfig config;
+ private final BlockingQueue<Partition> partitionQueue;
+ private final JournalingPartition[] partitionArray;
+ private final AtomicLong eventIdGenerator;
+ private volatile boolean shutdown = false;
+
+ private final Set<Partition> blackListedPartitions = Collections.synchronizedSet(new HashSet<Partition>());
+
+ public QueuingPartitionManager(final IndexManager indexManager, final AtomicLong eventIdGenerator, final JournalingRepositoryConfig config, final ScheduledExecutorService workerExecutor, final ExecutorService compressionExecutor) throws IOException {
+ this.indexManager = indexManager;
+ this.config = config;
+ this.eventIdGenerator = eventIdGenerator;
+
+ // We can consider using a PriorityQueue here instead. Keep track of how many Partitions are being written
+ // to for each container, as a container usually maps to a physical drive. Then, prioritize the queue
+ // so that the partitions that belong to Container A get a higher priority than those belonging to Container B
+ // if there are currently more partitions on Container B being written to (i.e., we prefer a partition for the
+ // container that is the least used at this moment). Would require significant performance testing to see if it
+ // really provides any benefit.
+ this.partitionQueue = new LinkedBlockingQueue<>(config.getPartitionCount());
+ this.partitionArray = new JournalingPartition[config.getPartitionCount()];
+
+ final List<Tuple<String, File>> containerTuples = new ArrayList<>(config.getContainers().size());
+ for ( final Map.Entry<String, File> entry : config.getContainers().entrySet() ) {
+ containerTuples.add(new Tuple<>(entry.getKey(), entry.getValue()));
+ }
+
+ final Map<String, AtomicLong> containerSizes = new HashMap<>();
+ for ( final String containerName : config.getContainers().keySet() ) {
+ containerSizes.put(containerName, new AtomicLong(0L));
+ }
+
+ for (int i=0; i < config.getPartitionCount(); i++) {
+ final Tuple<String, File> tuple = containerTuples.get(i % containerTuples.size());
+ final File section = new File(tuple.getValue(), String.valueOf(i));
+
+ final String containerName = tuple.getKey();
+ final JournalingPartition partition = new JournalingPartition(indexManager, containerName, i,
+ section, config, containerSizes.get(containerName), compressionExecutor);
+ partition.restore();
+ partitionQueue.offer(partition);
+ partitionArray[i] = partition;
+ }
+
+ workerExecutor.scheduleWithFixedDelay(new CheckBlackListedPartitions(), 30, 30, TimeUnit.SECONDS);
+ }
+
+ @Override
+ public void shutdown() {
+ this.shutdown = true;
+
+ for ( final Partition partition : partitionArray ) {
+ partition.shutdown();
+ }
+ }
+
+ Partition nextPartition(final boolean writeAction, final boolean waitIfNeeded) {
+ Partition partition = null;
+
+ final List<Partition> partitionsSkipped = new ArrayList<>();
+ try {
+ while (partition == null) {
+ if (shutdown) {
+ throw new RuntimeException("Journaling Provenance Repository is shutting down");
+ }
+
+ try {
+ partition = partitionQueue.poll(1, TimeUnit.SECONDS);
+ } catch (final InterruptedException ie) {
+ }
+
+ if ( partition == null ) {
+ if ( blackListedPartitions.size() >= config.getPartitionCount() ) {
+ throw new RuntimeException("Cannot persist to the Journal Provenance Repository because all partitions have been blacklisted due to write failures");
+ }
+
+ // we are out of partitions. Add back all of the partitions that we skipped so we
+ // can try them again.
+ partitionQueue.addAll(partitionsSkipped);
+ partitionsSkipped.clear();
+ } else if (writeAction) {
+ if ( waitIfNeeded ) {
+ // determine if the container is full.
+ final String containerName = partition.getContainerName();
+ long desiredMaxContainerCapacity = config.getMaxCapacity(containerName);
+
+ // If no max capacity set for the container itself, use 1/N of repo max
+ // where N is the number of containers
+ if ( desiredMaxContainerCapacity == config.getMaxStorageCapacity() ) {
+ desiredMaxContainerCapacity = config.getMaxStorageCapacity() / config.getContainers().size();
+ }
+
+ // if the partition is more than 10% over its desired capacity, we don't want to write to it.
+ if ( partition.getContainerSize() > 1.1 * desiredMaxContainerCapacity ) {
+ partitionsSkipped.add(partition);
+ continue;
+ }
+ } else {
+ return null;
+ }
+ }
+ }
+ } finally {
+ partitionQueue.addAll( partitionsSkipped );
+ }
+
+ return partition;
+ }
+
+
+ private void blackList(final Partition partition) {
+ blackListedPartitions.add(partition);
+ }
+
+ @Override
+ public <T> T withPartition(final PartitionAction<T> action, final boolean writeAction) throws IOException {
+ final Partition partition = nextPartition(writeAction, true);
+
+ boolean ioe = false;
+ try {
+ return action.perform(partition);
+ } catch (final IOException e) {
+ ioe = true;
+ throw e;
+ } finally {
+ if ( ioe && writeAction ) {
+ blackList(partition);
+ } else {
+ partitionQueue.offer(partition);
+ }
+ }
+ }
+
+ @Override
+ public void withPartition(final VoidPartitionAction action, final boolean writeAction) throws IOException {
+ final Partition partition = nextPartition(writeAction, true);
+
+ boolean ioe = false;
+ try {
+ action.perform(partition);
+ } catch (final IOException e) {
+ ioe = true;
+ throw e;
+ } finally {
+ if ( ioe && writeAction ) {
+ blackList(partition);
+ } else {
+ partitionQueue.offer(partition);
+ }
+ }
+ }
+
+
+ @Override
+ public <T> Set<T> withEachPartitionSerially(final PartitionAction<T> action, final boolean writeAction) throws IOException {
+ if ( writeAction && blackListedPartitions.size() > 0 ) {
+ throw new IOException("Cannot perform action {} because at least one partition has been blacklisted (i.e., writint to the partition failed)");
+ }
+
+ final Set<T> results = new HashSet<>(partitionArray.length);
+ for ( final Partition partition : partitionArray ) {
+ results.add( action.perform(partition) );
+ }
+
+ return results;
+ }
+
+ @Override
+ public void withEachPartitionSerially(final VoidPartitionAction action, final boolean writeAction) throws IOException {
+ if ( writeAction && blackListedPartitions.size() > 0 ) {
+ throw new IOException("Cannot perform action {} because at least one partition has been blacklisted (i.e., writint to the partition failed)");
+ }
+
+ for ( final Partition partition : partitionArray ) {
+ action.perform(partition);
+ }
+ }
+
+ private long getTotalSize() {
+ long totalSize = 0L;
+
+ for ( final JournalingPartition partition : partitionArray ) {
+ totalSize += partition.getPartitionSize();
+ }
+
+ for ( final String containerName : config.getContainers().keySet() ) {
+ totalSize += indexManager.getSize(containerName);
+ }
+
+ return totalSize;
+ }
+
+
+ /**
+ * Responsible for looking at partitions that have been marked as blacklisted and checking if they
+ * are able to be written to now. If so, adds them back to the partition queue; otherwise, leaves
+ * them as blacklisted
+ */
+ private class CheckBlackListedPartitions implements Runnable {
+ @Override
+ public void run() {
+ final Set<Partition> reclaimed = new HashSet<>();
+
+ final Set<Partition> partitions = new HashSet<>(blackListedPartitions);
+ for ( final Partition partition : partitions ) {
+ final long nextId = eventIdGenerator.get();
+ if ( nextId <= 0 ) {
+ // we don't have an ID to use yet. Don't attempt to do anything yet.
+ return;
+ }
+
+ try {
+ partition.verifyWritable(nextId);
+ reclaimed.add(partition);
+ } catch (final IOException ioe) {
+ logger.debug("{} is still blackListed due to {}", partition, ioe);
+ }
+ }
+
+ // any partition that is reclaimable is now removed from the set of blacklisted
+ // partitions and added back to our queue of partitions
+ blackListedPartitions.removeAll(reclaimed);
+ partitionQueue.addAll(reclaimed);
+ }
+ }
+
+
+ @Override
+ public void deleteEventsBasedOnSize() {
+ final Map<String, List<JournalingPartition>> containerPartitionMap = new HashMap<>();
+
+ for ( final JournalingPartition partition : partitionArray ) {
+ final String container = partition.getContainerName();
+ List<JournalingPartition> list = containerPartitionMap.get(container);
+ if ( list == null ) {
+ list = new ArrayList<>();
+ containerPartitionMap.put(container, list);
+ }
+
+ list.add(partition);
+ }
+
+ int iterations = 0;
+ for ( final String containerName : config.getContainers().keySet() ) {
+ // continue as long as we need to delete data from this container.
+ while (true) {
+ // don't hammer the disks if we can't delete anything
+ if ( iterations++ > 0 ) {
+ try {
+ Thread.sleep(1000L);
+ } catch (final InterruptedException ie) {}
+ }
+
+ final List<JournalingPartition> containerPartitions = containerPartitionMap.get(containerName);
+ final long containerSize = containerPartitions.get(0).getContainerSize();
+ final long maxContainerCapacity = config.getMaxCapacity(containerName);
+ if ( containerSize < maxContainerCapacity ) {
+ break;
+ }
+
+ logger.debug("Container {} exceeds max capacity of {} bytes with a size of {} bytes; deleting oldest events", containerName, maxContainerCapacity, containerSize);
+
+ // container is too large. Delete oldest journal from each partition in this container.
+ for ( final Partition partition : containerPartitions ) {
+ try {
+ partition.deleteOldest();
+ } catch (final IOException ioe) {
+ logger.error("Failed to delete events from {} due to {}", partition, ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+ }
+ }
+ }
+ }
+
+ long totalSize;
+ iterations = 0;
+ while ((totalSize = getTotalSize()) >= config.getMaxStorageCapacity()) {
+ logger.debug("Provenance Repository exceeds max capacity of {} bytes with a size of {}; deleting oldest events", config.getMaxStorageCapacity(), totalSize);
+
+ // don't hammer the disks if we can't delete anything
+ if ( iterations++ > 0 ) {
+ try {
+ Thread.sleep(1000L);
+ } catch (final InterruptedException ie) {}
+ }
+
+ for ( final Partition partition : partitionArray ) {
+ try {
+ partition.deleteOldest();
+ } catch (final IOException ioe) {
+ logger.error("Failed to delete events from {} due to {}", partition, ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+ }
+ }
+
+ // don't hammer the disks if we can't delete anything
+ try {
+ Thread.sleep(1000L);
+ } catch (final InterruptedException ie) {}
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/VoidPartitionAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/VoidPartitionAction.java
new file mode 100644
index 0000000..beaa187
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/VoidPartitionAction.java
@@ -0,0 +1,23 @@
+/*
+ * 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.provenance.journaling.partition;
+
+import java.io.IOException;
+
+public interface VoidPartitionAction {
+ void perform(Partition partition) throws IOException;
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
new file mode 100644
index 0000000..3fcdd77
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
@@ -0,0 +1,107 @@
+/*
+ * 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.provenance.journaling.query;
+
+import java.io.Closeable;
+
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+
+public interface QueryManager extends Closeable {
+ /**
+ * Submits an asynchronous request to process the given query, returning an
+ * identifier that can be used to fetch the results at a later time
+ *
+ * @param query
+ * @return
+ */
+ QuerySubmission submitQuery(Query query);
+
+ /**
+ * Registers the given query submission with this manager so that it can be retrieved later.
+ * @param submission
+ */
+ void registerSubmission(ProvenanceQuerySubmission submission);
+
+ /**
+ * Retrieves the ProvenanceQuerySubmission with the given identifier, if it exists, else returns <code>null</code>.
+ * @param id
+ * @return
+ */
+ ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(String id);
+
+ /**
+ * Returns the QueryResult associated with the given identifier, if the
+ * query has finished processing. If the query has not yet finished running,
+ * returns <code>null</code>.
+ *
+ * @param queryIdentifier
+ *
+ * @return
+ */
+ QuerySubmission retrieveQuerySubmission(String queryIdentifier);
+
+ /**
+ * Returns the {@link ComputeLineageSubmission} associated with the given
+ * identifier
+ *
+ * @param lineageIdentifier
+ * @return
+ */
+ ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier);
+
+ /**
+ * Submits a Lineage Computation to be completed and returns the
+ * AsynchronousLineageResult that indicates the status of the request and
+ * the results, if the computation is complete.
+ *
+ * @param flowFileUuid the UUID of the FlowFile for which the Lineage should
+ * be calculated
+ * @return a {@link ComputeLineageSubmission} object that can be used to
+ * check if the computing is complete and if so get the results
+ */
+ ComputeLineageSubmission submitLineageComputation(String flowFileUuid);
+
+ /**
+ * Submits a request to expand the parents of the event with the given id
+ *
+ * @param eventRepo the repository from which to obtain the event information
+ * @param eventId the one-up id of the Event to expand
+ * @return
+ *
+ * @throws IllegalArgumentException if the given identifier identifies a
+ * Provenance Event that has a Type that is not expandable or if the
+ * identifier cannot be found
+ */
+ ComputeLineageSubmission submitExpandParents(final ProvenanceEventRepository eventRepo, long eventId);
+
+ /**
+ * Submits a request to expand the children of the event with the given id
+ *
+ * @param eventRepo the repository from which to obtain the event information
+ * @param eventId the one-up id of the Event
+ * @return
+ *
+ * @throws IllegalArgumentException if the given identifier identifies a
+ * Provenance Event that has a Type that is not expandable or if the
+ * identifier cannot be found
+ */
+ ComputeLineageSubmission submitExpandChildren(final ProvenanceEventRepository eventRepo, long eventId);
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
new file mode 100644
index 0000000..aebeade
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
@@ -0,0 +1,445 @@
+/*
+ * 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.provenance.journaling.query;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.nifi.provenance.AsyncLineageSubmission;
+import org.apache.nifi.provenance.AsyncQuerySubmission;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.IndexAction;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
+import org.apache.nifi.provenance.journaling.index.QueryUtils;
+import org.apache.nifi.provenance.journaling.index.SearchResult;
+import org.apache.nifi.provenance.journaling.index.VoidIndexAction;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.lineage.LineageComputationType;
+import org.apache.nifi.provenance.query.ProvenanceQueryResult;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StandardQueryManager implements QueryManager {
+ private static final Logger logger = LoggerFactory.getLogger(StandardQueryManager.class);
+
+ private final int maxConcurrentQueries;
+ private final IndexManager indexManager;
+ private final ExecutorService executor;
+ private final JournalingRepositoryConfig config;
+ private final ConcurrentMap<String, AsyncQuerySubmission> querySubmissionMap = new ConcurrentHashMap<>();
+ private final ConcurrentMap<String, AsyncLineageSubmission> lineageSubmissionMap = new ConcurrentHashMap<>();
+ private final ConcurrentMap<String, ProvenanceQuerySubmission> provenanceQuerySubmissionMap = new ConcurrentHashMap<>();
+
+ public StandardQueryManager(final IndexManager indexManager, final ExecutorService executor, final JournalingRepositoryConfig config, final int maxConcurrentQueries) {
+ this.config = config;
+ this.maxConcurrentQueries = maxConcurrentQueries;
+ this.indexManager = indexManager;
+ this.executor = executor;
+ }
+
+ @Override
+ public QuerySubmission submitQuery(final Query query) {
+ final int numQueries = querySubmissionMap.size();
+ if (numQueries > maxConcurrentQueries) {
+ throw new IllegalStateException("Cannot process query because there are currently " + numQueries + " queries whose results have not been deleted (likely due to poorly behaving clients not issuing DELETE requests). Please try again later.");
+ }
+
+ if (query.getEndDate() != null && query.getStartDate() != null && query.getStartDate().getTime() > query.getEndDate().getTime()) {
+ throw new IllegalArgumentException("Query End Time cannot be before Query Start Time");
+ }
+
+ if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) {
+ final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
+
+ // empty query. Just get the latest events.
+ final Runnable runnable = new Runnable() {
+ @Override
+ public void run() {
+ try {
+ logger.debug("Fetching latest events from Provenance repo");
+ final long indexStartNanos = System.nanoTime();
+
+ // Query each index for the latest events.
+ final Set<List<JournaledStorageLocation>> locationSet = indexManager.withEachIndex(new IndexAction<List<JournaledStorageLocation>>() {
+ @Override
+ public List<JournaledStorageLocation> perform(final EventIndexSearcher searcher) throws IOException {
+ return searcher.getLatestEvents(query.getMaxResults());
+ }
+ });
+ final long indexMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - indexStartNanos);
+ final long retrievalStartNanos = System.nanoTime();
+
+ final List<JournaledStorageLocation> orderedLocations = new ArrayList<>();
+ for ( final List<JournaledStorageLocation> locations : locationSet ) {
+ orderedLocations.addAll(locations);
+ }
+
+ Collections.sort(orderedLocations, new Comparator<JournaledStorageLocation>() {
+ @Override
+ public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+ return Long.compare(o1.getEventId(), o2.getEventId());
+ }
+ });
+
+ final List<JournaledStorageLocation> locationsToKeep;
+ if ( orderedLocations.size() > query.getMaxResults() ) {
+ locationsToKeep = orderedLocations.subList(0, query.getMaxResults());
+ } else {
+ locationsToKeep = orderedLocations;
+ }
+
+ final List<StoredProvenanceEvent> matchingRecords = getEvents(locationsToKeep, new AtomicInteger(locationsToKeep.size()));
+
+ final long totalNumEvents = indexManager.getNumberOfEvents();
+ final long retrievalMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - retrievalStartNanos);
+ logger.debug("Updated query result with {} matching records; total number of events = {}; index search took {} millis, event retrieval took {} millis", matchingRecords.size(), totalNumEvents, indexMillis, retrievalMillis);
+ result.getResult().update(matchingRecords, totalNumEvents);
+ } catch (final Exception e) {
+ result.getResult().setError("Failed to obtain latest events in repository due to " + e);
+ logger.error("Failed to obtain latest events in repository due to {}", e.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", e);
+ }
+ }
+ }
+ };
+
+ executor.submit(runnable);
+ querySubmissionMap.put(query.getIdentifier(), result);
+ return result;
+ }
+
+ final AtomicInteger retrievalCount = new AtomicInteger(query.getMaxResults());
+ final AsyncQuerySubmission submission = new AsyncQuerySubmission(query, indexManager.getNumberOfIndices()) {
+ @Override
+ public void cancel() {
+ querySubmissionMap.remove(query.getIdentifier());
+ super.cancel();
+ }
+ };
+
+ querySubmissionMap.put(query.getIdentifier(), submission);
+
+ try {
+ indexManager.withEachIndex(new VoidIndexAction() {
+ @Override
+ public void perform(final EventIndexSearcher searcher) throws IOException {
+ try {
+ logger.debug("Running {} against {}", query, searcher);
+
+ final long indexStart = System.nanoTime();
+ final SearchResult searchResult = searcher.search(query);
+ final long indexMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - indexStart);
+ logger.debug("{} has {} hits against {} over {} files", query, searchResult.getTotalCount(), searcher, searchResult.getLocations().size());
+
+ final long retrievalStart = System.nanoTime();
+ final List<StoredProvenanceEvent> matchingRecords = getEvents(searchResult.getLocations(), retrievalCount);
+ final long retrievalMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - retrievalStart);
+
+ logger.debug("Finished executing {} against {}; found {} total matches, retrieved {} of them; index search took {} millis, record retrieval took {} millis",
+ query, searcher, searchResult.getTotalCount(), matchingRecords.size(), indexMillis, retrievalMillis);
+ submission.getResult().update(matchingRecords, searchResult.getTotalCount());
+ } catch (final Throwable t) {
+ submission.getResult().setError("Failed to execute query " + query + " against " + searcher + " due to " + t);
+ throw t;
+ }
+ }
+ }, true);
+ } catch (final IOException ioe) {
+ // only set the error here if it's not already set because we have the least amount of information here
+ if ( submission.getResult().getError() == null ) {
+ submission.getResult().setError("Failed to execute query " + query + " due to " + ioe);
+ }
+ }
+
+ return submission;
+ }
+
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ private List<StoredProvenanceEvent> getEvents(final List<JournaledStorageLocation> allLocations, final AtomicInteger retrievalCount) throws IOException {
+ final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>();
+ final Map<File, List<JournaledStorageLocation>> locationMap = QueryUtils.orderLocations((List) allLocations, config);
+ for ( final Map.Entry<File, List<JournaledStorageLocation>> entry : locationMap.entrySet() ) {
+ final File journalFile = entry.getKey();
+ final List<JournaledStorageLocation> locations = entry.getValue();
+
+ if ( retrievalCount.get() <= 0 ) {
+ break;
+ }
+
+ try (final JournalReader reader = new StandardJournalReader(journalFile);
+ final TocReader tocReader = new StandardTocReader(QueryUtils.getTocFile(journalFile))) {
+
+ for ( final JournaledStorageLocation location : locations ) {
+ final long blockOffset = tocReader.getBlockOffset(location.getBlockIndex());
+ final ProvenanceEventRecord event = reader.getEvent(blockOffset, location.getEventId());
+ matchingRecords.add(new JournaledProvenanceEvent(event, location));
+
+ final int recordsLeft = retrievalCount.decrementAndGet();
+ if ( recordsLeft <= 0 ) {
+ break;
+ }
+ }
+ }
+ }
+
+ return matchingRecords;
+ }
+
+ @Override
+ public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
+ return querySubmissionMap.get(queryIdentifier);
+ }
+
+ @Override
+ public ComputeLineageSubmission retrieveLineageSubmission(final String lineageIdentifier) {
+ return lineageSubmissionMap.get(lineageIdentifier);
+ }
+
+ @Override
+ public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) {
+ return submitLineageComputation(Collections.singleton(flowFileUuid), LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
+ }
+
+ private AsyncLineageSubmission submitLineageComputation(final Collection<String> flowFileUuids, final LineageComputationType computationType, final Long eventId, final long startTimestamp, final long endTimestamp) {
+ final AsyncLineageSubmission lineageSubmission = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, indexManager.getNumberOfIndices());
+
+ final AtomicInteger retrievalCount = new AtomicInteger(2000);
+ final Runnable runnable = new Runnable() {
+ @Override
+ public void run() {
+ try {
+ indexManager.withEachIndex(new VoidIndexAction() {
+ @Override
+ public void perform(EventIndexSearcher searcher) throws IOException {
+ logger.debug("Obtaining lineage events for FlowFile UUIDs {} for {}", flowFileUuids, searcher);
+ final long startNanos = System.nanoTime();
+
+ final List<JournaledStorageLocation> locations = searcher.getEventsForFlowFiles(flowFileUuids, startTimestamp, endTimestamp);
+ final List<StoredProvenanceEvent> matchingRecords = getEvents(locations, retrievalCount);
+ lineageSubmission.getResult().update(matchingRecords);
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+ logger.debug("Finished querying for lineage events; found {} events in {} millis", matchingRecords.size(), millis);
+ }
+ });
+ } catch (final IOException ioe) {
+ lineageSubmission.getResult().setError("Failed to calculate FlowFile Lineage due to " + ioe);
+ logger.error("Failed to calculate FlowFile Lineage due to {}", ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+ }
+ }
+ };
+
+ executor.submit(runnable);
+ lineageSubmissionMap.putIfAbsent(lineageSubmission.getLineageIdentifier(), lineageSubmission);
+ return lineageSubmission;
+ }
+
+
+ @Override
+ public ComputeLineageSubmission submitExpandChildren(final ProvenanceEventRepository eventRepo, final long eventId) {
+ final Set<String> flowFileUuids = Collections.synchronizedSet(new HashSet<String>());
+ final AsyncLineageSubmission lineageSubmission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, flowFileUuids, indexManager.getNumberOfIndices());
+
+ final Runnable runnable = new Runnable() {
+ @Override
+ public void run() {
+ try {
+ logger.debug("Obtaining event with id {} in order to expand children", eventId);
+ final StoredProvenanceEvent event = eventRepo.getEvent(eventId);
+ if ( event == null ) {
+ lineageSubmission.getResult().setError("Cannot expand children of event with ID " + eventId + " because that event cannot be found");
+ logger.warn("Cannot expand children of event with ID {} because that event cannot be found", eventId);
+ return;
+ }
+
+ logger.debug("Found event with id {}; searching for children", eventId);
+
+ switch (event.getEventType()) {
+ case CLONE:
+ case FORK:
+ case JOIN:
+ case REPLAY:
+ break;
+ default:
+ logger.warn("Cannot expand children of event with ID {} because event type is {}", eventId, event.getEventType());
+ lineageSubmission.getResult().setError("Cannot expand children of event with ID " + eventId +
+ " because that event is of type " + event.getEventType() +
+ ", and that type does not support expansion of children");
+ return;
+ }
+
+ final List<String> childUuids = event.getChildUuids();
+ flowFileUuids.addAll(childUuids);
+
+ final AtomicInteger retrievalCount = new AtomicInteger(100);
+ indexManager.withEachIndex(new VoidIndexAction() {
+ @Override
+ public void perform(EventIndexSearcher searcher) throws IOException {
+ final long startNanos = System.nanoTime();
+ logger.debug("Finding children of event with id {} using {}", eventId, searcher);
+
+ final List<JournaledStorageLocation> locations = searcher.getEventsForFlowFiles(flowFileUuids, event.getEventTime(), Long.MAX_VALUE);
+ final List<StoredProvenanceEvent> matchingRecords = getEvents(locations, retrievalCount);
+ lineageSubmission.getResult().update(matchingRecords);
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+ logger.debug("Found {} children of event {} in {} millis", matchingRecords.size(), eventId, millis);
+ }
+ });
+ } catch (final IOException ioe) {
+
+ }
+ }
+ };
+
+ executor.submit(runnable);
+ lineageSubmissionMap.putIfAbsent(lineageSubmission.getLineageIdentifier(), lineageSubmission);
+ return lineageSubmission;
+ }
+
+ @Override
+ public ComputeLineageSubmission submitExpandParents(final ProvenanceEventRepository eventRepo, final long eventId) {
+ final Set<String> flowFileUuids = Collections.synchronizedSet(new HashSet<String>());
+ final AsyncLineageSubmission lineageSubmission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, flowFileUuids, indexManager.getNumberOfIndices());
+
+ final Runnable runnable = new Runnable() {
+ @Override
+ public void run() {
+ try {
+ logger.debug("Obtaining event with id {} in order to expand children", eventId);
+ final StoredProvenanceEvent event = eventRepo.getEvent(eventId);
+ if ( event == null ) {
+ logger.warn("Cannot expand children of event with ID {} because that event cannot be found", eventId);
+ lineageSubmission.getResult().setError("Cannot expand children of event with ID " + eventId + " because that event cannot be found");
+ return;
+ }
+
+ logger.debug("Found event with id {}; searching for children", eventId);
+
+ switch (event.getEventType()) {
+ case CLONE:
+ case FORK:
+ case JOIN:
+ case REPLAY:
+ break;
+ default:
+ logger.warn("Cannot expand parents of event with ID {} because event type is {}", eventId, event.getEventType());
+ lineageSubmission.getResult().setError("Cannot expand parents of event with ID " + eventId +
+ " because that event is of type " + event.getEventType() +
+ ", and that type does not support expansion of children");
+ return;
+ }
+
+ final List<String> parentUuids = event.getParentUuids();
+ flowFileUuids.addAll(parentUuids);
+
+ final AtomicInteger retrievalCount = new AtomicInteger(100);
+ indexManager.withEachIndex(new VoidIndexAction() {
+ @Override
+ public void perform(EventIndexSearcher searcher) throws IOException {
+ final long startNanos = System.nanoTime();
+ logger.debug("Finding parents of event with id {} using {}", eventId, searcher);
+
+ final List<JournaledStorageLocation> locations = searcher.getEventsForFlowFiles(flowFileUuids, event.getLineageStartDate(), event.getEventTime());
+ final List<StoredProvenanceEvent> matchingRecords = getEvents(locations, retrievalCount);
+ lineageSubmission.getResult().update(matchingRecords);
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+ logger.debug("Found {} parents of event {} in {} millis", matchingRecords.size(), eventId, millis);
+ }
+ });
+ } catch (final IOException ioe) {
+
+ }
+ }
+ };
+
+ executor.submit(runnable);
+ lineageSubmissionMap.putIfAbsent(lineageSubmission.getLineageIdentifier(), lineageSubmission);
+ return lineageSubmission;
+ }
+
+
+ @Override
+ public void registerSubmission(final ProvenanceQuerySubmission submission) {
+ final ProvenanceQuerySubmission cancelable = new ProvenanceQuerySubmission() {
+ @Override
+ public String getQuery() { return submission.getQuery(); }
+
+ @Override
+ public ProvenanceQueryResult getResult() { return submission.getResult(); }
+
+ @Override
+ public Date getSubmissionTime() { return submission.getSubmissionTime(); }
+
+ @Override
+ public String getQueryIdentifier() { return submission.getQueryIdentifier(); }
+
+ @Override
+ public void cancel() {
+ provenanceQuerySubmissionMap.remove(submission.getQueryIdentifier());
+ submission.cancel();
+ }
+
+ @Override
+ public boolean isCanceled() { return submission.isCanceled(); }
+ };
+
+ provenanceQuerySubmissionMap.putIfAbsent(submission.getQueryIdentifier(), cancelable);
+ }
+
+ @Override
+ public ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(final String id) {
+ return provenanceQuerySubmissionMap.get(id);
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
new file mode 100644
index 0000000..7977620
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
@@ -0,0 +1,211 @@
+/*
+ * 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.provenance.journaling.tasks;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.provenance.journaling.journals.DeflatorCompressionCodec;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.JournalWriter;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalWriter;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
+import org.apache.nifi.provenance.journaling.toc.StandardTocWriter;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
+import org.apache.nifi.provenance.journaling.toc.TocWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Compresses a journal file and returns the new size of the journal
+ */
+public class CompressionTask implements Callable<Long> {
+ public static final String FILE_EXTENSION = ".compress";
+
+ private static final Logger logger = LoggerFactory.getLogger(CompressionTask.class);
+
+ private final File journalFile;
+ private final long journalId;
+ private final File tocFile;
+
+ public CompressionTask(final File journalFile, final long journalId, final File tocFile) {
+ this.journalFile = journalFile;
+ this.journalId = journalId;
+ this.tocFile = tocFile;
+ }
+
+ public void compress(final JournalReader reader, final JournalWriter writer, final TocReader tocReader, final TocWriter tocWriter) throws IOException {
+ ProvenanceEventRecord event;
+
+ int blockIndex = 0;
+ long blockOffset = tocReader.getBlockOffset(blockIndex);
+ tocWriter.addBlockOffset(blockOffset);
+ long nextBlockOffset = tocReader.getBlockOffset(blockIndex + 1);
+
+ // we write the events one at a time here so that we can ensure that when the block
+ // changes we are able to insert a new block into the TOC, as the blocks have to contain
+ // the same number of events, since the index just knows about the block index.
+ try {
+ while ((event = reader.nextEvent()) != null) {
+ // Check if we've gone beyond the offset of the next block. If so, write
+ // out a new block in the TOC.
+ final long newPosition = reader.getPosition();
+ if ( newPosition > nextBlockOffset && nextBlockOffset > 0 ) {
+ blockIndex++;
+ blockOffset = tocReader.getBlockOffset(blockIndex);
+ tocWriter.addBlockOffset(writer.getSize());
+
+ nextBlockOffset = tocReader.getBlockOffset(blockIndex + 1);
+ }
+
+ // Write the event to the compressed writer
+ writer.write(Collections.singleton(event), event.getEventId());
+ }
+ } catch (final EOFException eof) {
+ logger.warn("Found unexpected End-of-File when compressing {}", reader);
+ }
+ }
+
+ /**
+ * Attempts to delete the given file up to 10 times, waiting a bit in between each failed
+ * iteration, in case another process (for example, a virus scanner) has the file locked
+ *
+ * @param file
+ * @return
+ */
+ private boolean delete(final File file) {
+ for (int i=0; i < 10; i++) {
+ if ( file.delete() || !file.exists() ) {
+ return true;
+ }
+
+ try {
+ Thread.sleep(100L);
+ } catch (final InterruptedException ie) {
+ }
+ }
+
+ return !file.exists();
+ }
+
+ /**
+ * Attempts to rename the given original file to the renamed file up to 20 times, waiting a bit
+ * in between each failed iteration, in case another process (for example, a virus scanner) has
+ * the file locked
+ *
+ * @param original
+ * @param renamed
+ * @return
+ */
+ public static boolean rename(final File original, final File renamed) {
+ for (int i=0; i < 20; i++) {
+ if ( original.renameTo(renamed) ) {
+ return true;
+ }
+
+ try {
+ Thread.sleep(100L);
+ } catch (final InterruptedException ie) {
+ }
+ }
+
+ return false;
+ }
+
+ @Override
+ public Long call() {
+ final long startNanos = System.nanoTime();
+ final long preCompressionSize = journalFile.length();
+
+ try {
+ final File compressedFile = new File(journalFile.getParentFile(), journalFile.getName() + FILE_EXTENSION);
+ final File compressedTocFile = new File(tocFile.getParentFile(), tocFile.getName() + FILE_EXTENSION);
+
+ if ( compressedFile.exists() && !compressedFile.delete() ) {
+ logger.error("Compressed file {} already exists and could not remove it; compression task failed", compressedFile);
+ return preCompressionSize;
+ }
+
+ if ( compressedTocFile.exists() && !compressedTocFile.delete() ) {
+ logger.error("Compressed TOC file {} already exists and could not remove it; compression task failed", compressedTocFile);
+ return preCompressionSize;
+ }
+
+ try (final JournalReader journalReader = new StandardJournalReader(journalFile);
+ final JournalWriter compressedWriter = new StandardJournalWriter(journalId, compressedFile, new DeflatorCompressionCodec(), new StandardEventSerializer());
+ final TocReader tocReader = new StandardTocReader(tocFile);
+ final TocWriter compressedTocWriter = new StandardTocWriter(compressedTocFile, true, false)) {
+
+ compress(journalReader, compressedWriter, tocReader, compressedTocWriter);
+ compressedWriter.sync();
+ } catch (final FileNotFoundException fnfe) {
+ logger.info("Failed to compress Journal File {} because it has already been removed", journalFile);
+ return 0L;
+ }
+
+ final long postCompressionSize = compressedFile.length();
+
+ final boolean deletedJournal = delete(journalFile);
+ if ( !deletedJournal ) {
+ delete(compressedFile);
+ delete(compressedTocFile);
+ logger.error("Failed to remove Journal file {}; considering compression task a failure", journalFile);
+ return preCompressionSize;
+ }
+
+ final boolean deletedToc = delete(tocFile);
+ if ( !deletedToc ) {
+ delete(compressedFile);
+ delete(compressedTocFile);
+ logger.error("Failed to remove TOC file for {}; considering compression task a failure", journalFile);
+ return preCompressionSize;
+ }
+
+ final boolean renamedJournal = rename(compressedFile, journalFile);
+ if ( !renamedJournal ) {
+ logger.error("Failed to rename {} to {}; this journal file may be inaccessible until it is renamed", compressedFile, journalFile);
+ }
+
+ final boolean renamedToc = rename(compressedTocFile, tocFile);
+ if ( !renamedToc ) {
+ logger.error("Failed to rename {} to {}; this journal file may be inaccessible until it is renamed", compressedTocFile, tocFile);
+ }
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+ final double percent = (postCompressionSize / preCompressionSize) * 100D;
+ final String pct = String.format("%.2f", percent);
+ logger.info("Successfully compressed Journal File {} in {} millis; size changed from {} bytes to {} bytes ({}% of original size)", journalFile, millis, preCompressionSize, postCompressionSize, pct);
+ return postCompressionSize;
+ } catch (final IOException ioe) {
+ logger.error("Failed to compress Journal File {} due to {}", journalFile, ioe.toString());
+ if ( logger.isDebugEnabled() ) {
+ logger.error("", ioe);
+ }
+
+ return preCompressionSize;
+ }
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
new file mode 100644
index 0000000..ae4635f
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
@@ -0,0 +1,97 @@
+/*
+ * 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.provenance.journaling.toc;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+/**
+ * Standard implementation of TocReader.
+ *
+ * Expects .toc file to be in the following format;
+ *
+ * byte 0: version
+ * byte 1: boolean: compressionFlag -> 0 = journal is NOT compressed, 1 = journal is compressed
+ * byte 2-9: long: offset of block 0
+ * byte 10-17: long: offset of block 1
+ * ...
+ * byte (N*8+2)-(N*8+9): long: offset of block N
+ */
+public class StandardTocReader implements TocReader {
+ private final boolean compressed;
+ private final long[] offsets;
+
+ public StandardTocReader(final File file) throws IOException {
+ try (final FileInputStream fis = new FileInputStream(file);
+ final DataInputStream dis = new DataInputStream(fis)) {
+
+ final int version = dis.read();
+ if ( version < 0 ) {
+ throw new EOFException();
+ }
+
+ final int compressionFlag = dis.read();
+ if ( compressionFlag < 0 ) {
+ throw new EOFException();
+ }
+
+ if ( compressionFlag == 0 ) {
+ compressed = false;
+ } else if ( compressionFlag == 1 ) {
+ compressed = true;
+ } else {
+ throw new IOException("Table of Contents appears to be corrupt: could not read 'compression flag' from header; expected value of 0 or 1 but got " + compressionFlag);
+ }
+
+ final int numBlocks = (int) ((file.length() - 2) / 8);
+ offsets = new long[numBlocks];
+
+ for (int i=0; i < numBlocks; i++) {
+ offsets[i] = dis.readLong();
+ }
+ }
+ }
+
+ @Override
+ public boolean isCompressed() {
+ return compressed;
+ }
+
+ @Override
+ public long getBlockOffset(final int blockIndex) {
+ if ( blockIndex >= offsets.length ) {
+ return -1L;
+ }
+ return offsets[blockIndex];
+ }
+
+ @Override
+ public long getLastBlockOffset() {
+ if ( offsets.length == 0 ) {
+ return 0L;
+ }
+ return offsets[offsets.length - 1];
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
new file mode 100644
index 0000000..9ee07e0
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
@@ -0,0 +1,133 @@
+/*
+ * 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.provenance.journaling.toc;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.FileAlreadyExistsException;
+
+/**
+ * Standard implementation of {@link TocWriter}.
+ *
+ * Format of .toc file:
+ * byte 0: version
+ * byte 1: compressed: 0 -> not compressed, 1 -> compressed
+ * byte 2-9: long: offset of block 0
+ * byte 10-17: long: offset of block 1
+ * ...
+ * byte (N*8+2)-(N*8+9): long: offset of block N
+ */
+public class StandardTocWriter implements TocWriter {
+ public static final byte VERSION = 1;
+
+ private final File file;
+ private final FileOutputStream fos;
+ private final boolean alwaysSync;
+ private int index = 0;
+
+ /**
+ * Creates a StandardTocWriter that writes to the given file.
+ * @param file the file to write to
+ * @param compressionFlag whether or not the journal is compressed
+ * @throws FileNotFoundException
+ */
+ public StandardTocWriter(final File file, final boolean compressionFlag, final boolean alwaysSync) throws IOException {
+ if ( file.exists() ) {
+ // Check if the header actually exists. If so, throw FileAlreadyExistsException
+ // If no data is in the file, we will just overwrite it.
+ try (final InputStream fis = new FileInputStream(file);
+ final InputStream bis = new BufferedInputStream(fis);
+ final DataInputStream dis = new DataInputStream(bis)) {
+ dis.read();
+ dis.read();
+
+ // we always add the first offset when the writer is created so we allow this to exist.
+ dis.readLong();
+ final int nextByte = dis.read();
+
+ if ( nextByte > -1 ) {
+ throw new FileAlreadyExistsException(file.getAbsolutePath());
+ }
+ } catch (final EOFException eof) {
+ // no real data. overwrite file.
+ }
+ }
+
+ if ( !file.getParentFile().exists() && !file.getParentFile().mkdirs() ) {
+ throw new IOException("Could not create directory " + file.getParent());
+ }
+
+ this.file = file;
+ fos = new FileOutputStream(file);
+ this.alwaysSync = alwaysSync;
+
+ final byte[] header = new byte[2];
+ header[0] = VERSION;
+ header[1] = (byte) (compressionFlag ? 1 : 0);
+ fos.write(header);
+ fos.flush();
+
+ if ( alwaysSync ) {
+ fos.getFD().sync();
+ }
+ }
+
+ @Override
+ public void addBlockOffset(final long offset) throws IOException {
+ final BufferedOutputStream bos = new BufferedOutputStream(fos);
+ final DataOutputStream dos = new DataOutputStream(bos);
+ dos.writeLong(offset);
+ dos.flush();
+
+ if ( alwaysSync ) {
+ fos.getFD().sync();
+ }
+ }
+
+ @Override
+ public int getCurrentBlockIndex() {
+ return index;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (alwaysSync) {
+ fos.getFD().sync();
+ }
+
+ fos.close();
+ }
+
+ @Override
+ public File getFile() {
+ return file;
+ }
+
+ @Override
+ public String toString() {
+ return "TOC Writer for " + file;
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
new file mode 100644
index 0000000..c3f9df5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
@@ -0,0 +1,98 @@
+/*
+ * 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.provenance.journaling.toc;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+
+public class TocJournalReader implements Closeable {
+
+ private final TocReader tocReader;
+ private final JournalReader reader;
+
+ private final String containerName;
+ private final String sectionName;
+ private final Long journalId;
+
+ private int blockIndex;
+ private long nextBlockOffset;
+
+
+ public TocJournalReader(final String containerName, final String sectionName, final Long journalId, final File journalFile) throws IOException {
+ this.containerName = containerName;
+ this.sectionName = sectionName;
+ this.journalId = journalId;
+
+ final File tocFile = new File(journalFile.getParentFile(), journalFile.getName() + ".toc");
+ tocReader = new StandardTocReader(tocFile);
+ reader = new StandardJournalReader(journalFile);
+
+ blockIndex = 0;
+ nextBlockOffset = tocReader.getBlockOffset(1);
+ }
+
+ @Override
+ public void close() throws IOException {
+ IOException suppressed = null;
+ try {
+ tocReader.close();
+ } catch (final IOException ioe) {
+ suppressed = ioe;
+ }
+
+ try {
+ reader.close();
+ } catch (final IOException ioe) {
+ if ( suppressed != null ) {
+ ioe.addSuppressed(suppressed);
+ }
+ throw ioe;
+ }
+
+ if ( suppressed != null ) {
+ throw suppressed;
+ }
+ }
+
+ public JournaledProvenanceEvent nextJournaledEvent() throws IOException {
+ ProvenanceEventRecord event = reader.nextEvent();
+ if ( event == null ) {
+ return null;
+ }
+
+ final JournaledStorageLocation location = new JournaledStorageLocation(containerName, sectionName,
+ journalId, blockIndex, event.getEventId());
+
+ // Check if we've gone beyond the offset of the next block. If so, write
+ // out a new block in the TOC.
+ final long newPosition = reader.getPosition();
+ if ( newPosition > nextBlockOffset && nextBlockOffset > 0 ) {
+ blockIndex++;
+ nextBlockOffset = tocReader.getBlockOffset(blockIndex + 1);
+ }
+
+ return new JournaledProvenanceEvent(event, location);
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
new file mode 100644
index 0000000..18d7189
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
@@ -0,0 +1,51 @@
+/*
+ * 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.provenance.journaling.toc;
+
+import java.io.Closeable;
+
+/**
+ * <p>
+ * Reads a Table of Contents (.toc file) for a corresponding Journal File. We use a Table of Contents
+ * to map a Block Index to an offset into the Journal file where that Block begins. We do this so that
+ * we can then persist a Block Index for an event and then compress the Journal later. This way, we can
+ * get good compression by compressing a large batch of events at once, and this way we can also look up
+ * an event in a Journal that has not been compressed by looking in the Table of Contents or lookup the
+ * event in a Journal post-compression by simply rewriting the TOC while we compress the data.
+ * </p>
+ */
+public interface TocReader extends Closeable {
+
+ /**
+ * Indicates whether or not the corresponding Journal file is compressed
+ * @return
+ */
+ boolean isCompressed();
+
+ /**
+ * Returns the byte offset into the Journal File for the Block with the given index.
+ * @param blockIndex
+ * @return
+ */
+ long getBlockOffset(int blockIndex);
+
+ /**
+ * Returns the byte offset into the Journal File of the last Block in the given index
+ * @return
+ */
+ long getLastBlockOffset();
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocWriter.java
new file mode 100644
index 0000000..b44b55b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocWriter.java
@@ -0,0 +1,46 @@
+/*
+ * 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.provenance.journaling.toc;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Writes a .toc file
+ */
+public interface TocWriter extends Closeable {
+
+ /**
+ * Adds the given block offset as the next Block Offset in the Table of Contents
+ * @param offset
+ * @throws IOException
+ */
+ void addBlockOffset(long offset) throws IOException;
+
+ /**
+ * Returns the index of the current Block
+ * @return
+ */
+ int getCurrentBlockIndex();
+
+ /**
+ * Returns the file that is currently being written to
+ * @return
+ */
+ File getFile();
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceEventRepository b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceEventRepository
new file mode 100644
index 0000000..e224c51
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceEventRepository
@@ -0,0 +1,15 @@
+# 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.
+org.apache.nifi.provenance.journaling.JournalingProvenanceRepository
\ No newline at end of file
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
new file mode 100644
index 0000000..89af0aa
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
@@ -0,0 +1,837 @@
+/*
+ * 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.provenance.journaling;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.lineage.ComputeLineageResult;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.lineage.LineageNode;
+import org.apache.nifi.provenance.lineage.LineageNodeType;
+import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QueryResult;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.search.SearchTerms;
+import org.apache.nifi.provenance.search.SearchableField;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestJournalingProvenanceRepository {
+
+
+ @BeforeClass
+ public static void setupLogging() {
+ System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance.journaling", "DEBUG");
+ }
+
+ @Test
+ public void testStoreAndRetrieve() throws IOException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+ config.setPartitionCount(3);
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+ final Map<String, String> attributes = new HashMap<>();
+
+ for (int i=0; i < 10; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+
+ // retrieve records one at a time.
+ for (int i=0; i < 10; i++) {
+ final StoredProvenanceEvent event = repo.getEvent(i);
+ assertNotNull(event);
+ assertEquals((long) i, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+ }
+
+ final List<StoredProvenanceEvent> events = repo.getEvents(0, 1000);
+ assertNotNull(events);
+ assertEquals(10, events.size());
+ for (int i=0; i < 10; i++) {
+ final StoredProvenanceEvent event = events.get(i);
+ assertNotNull(event);
+ assertEquals((long) i, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+ }
+ } finally {
+ for ( final File file : containers.values() ) {
+ if ( file.exists() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+ }
+
+
+ @Test
+ public void testStoreRestartAndRetrieve() throws IOException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+ config.setPartitionCount(3);
+
+ try {
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+ final Map<String, String> attributes = new HashMap<>();
+
+ for (int i=0; i < 10; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+
+ assertEquals(10L, repo.getMaxEventId().longValue());
+ }
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ assertEquals(10L, repo.getMaxEventId().longValue());
+
+ // retrieve records one at a time.
+ for (int i=0; i < 10; i++) {
+ final StoredProvenanceEvent event = repo.getEvent(i);
+ assertNotNull(event);
+ assertEquals((long) i, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+ }
+
+ final List<StoredProvenanceEvent> events = repo.getEvents(0, 1000);
+ assertNotNull(events);
+ assertEquals(10, events.size());
+ for (int i=0; i < 10; i++) {
+ final StoredProvenanceEvent event = events.get(i);
+ assertNotNull(event);
+ assertEquals((long) i, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+ }
+ }
+ } finally {
+ for ( final File file : containers.values() ) {
+ if ( file.exists() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+ }
+
+
+ @Test
+ public void testStoreRestartRetrieveAndExpireOnTime() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+ config.setPartitionCount(3);
+
+ try {
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+ final Map<String, String> attributes = new HashMap<>();
+
+ for (int i=0; i < 10; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+
+ assertEquals(10L, repo.getMaxEventId().longValue());
+ }
+
+ config.setExpirationFrequency(1, TimeUnit.SECONDS);
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ assertEquals(10L, repo.getMaxEventId().longValue());
+
+ // retrieve records one at a time.
+ for (int i=0; i < 10; i++) {
+ final StoredProvenanceEvent event = repo.getEvent(i);
+ assertNotNull(event);
+ assertEquals((long) i, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+ }
+
+ final List<StoredProvenanceEvent> events = repo.getEvents(0, 1000);
+ assertNotNull(events);
+ assertEquals(10, events.size());
+ for (int i=0; i < 10; i++) {
+ final StoredProvenanceEvent event = events.get(i);
+ assertNotNull(event);
+ assertEquals((long) i, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+ }
+
+ // wait a bit for the events to be expired
+ TimeUnit.SECONDS.sleep(2L);
+
+ // retrieve records one at a time.
+ for (int i=0; i < 10; i++) {
+ final StoredProvenanceEvent event = repo.getEvent(i);
+ assertNull("Event " + i + " still exists", event);
+ }
+
+ final List<StoredProvenanceEvent> allEvents = repo.getEvents(0, 1000);
+ assertNotNull(allEvents);
+ assertEquals(0, allEvents.size());
+ }
+ } finally {
+ for ( final File file : containers.values() ) {
+ if ( file.exists() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+ }
+
+
+ @Test
+ public void testExpireOnSize() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+ config.setPartitionCount(3);
+ config.setMaxStorageCapacity(1024L * 50);
+ config.setEventExpiration(2, TimeUnit.SECONDS);
+ config.setExpirationFrequency(1, TimeUnit.SECONDS);
+ config.setJournalRolloverPeriod(1, TimeUnit.SECONDS);
+ config.setCompressOnRollover(false);
+
+ try {
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+ final Map<String, String> attributes = new HashMap<>();
+
+ final int numEventsToInsert = 1000;
+ for (int i=0; i < numEventsToInsert; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+
+ final List<StoredProvenanceEvent> eventsBeforeExpire = repo.getEvents(0, numEventsToInsert * 2);
+ assertNotNull(eventsBeforeExpire);
+ assertEquals(numEventsToInsert, eventsBeforeExpire.size());
+
+ // wait a bit for expiration to occur
+ TimeUnit.SECONDS.sleep(3L);
+
+ // generate an event for each partition to force a rollover of the journals
+ for (int i=0; i < config.getPartitionCount(); i++) {
+ repo.registerEvent(TestUtil.generateEvent(100000L));
+ }
+
+ TimeUnit.SECONDS.sleep(1L);
+
+ // retrieve records one at a time.
+ for (int i=0; i < numEventsToInsert; i++) {
+ final StoredProvenanceEvent event = repo.getEvent(i);
+ assertNull("Event " + i + " still exists", event);
+ }
+
+ final List<StoredProvenanceEvent> eventsAfterExpire = repo.getEvents(0, numEventsToInsert * 2);
+ assertNotNull(eventsAfterExpire);
+ assertEquals(3, eventsAfterExpire.size());
+ }
+ } finally {
+ for ( final File file : containers.values() ) {
+ if ( file.exists() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+ }
+
+
+
+ @Test(timeout=10000)
+ public void testSearchByUUID() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(3);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ for (int i=0; i < 10; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+
+ final Query query = new Query("query");
+ query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.FlowFileUUID, "00000000-0000-0000-0000-000000000005"));
+ final QuerySubmission submission = repo.submitQuery(query);
+ assertNotNull(submission);
+
+ final QueryResult result = submission.getResult();
+ while ( !result.isFinished() ) {
+ Thread.sleep(50L);
+ }
+
+ assertNull(result.getError());
+ final List<StoredProvenanceEvent> matches = result.getMatchingEvents();
+ assertNotNull(matches);
+ assertEquals(1, matches.size());
+
+ final StoredProvenanceEvent event = matches.get(0);
+ assertEquals(5, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-000000000005", event.getFlowFileUuid());
+ System.out.println("Query time was " + result.getQueryTime() + " millis");
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+ @Test()
+ public void testSearchByUUIDTextQuery() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(1);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID,
+ SearchableFields.EventTime
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ final long start = System.nanoTime();
+ for (int i=0; i < 10; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+ final long registerFinish = System.nanoTime();
+
+ // Ensure that we get the Events, even though we are querying for Event.Time because selectMatchingEvents
+ // only takes into account the WHERE clause
+ final Set<String> fieldsToLoad = new HashSet<>();
+ fieldsToLoad.add(SearchableFields.EventTime.getSearchableFieldName());
+
+ final Iterator<? extends StoredProvenanceEvent> itr = repo.selectMatchingEvents("SELECT Event WHERE Event.uuid = '00000000-0000-0000-0000-000000000005'", fieldsToLoad, new AtomicLong(0L));
+ assertTrue(itr.hasNext());
+ final StoredProvenanceEvent event = itr.next();
+ assertNotNull(event);
+ assertFalse(itr.hasNext());
+
+ final long searchFinish = System.nanoTime();
+ assertEquals(5, event.getEventId());
+ assertEquals("00000000-0000-0000-0000-000000000005", event.getFlowFileUuid());
+
+ System.out.println("Register records: " + TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+ System.out.println("Query records: " + TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+ } catch (final Exception e) {
+ e.printStackTrace();
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+
+ @Test()
+ public void testSearchFieldTextQuery() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(3);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ final long start = System.nanoTime();
+ for (int i=0; i < 10; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+ final long registerFinish = System.nanoTime();
+
+ final ProvenanceQuerySubmission submission = repo.submitQuery("SELECT Event.uuid WHERE Event.uuid = '00000000-0000-0000-0000-000000000005'");
+ while ( !submission.getResult().isFinished() ) {
+ Thread.sleep(100L);
+ System.out.println(submission.getResult().getPercentComplete() + "% complete");
+ }
+
+ final ProvenanceResultSet rs = repo.query("SELECT Event.uuid WHERE Event.uuid = '00000000-0000-0000-0000-000000000005'");
+ assertTrue(rs.hasNext());
+ final List<?> cols = rs.next();
+ assertEquals(1, cols.size());
+ assertEquals(String.class, rs.getReturnType().get(0));
+ final Object firstCol = cols.get(0);
+ assertEquals("00000000-0000-0000-0000-000000000005", firstCol);
+ assertFalse(rs.hasNext());
+
+ final long searchFinish = System.nanoTime();
+ System.out.println("Register records: " + TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+ System.out.println("Query records: " + TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+ } catch (final Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.toString());
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+ @Test()
+ public void testSearchEventTextQuery() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(3);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ final long start = System.nanoTime();
+ for (int i=0; i < 10; i++) {
+ attributes.put("i", String.valueOf(i));
+ repo.registerEvent(TestUtil.generateEvent(i, attributes));
+ }
+ final long registerFinish = System.nanoTime();
+
+ final ProvenanceResultSet rs = repo.query("SELECT Event WHERE Event.uuid = '00000000-0000-0000-0000-000000000005'");
+ assertTrue(rs.hasNext());
+ final List<?> cols = rs.next();
+ assertEquals(1, cols.size());
+ assertTrue(ProvenanceEventRecord.class.isAssignableFrom(rs.getReturnType().get(0)));
+ final ProvenanceEventRecord firstCol = (ProvenanceEventRecord) cols.get(0);
+ assertEquals("00000000-0000-0000-0000-000000000005", firstCol.getFlowFileUuid());
+ assertEquals("5", firstCol.getAttribute("i"));
+ assertEquals("5", firstCol.getAttributes().get("i"));
+ assertEquals("5", firstCol.getUpdatedAttributes().get("i"));
+ assertFalse(rs.hasNext());
+
+ final long searchFinish = System.nanoTime();
+ System.out.println("Register records: " + TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+ System.out.println("Query records: " + TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+
+ @Test()
+ public void testSearchEventTextQueryAgainstMany() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+ config.setCompressOnRollover(false);
+
+ config.setPartitionCount(16);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID,
+ SearchableFields.FileSize,
+ SearchableFields.AlternateIdentifierURI,
+ SearchableFields.ComponentID,
+ SearchableFields.EventTime,
+ SearchableFields.EventType,
+ SearchableFields.Details,
+ SearchableFields.TransitURI
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ final long start = System.nanoTime();
+ final List<ProvenanceEventRecord> events = new ArrayList<>(1000);
+ for (int i=0; i < 1000000; i++) {
+ attributes.put("i", String.valueOf(i));
+ final ProvenanceEventRecord event = TestUtil.generateEvent(i, attributes);
+ events.add(event);
+ if ( events.size() % 1000 == 0 ) {
+ repo.registerEvents(events);
+ events.clear();
+ }
+ }
+ final long registerFinish = System.nanoTime();
+
+ System.out.println("Inserted 1 million events. Waiting for profiler...");
+// try { Thread.sleep(25000L); } catch (final InterruptedException ie) {}
+ System.out.println("Running query");
+
+ final long searchStart = System.nanoTime();
+ for (int i=0; i < 1; i++) {
+ final ProvenanceResultSet rs = repo.query("SELECT SUM(Event.Size)");
+ assertTrue(rs.hasNext());
+ final List<?> cols = rs.next();
+ assertEquals(1, cols.size());
+// assertTrue(ProvenanceEventRecord.class.isAssignableFrom(rs.getReturnType().get(0)));
+// final ProvenanceEventRecord firstCol = (ProvenanceEventRecord) cols.get(0);
+// assertEquals("00000000-0000-0000-0000-000000000005", firstCol.getFlowFileUuid());
+ assertFalse(rs.hasNext());
+ }
+
+ final long searchFinish = System.nanoTime();
+ System.out.println("Register records: " + TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+ System.out.println("Query records: " + TimeUnit.NANOSECONDS.toMillis(searchFinish - searchStart) + " millis");
+ } catch (final Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.toString());
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+
+ @Test()
+ public void testSearchManyEventsTextQueryAgainstMany() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(3);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ final long start = System.nanoTime();
+ final List<ProvenanceEventRecord> events = new ArrayList<>(1000);
+ for (int i=0; i < 100000; i++) {
+ attributes.put("i", String.valueOf(i));
+ final ProvenanceEventRecord event = TestUtil.generateEvent(i, attributes);
+ events.add(event);
+ if ( events.size() % 1000 == 0 ) {
+ repo.registerEvents(events);
+ events.clear();
+ }
+ }
+ final long registerFinish = System.nanoTime();
+
+ final ProvenanceResultSet rs = repo.query("SELECT Event");
+ for (int i=0; i < 100000; i++) {
+ assertTrue(rs.hasNext());
+ final List<?> cols = rs.next();
+ assertEquals(1, cols.size());
+ }
+ assertFalse(rs.hasNext());
+
+ final long searchFinish = System.nanoTime();
+ System.out.println("Register records: " + TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+ System.out.println("Query records: " + TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+ } catch (final Exception e) {
+ e.printStackTrace();
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+
+ @Test()
+ public void testAggregateQueryAgainstMany() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(3);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID,
+ SearchableFields.FileSize
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ final long start = System.nanoTime();
+ final List<ProvenanceEventRecord> events = new ArrayList<>(1000);
+ for (int i=0; i < 100000; i++) {
+ attributes.put("i", String.valueOf(i));
+ final ProvenanceEventRecord event = TestUtil.generateEvent(i, attributes);
+ events.add(event);
+ if ( events.size() % 1000 == 0 ) {
+ repo.registerEvents(events);
+ events.clear();
+ }
+ }
+ final long registerFinish = System.nanoTime();
+
+ final ProvenanceResultSet rs = repo.query("SELECT SUM(Event.Size), AVG(Event.Size)");
+ assertTrue(rs.hasNext());
+ assertEquals(Long.class, rs.getReturnType().get(0));
+ assertEquals(Double.class, rs.getReturnType().get(1));
+ final List<?> cols = rs.next();
+ assertEquals(100000L, cols.get(0));
+ assertEquals(1D, cols.get(1));
+ assertFalse(rs.hasNext());
+
+ final long searchFinish = System.nanoTime();
+ System.out.println("Register records: " + TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+ System.out.println("Query records: " + TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+ } catch (final Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.toString());
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+
+
+ @Test()
+ public void testAggregateQuery2AgainstMany() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ config.setEventExpiration(10, TimeUnit.MINUTES);
+ config.setJournalRolloverPeriod(10, TimeUnit.MINUTES);
+ config.setJournalCapacity(1024 * 1024 * 1024);
+
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(1);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID
+ }));
+ config.setSearchableAttributes(Arrays.asList(new SearchableField[] {
+ SearchableFields.newSearchableAttribute("i"),
+ SearchableFields.newSearchableAttribute("j")
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final Map<String, String> attributes = new HashMap<>();
+
+ final long start = System.nanoTime();
+ final List<ProvenanceEventRecord> events = new ArrayList<>(1000);
+ for (int j=0; j < 100; j++) {
+ attributes.put("j", String.valueOf(j));
+ for (int i=0; i < 100; i++) {
+ attributes.put("i", String.valueOf(i));
+ final ProvenanceEventRecord event = TestUtil.generateEvent(i, attributes);
+ events.add(event);
+ if ( events.size() % 100 == 0 ) {
+ repo.registerEvents(events);
+ events.clear();
+ }
+ }
+ }
+ final long registerFinish = System.nanoTime();
+
+ final ProvenanceResultSet rs = repo.query("SELECT Event['j'], COUNT(Event['i']), SUM(Event['i']) GROUP BY Event['j']");
+ System.out.println(rs.getLabels());
+ while ( rs.hasNext() ) {
+ final List<?> cols = rs.next();
+ System.out.println(cols);
+ }
+
+ final long searchFinish = System.nanoTime();
+ System.out.println("Register records: " + TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+ System.out.println("Query records: " + TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+
+ } catch (final Exception e) {
+ e.printStackTrace();
+ Assert.fail(e.toString());
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+
+
+
+ @Test(timeout=10000)
+ public void testReceiveDropLineage() throws IOException, InterruptedException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+ containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+ config.setContainers(containers);
+
+ config.setPartitionCount(3);
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID
+ }));
+
+ try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+ repo.initialize(null);
+
+ final String uuid = "00000000-0000-0000-0000-000000000001";
+ final Map<String, String> attributes = new HashMap<>();
+ attributes.put("abc", "xyz");
+ attributes.put("uuid", uuid);
+ attributes.put("filename", "file-" + uuid);
+
+ final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder()
+ .setEventType(ProvenanceEventType.RECEIVE)
+ .setFlowFileUUID(uuid)
+ .setComponentType("Unit Test")
+ .setComponentId(UUID.randomUUID().toString())
+ .setEventTime(System.currentTimeMillis())
+ .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
+ .setLineageStartDate(System.currentTimeMillis() - 2000L)
+ .setCurrentContentClaim(null, null, null, null, 0L)
+ .setAttributes(null, attributes == null ? Collections.<String, String>emptyMap() : attributes);
+
+ builder.setTransitUri("nifi://unit-test");
+ attributes.put("uuid", uuid);
+ builder.setComponentId("1234");
+ builder.setComponentType("dummy processor");
+
+ // Add RECEIVE Event
+ repo.registerEvent(builder.build());
+
+ builder.setEventTime(System.currentTimeMillis() + 1);
+ builder.setEventType(ProvenanceEventType.DROP);
+ builder.setTransitUri(null);
+
+ // Add DROP event
+ repo.registerEvent(builder.build());
+
+ // register unrelated even to make sure we don't get this one.
+ builder.setFlowFileUUID("00000000-0000-0000-0000-000000000002");
+ repo.registerEvent(builder.build());
+
+ final ComputeLineageSubmission submission = repo.submitLineageComputation(uuid);
+ assertNotNull(submission);
+
+ final ComputeLineageResult result = submission.getResult();
+ while ( !result.isFinished() ) {
+ Thread.sleep(50L);
+ }
+
+ assertNull(result.getError());
+
+ final List<LineageNode> nodes = result.getNodes();
+ assertEquals(3, nodes.size()); // RECEIVE, FlowFile node, DROP
+
+ int receiveCount = 0;
+ int dropCount = 0;
+ int flowFileNodeCount = 0;
+ for ( final LineageNode node : nodes ) {
+ assertEquals(uuid, node.getFlowFileUuid());
+
+ if ( LineageNodeType.PROVENANCE_EVENT_NODE.equals(node.getNodeType()) ) {
+ final ProvenanceEventLineageNode eventNode = (ProvenanceEventLineageNode) node;
+ if ( eventNode.getEventType() == ProvenanceEventType.RECEIVE ) {
+ receiveCount++;
+ } else if ( eventNode.getEventType() == ProvenanceEventType.DROP ) {
+ dropCount++;
+ }
+ } else {
+ flowFileNodeCount++;
+ }
+ }
+
+ assertEquals(1, receiveCount);
+ assertEquals(1, dropCount);
+ assertEquals(1, flowFileNodeCount);
+ } finally {
+ for ( final File file : containers.values() ) {
+ FileUtils.deleteFile(file, true);
+ }
+ }
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
new file mode 100644
index 0000000..88201ae
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
@@ -0,0 +1,63 @@
+/*
+ * 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.provenance.journaling;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+
+public class TestUtil {
+
+ public static ProvenanceEventRecord generateEvent(final long id) {
+ return generateEvent(id, Collections.<String, String>emptyMap());
+ }
+
+ public static ProvenanceEventRecord generateEvent(final long id, final Map<String, String> attributes) {
+ // Create prov event to add to the stream
+ final ProvenanceEventRecord event = new StandardProvenanceEventRecord.Builder()
+ .setEventType(ProvenanceEventType.CREATE)
+ .setFlowFileUUID("00000000-0000-0000-0000-" + pad(String.valueOf(id), 12, '0'))
+ .setComponentType("Unit Test")
+ .setComponentId(UUID.randomUUID().toString())
+ .setEventTime(System.currentTimeMillis())
+ .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
+ .setLineageStartDate(System.currentTimeMillis() - 2000L)
+ .setCurrentContentClaim(null, null, null, null, 1L)
+ .setAttributes(null, attributes == null ? Collections.<String, String>emptyMap() : attributes)
+ .build();
+
+ return event;
+ }
+
+ public static String pad(final String value, final int charCount, final char padding) {
+ if ( value.length() >= charCount ) {
+ return value;
+ }
+
+ final StringBuilder sb = new StringBuilder();
+ for (int i=value.length(); i < charCount; i++) {
+ sb.append(padding);
+ }
+ sb.append(value);
+
+ return sb.toString();
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
new file mode 100644
index 0000000..874727d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
@@ -0,0 +1,84 @@
+/*
+ * 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.provenance.journaling.index;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.TestUtil;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.SearchTerms;
+import org.apache.nifi.provenance.search.SearchableField;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.Test;
+
+public class TestEventIndexWriter {
+
+ @Test
+ public void testIndexAndFetch() throws IOException {
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ config.setSearchableAttributes(Arrays.asList(new SearchableField[] {
+ SearchableFields.newSearchableAttribute("test.1")
+ }));
+ config.setSearchableFields(Arrays.asList(new SearchableField[] {
+ SearchableFields.FlowFileUUID
+ }));
+
+ final File indexDir = new File("target/" + UUID.randomUUID().toString());
+
+ try (final LuceneIndexWriter indexWriter = new LuceneIndexWriter(null, indexDir, config)) {
+ final ProvenanceEventRecord event = TestUtil.generateEvent(23L);
+ final JournaledStorageLocation location = new JournaledStorageLocation("container", "section", 1L, 2, 23L);
+ final JournaledProvenanceEvent storedEvent = new JournaledProvenanceEvent(event, location);
+ indexWriter.index(Collections.singleton(storedEvent));
+
+ final Query query = new Query("123");
+ query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.FlowFileUUID, "00000000-0000-0000-0000-000000000023"));
+
+ try (final EventIndexSearcher searcher = indexWriter.newIndexSearcher()) {
+ final SearchResult searchResult = searcher.search(query);
+ final List<JournaledStorageLocation> locations = searchResult.getLocations();
+ assertNotNull(locations);
+ assertEquals(1, locations.size());
+
+ final JournaledStorageLocation found = locations.get(0);
+ assertNotNull(found);
+ assertEquals("container", found.getContainerName());
+ assertEquals("section", found.getSectionName());
+ assertEquals(1L, found.getJournalId().longValue());
+ assertEquals(2, found.getBlockIndex());
+ assertEquals(23L, found.getEventId());
+ }
+ } finally {
+ FileUtils.deleteFile(indexDir, true);
+ }
+
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestJournalReadWrite.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestJournalReadWrite.java
new file mode 100644
index 0000000..89eace7
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestJournalReadWrite.java
@@ -0,0 +1,83 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.TestUtil;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestJournalReadWrite {
+
+ @Test
+ public void testReadWrite100Blocks() throws IOException {
+ testReadWrite100Blocks(true);
+ testReadWrite100Blocks(false);
+ }
+
+ private void testReadWrite100Blocks(final boolean compressed) throws IOException {
+ final long journalId = 1L;
+ final File journalFile = new File("target/1.journal");
+ final StandardEventSerializer serializer = new StandardEventSerializer();
+
+ try {
+ final CompressionCodec codec = compressed ? new DeflatorCompressionCodec() : null;
+ try (final StandardJournalWriter writer = new StandardJournalWriter(journalId, journalFile, codec, serializer)) {
+ for (int block=0; block < 100; block++) {
+ writer.beginNewBlock();
+
+ for (int i=0; i < 5; i++) {
+ final ProvenanceEventRecord event = TestUtil.generateEvent(i);
+ writer.write(Collections.singleton(event), i);
+ }
+
+ final List<ProvenanceEventRecord> events = new ArrayList<>();
+ for (int i=0; i < 90; i++) {
+ events.add(TestUtil.generateEvent(i + 5));
+ }
+ writer.write(events, 5);
+
+ for (int i=0; i < 5; i++) {
+ final ProvenanceEventRecord event = TestUtil.generateEvent(i);
+ writer.write(Collections.singleton(event), 95 + i);
+ }
+
+ writer.finishBlock();
+ }
+ }
+
+ try (final StandardJournalReader reader = new StandardJournalReader(journalFile)) {
+ for (int block=0; block < 100; block++) {
+ for (int i=0; i < 100; i++) {
+ final ProvenanceEventRecord record = reader.nextEvent();
+ Assert.assertNotNull(record);
+ Assert.assertEquals((long) i, record.getEventId());
+ }
+ }
+ }
+ } finally {
+ journalFile.delete();
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalReader.java
new file mode 100644
index 0000000..f29af1b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalReader.java
@@ -0,0 +1,528 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.remote.io.CompressionOutputStream;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestStandardJournalReader {
+
+ private ByteArrayOutputStream baos;
+ private DataOutputStream dos;
+
+ @Before
+ public void setup() throws IOException {
+ // Create a BAOS to write the record to.
+ baos = new ByteArrayOutputStream();
+ dos = new DataOutputStream(baos);
+
+ // Write out header: codec name and serialization version
+ StandardJournalMagicHeader.write(dos);
+ dos.writeUTF(StandardEventSerializer.CODEC_NAME);
+ dos.writeInt(0);
+ }
+
+
+ @Test
+ public void testReadFirstEventUncompressed() throws IOException {
+ dos.writeBoolean(false);
+ writeRecord(88L);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ final ProvenanceEventRecord restored = reader.nextEvent();
+ assertNotNull(restored);
+ assertEquals(88L, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ } finally {
+ file.delete();
+ }
+ }
+
+
+ @Test
+ public void testReadManyUncompressed() throws IOException {
+ dos.writeBoolean(false);
+ writeRecords(0, 1024, false);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ for (int i=0; i < 1024; i++) {
+ final ProvenanceEventRecord restored = reader.nextEvent();
+ assertNotNull(restored);
+ assertEquals((long) i, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ }
+
+ assertNull(reader.nextEvent());
+ } finally {
+ file.delete();
+ }
+ }
+
+ @Test
+ public void testReadFirstEventWithBlockOffsetUncompressed() throws IOException {
+ dos.writeBoolean(false);
+ writeRecords(0, 10, false);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, false);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(restored);
+ assertEquals(10L, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ } finally {
+ file.delete();
+ }
+ }
+
+ @Test
+ public void testReadSubsequentEventWithBlockOffsetUncompressed() throws IOException {
+ dos.writeBoolean(false);
+ writeRecords(0, 10, false);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, false);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(restored);
+ assertEquals(10L, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ } finally {
+ file.delete();
+ }
+ }
+
+ @Test
+ public void testReadMultipleEventsWithBlockOffsetUncompressed() throws IOException {
+ dos.writeBoolean(false);
+ writeRecords(0, 10, false);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, false);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ for (int i=0; i < 2; i++) {
+ final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(event10);
+ assertEquals(10L, event10.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+
+ final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+ assertNotNull(event13);
+ assertEquals(13L, event13.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+ }
+ } finally {
+ file.delete();
+ }
+ }
+
+
+ @Test
+ public void testReadFirstEventCompressed() throws IOException {
+ dos.writeBoolean(true);
+ dos.writeUTF(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC);
+
+ writeRecords(88L, 1, true);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ final ProvenanceEventRecord restored = reader.nextEvent();
+ assertNotNull(restored);
+ assertEquals(88L, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ } finally {
+ file.delete();
+ }
+ }
+
+ @Test
+ public void testReadManyCompressed() throws IOException {
+ dos.writeBoolean(true);
+ dos.writeUTF(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC);
+
+ writeRecords(0, 1024, true);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ for (int i=0; i < 1024; i++) {
+ final ProvenanceEventRecord restored = reader.nextEvent();
+ assertNotNull(restored);
+ assertEquals((long) i, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ }
+
+ assertNull(reader.nextEvent());
+ } finally {
+ file.delete();
+ }
+ }
+
+
+ @Test
+ public void testReadFirstEventWithBlockOffsetCompressed() throws IOException {
+ dos.writeBoolean(true);
+ dos.writeUTF(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC);
+ writeRecords(0, 10, true);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, true);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(restored);
+ assertEquals(10L, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ } finally {
+ file.delete();
+ }
+ }
+
+ @Test
+ public void testReadSubsequentEventWithBlockOffsetCompressed() throws IOException {
+ dos.writeBoolean(true);
+ dos.writeUTF(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC);
+
+ writeRecords(0, 10, true);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, true);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(restored);
+ assertEquals(10L, restored.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+ } finally {
+ file.delete();
+ }
+ }
+
+ @Test
+ public void testReadMultipleEventsWithBlockOffsetCompressed() throws IOException {
+ dos.writeBoolean(true);
+ dos.writeUTF(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC);
+
+ writeRecords(0, 10, true);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, true);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ for (int i=0; i < 2; i++) {
+ final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(event10);
+ assertEquals(10L, event10.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+
+ final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+ assertNotNull(event13);
+ assertEquals(13L, event13.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+ }
+ } finally {
+ file.delete();
+ }
+ }
+
+
+ @Test
+ public void testReadEventWithBlockOffsetThenPreviousBlockOffsetUncompressed() throws IOException {
+ dos.writeBoolean(false);
+ final int firstBlockOffset = baos.size();
+ writeRecords(0, 10, false);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, false);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ for (int j=0; j < 2; j++) {
+ for (int i=0; i < 2; i++) {
+ final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(event10);
+ assertEquals(10L, event10.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+
+ final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+ assertNotNull(event13);
+ assertEquals(13L, event13.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+ }
+
+ for (int i=0; i < 2; i++) {
+ final ProvenanceEventRecord event2 = reader.getEvent(firstBlockOffset, 2L);
+ assertNotNull(event2);
+ assertEquals(2L, event2.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event2.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event2.getFlowFileUuid());
+
+ final ProvenanceEventRecord event6 = reader.getEvent(firstBlockOffset, 6L);
+ assertNotNull(event6);
+ assertEquals(6L, event6.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event6.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event6.getFlowFileUuid());
+ }
+ }
+ } finally {
+ file.delete();
+ }
+ }
+
+
+ @Test
+ public void testReadEventWithBlockOffsetThenPreviousBlockOffsetCompressed() throws IOException {
+ dos.writeBoolean(true);
+ dos.writeUTF(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC);
+
+ final int firstBlockOffset = baos.size();
+ writeRecords(0, 10, true);
+
+ final int secondBlockOffset = baos.size();
+ writeRecords(10, 10, true);
+
+ // write data to a file so that we can read it with the journal reader
+ final File dir = new File("target/testData");
+ final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+ dir.mkdirs();
+
+ try (final FileOutputStream fos = new FileOutputStream(file)) {
+ baos.writeTo(fos);
+ }
+
+ // read the record and verify its contents
+ try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+ for (int j=0; j < 2; j++) {
+ for (int i=0; i < 2; i++) {
+ final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+ assertNotNull(event10);
+ assertEquals(10L, event10.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+
+ final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+ assertNotNull(event13);
+ assertEquals(13L, event13.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+ }
+
+ for (int i=0; i < 2; i++) {
+ final ProvenanceEventRecord event2 = reader.getEvent(firstBlockOffset, 2L);
+ assertNotNull(event2);
+ assertEquals(2L, event2.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event2.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event2.getFlowFileUuid());
+
+ final ProvenanceEventRecord event6 = reader.getEvent(firstBlockOffset, 6L);
+ assertNotNull(event6);
+ assertEquals(6L, event6.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event6.getEventType());
+ assertEquals("00000000-0000-0000-0000-000000000000", event6.getFlowFileUuid());
+ }
+ }
+ } finally {
+ file.delete();
+ }
+ }
+
+
+
+
+ private void writeRecord(final long id) throws IOException {
+ writeRecord(id, dos);
+ }
+
+ private void writeRecords(final long startId, final int numRecords, final boolean compressed) throws IOException {
+ if ( compressed ) {
+ final CompressionOutputStream compressedOut = new CompressionOutputStream(dos);
+ for (long id = startId; id < startId + numRecords; id++) {
+ writeRecord(id, new DataOutputStream(compressedOut));
+ }
+ compressedOut.close();
+ } else {
+ for (long id = startId; id < startId + numRecords; id++) {
+ writeRecord(id, dos);
+ }
+ }
+ }
+
+ private void writeRecord(final long id, final DataOutputStream dos) throws IOException {
+ // Create prov event to add to the stream
+ final ProvenanceEventRecord event = new StandardProvenanceEventRecord.Builder()
+ .setEventType(ProvenanceEventType.CREATE)
+ .setFlowFileUUID("00000000-0000-0000-0000-000000000000")
+ .setComponentType("Unit Test")
+ .setComponentId(UUID.randomUUID().toString())
+ .setEventTime(System.currentTimeMillis())
+ .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
+ .setLineageStartDate(System.currentTimeMillis() - 2000L)
+ .setCurrentContentClaim(null, null, null, null, 0L)
+ .build();
+
+ // Serialize the prov event
+ final ByteArrayOutputStream serializationStream = new ByteArrayOutputStream();
+ final StandardEventSerializer serializer = new StandardEventSerializer();
+ serializer.serialize(event, new DataOutputStream(serializationStream));
+
+ // Write out to our stream the event length, followed by the id, and then the serialized event
+ final int recordLen = 8 + serializationStream.size();
+
+ dos.writeInt(recordLen);
+ dos.writeLong(id);
+ serializationStream.writeTo(dos);
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
new file mode 100644
index 0000000..956df80
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
@@ -0,0 +1,176 @@
+/*
+ * 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.provenance.journaling.journals;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileAlreadyExistsException;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.journaling.TestUtil;
+import org.apache.nifi.provenance.journaling.io.StandardEventDeserializer;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.remote.io.CompressionInputStream;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestStandardJournalWriter {
+
+ @Test
+ public void testOverwriteEmptyFile() throws IOException {
+ final File journalFile = new File("target/" + UUID.randomUUID().toString());
+ try {
+ assertTrue( journalFile.createNewFile() );
+
+ try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, new DeflatorCompressionCodec(), new StandardEventSerializer())) {
+
+ }
+ } finally {
+ FileUtils.deleteFile(journalFile, false);
+ }
+ }
+
+ @Test
+ public void testDoNotOverwriteNonEmptyFile() throws IOException {
+ final File journalFile = new File("target/" + UUID.randomUUID().toString());
+ try {
+ assertTrue( journalFile.createNewFile() );
+
+ try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, new DeflatorCompressionCodec(), new StandardEventSerializer())) {
+ writer.write(Collections.singleton(TestUtil.generateEvent(1L)), 1L);
+ }
+
+ try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, new DeflatorCompressionCodec(), new StandardEventSerializer())) {
+ Assert.fail("StandardJournalWriter attempted to overwrite existing file");
+ } catch (final FileAlreadyExistsException faee) {
+ // expected
+ }
+ } finally {
+ FileUtils.deleteFile(journalFile, false);
+ }
+ }
+
+ @Test
+ public void testOneBlockOneRecordWriteCompressed() throws IOException {
+ final File journalFile = new File("target/" + UUID.randomUUID().toString());
+
+ final StandardEventSerializer serializer = new StandardEventSerializer();
+ try {
+ try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, new DeflatorCompressionCodec(), serializer)) {
+ writer.beginNewBlock();
+ writer.write(Collections.singleton(TestUtil.generateEvent(1L)), 1L);
+ writer.finishBlock();
+ }
+
+ final byte[] data = Files.readAllBytes(journalFile.toPath());
+ final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+ final DataInputStream dis = new DataInputStream(bais);
+
+ StandardJournalMagicHeader.read(dis);
+ final String codecName = dis.readUTF();
+ assertEquals(StandardEventSerializer.CODEC_NAME, codecName);
+
+ final int version = dis.readInt();
+ assertEquals(1, version);
+
+ // compression flag
+ assertEquals(true, dis.readBoolean());
+
+ // compression codec name
+ final String compressionCodecName = dis.readUTF();
+ assertEquals(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC, compressionCodecName);
+
+ // read block start
+ final CompressionInputStream decompressedIn = new CompressionInputStream(bais);
+ final StandardEventDeserializer deserializer = new StandardEventDeserializer();
+
+ final DataInputStream decompressedDis = new DataInputStream(decompressedIn);
+ final int eventLength = decompressedDis.readInt();
+ assertEquals(131, eventLength);
+ final ProvenanceEventRecord event = deserializer.deserialize(decompressedDis, 0);
+ assertEquals(1, event.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event.getEventType());
+
+ assertEquals(-1, decompressedIn.read());
+ } finally {
+ journalFile.delete();
+ }
+ }
+
+ @Test
+ public void testManyBlocksOneRecordWriteCompressed() throws IOException {
+ final File journalFile = new File("target/" + UUID.randomUUID().toString());
+
+ final StandardEventSerializer serializer = new StandardEventSerializer();
+ try {
+ try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, new DeflatorCompressionCodec(), serializer)) {
+ for (int i=0; i < 1024; i++) {
+ writer.beginNewBlock();
+ writer.write(Collections.singleton(TestUtil.generateEvent(1L)), 1L);
+ writer.finishBlock();
+ }
+ }
+
+ final byte[] data = Files.readAllBytes(journalFile.toPath());
+ final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+ final DataInputStream dis = new DataInputStream(bais);
+
+ StandardJournalMagicHeader.read(dis);
+ final String codecName = dis.readUTF();
+ assertEquals(StandardEventSerializer.CODEC_NAME, codecName);
+
+ final int version = dis.readInt();
+ assertEquals(1, version);
+
+ // compression flag
+ assertEquals(true, dis.readBoolean());
+
+ assertEquals(DeflatorCompressionCodec.DEFLATOR_COMPRESSION_CODEC, dis.readUTF());
+
+ // read block start
+ for (int i=0; i < 1024; i++) {
+ final CompressionInputStream decompressedIn = new CompressionInputStream(bais);
+ final StandardEventDeserializer deserializer = new StandardEventDeserializer();
+
+ final DataInputStream decompressedDis = new DataInputStream(decompressedIn);
+ final int eventLength = decompressedDis.readInt();
+ assertEquals(131, eventLength);
+ final ProvenanceEventRecord event = deserializer.deserialize(decompressedDis, 0);
+ assertEquals(1, event.getEventId());
+ assertEquals(ProvenanceEventType.CREATE, event.getEventType());
+
+ if ( i == 1023 ) {
+ assertEquals(-1, decompressedIn.read());
+ }
+ }
+ } finally {
+ journalFile.delete();
+ }
+ }
+
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/partition/TestQueuingPartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/partition/TestQueuingPartitionManager.java
new file mode 100644
index 0000000..683e40a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/partition/TestQueuingPartitionManager.java
@@ -0,0 +1,75 @@
+/*
+ * 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.provenance.journaling.partition;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestQueuingPartitionManager {
+
+ @Test(timeout=5000)
+ public void testWriteWaitsForDeletion() throws IOException {
+ final Map<String, File> containers = new HashMap<>();
+ containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+
+ final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+ config.setCompressOnRollover(false);
+ config.setMaxStorageCapacity(50L);
+ config.setContainers(containers);
+
+ final ScheduledExecutorService exec = Executors.newScheduledThreadPool(1);
+
+ final AtomicLong indexSize = new AtomicLong(0L);
+ final IndexManager indexManager = Mockito.mock(IndexManager.class);
+ Mockito.doAnswer(new Answer<Long>() {
+ @Override
+ public Long answer(InvocationOnMock invocation) throws Throwable {
+ return indexSize.get();
+ }
+ }).when(indexManager).getSize(Mockito.any(String.class));
+
+ final QueuingPartitionManager mgr = new QueuingPartitionManager(indexManager, new AtomicLong(0L), config, exec, exec);
+
+ Partition partition = mgr.nextPartition(true, true);
+ assertNotNull(partition);
+
+ indexSize.set(1024L);
+ partition = mgr.nextPartition(true, false);
+ assertNull(partition);
+
+ indexSize.set(0L);
+ partition = mgr.nextPartition(true, true);
+ assertNotNull(partition);
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocReader.java
new file mode 100644
index 0000000..d5c4037
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocReader.java
@@ -0,0 +1,91 @@
+/*
+ * 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.provenance.journaling.toc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.UUID;
+
+import org.junit.Test;
+
+public class TestStandardTocReader {
+
+ @Test
+ public void testDetectsCompression() throws IOException {
+ final File file = new File("target/" + UUID.randomUUID().toString());
+ try (final OutputStream out = new FileOutputStream(file)) {
+ out.write(0);
+ out.write(0);
+ }
+
+ try {
+ try(final StandardTocReader reader = new StandardTocReader(file)) {
+ assertFalse(reader.isCompressed());
+ }
+ } finally {
+ file.delete();
+ }
+
+
+ try (final OutputStream out = new FileOutputStream(file)) {
+ out.write(0);
+ out.write(1);
+ }
+
+ try {
+ try(final StandardTocReader reader = new StandardTocReader(file)) {
+ assertTrue(reader.isCompressed());
+ }
+ } finally {
+ file.delete();
+ }
+ }
+
+
+ @Test
+ public void testGetBlockIndex() throws IOException {
+ final File file = new File("target/" + UUID.randomUUID().toString());
+ try (final OutputStream out = new FileOutputStream(file);
+ final DataOutputStream dos = new DataOutputStream(out)) {
+ out.write(0);
+ out.write(0);
+
+ for (int i=0; i < 1024; i++) {
+ dos.writeLong(i * 1024L);
+ }
+ }
+
+ try {
+ try(final StandardTocReader reader = new StandardTocReader(file)) {
+ assertFalse(reader.isCompressed());
+
+ for (int i=0; i < 1024; i++) {
+ assertEquals(i * 1024, reader.getBlockOffset(i));
+ }
+ }
+ } finally {
+ file.delete();
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocWriter.java
new file mode 100644
index 0000000..3cddb5e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocWriter.java
@@ -0,0 +1,64 @@
+/*
+ * 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.provenance.journaling.toc;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileAlreadyExistsException;
+import java.util.UUID;
+
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestStandardTocWriter {
+ @Test
+ public void testOverwriteEmptyFile() throws IOException {
+ final File tocFile = new File("target/" + UUID.randomUUID().toString() + ".toc");
+ try {
+ assertTrue( tocFile.createNewFile() );
+
+ try (final StandardTocWriter writer = new StandardTocWriter(tocFile, false, false)) {
+ }
+ } finally {
+ FileUtils.deleteFile(tocFile, false);
+ }
+ }
+
+ @Test
+ public void testDoNotOverwriteNonEmptyFile() throws IOException {
+ final File tocFile = new File("target/" + UUID.randomUUID().toString() + ".toc");
+ try {
+ assertTrue( tocFile.createNewFile() );
+
+ try (final StandardTocWriter writer = new StandardTocWriter(tocFile, false, false)) {
+ writer.addBlockOffset(0L);
+ writer.addBlockOffset(34L);
+ }
+
+ try (final StandardTocWriter writer = new StandardTocWriter(tocFile, false, false)) {
+ Assert.fail("StandardTocWriter attempted to overwrite existing file");
+ } catch (final FileAlreadyExistsException faee) {
+ // expected
+ }
+ } finally {
+ FileUtils.deleteFile(tocFile, false);
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
index f46b5fc..fef120d 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
@@ -71,6 +71,7 @@
import org.apache.nifi.provenance.lucene.IndexingAction;
import org.apache.nifi.provenance.lucene.LineageQuery;
import org.apache.nifi.provenance.lucene.LuceneUtil;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
import org.apache.nifi.provenance.rollover.CompressionAction;
import org.apache.nifi.provenance.rollover.RolloverAction;
import org.apache.nifi.provenance.search.Query;
@@ -152,19 +153,25 @@
private final AtomicBoolean repoDirty = new AtomicBoolean(false);
// we keep the last 1000 records on hand so that when the UI is opened and it asks for the last 1000 records we don't need to
// read them. Since this is a very cheap operation to keep them, it's worth the tiny expense for the improved user experience.
- private final RingBuffer<ProvenanceEventRecord> latestRecords = new RingBuffer<>(1000);
+ private final RingBuffer<StoredProvenanceEvent> latestRecords = new RingBuffer<>(1000);
private EventReporter eventReporter;
+ private long rolloverCheckMillis = 10;
public PersistentProvenanceRepository() throws IOException {
this(createRepositoryConfiguration());
}
public PersistentProvenanceRepository(final RepositoryConfiguration configuration) throws IOException {
+ this(configuration, 10);
+ }
+
+ PersistentProvenanceRepository(final RepositoryConfiguration configuration, final int rolloverCheckMillis) throws IOException {
if (configuration.getStorageDirectories().isEmpty()) {
throw new IllegalArgumentException("Must specify at least one storage directory");
}
this.configuration = configuration;
+ this.rolloverCheckMillis = rolloverCheckMillis;
for (final File file : configuration.getStorageDirectories()) {
final Path storageDirectory = file.toPath();
@@ -205,8 +212,9 @@
rolloverExecutor = Executors.newFixedThreadPool(numRolloverThreads, new NamedThreadFactory("Provenance Repository Rollover Thread"));
}
+
@Override
- public void initialize(final EventReporter eventReporter) throws IOException {
+ public synchronized void initialize(final EventReporter eventReporter) throws IOException {
if (initialized.getAndSet(true)) {
return;
}
@@ -244,7 +252,7 @@
}
}
}
- }, 10L, 10L, TimeUnit.SECONDS);
+ }, rolloverCheckMillis, rolloverCheckMillis, TimeUnit.MILLISECONDS);
scheduledExecService.scheduleWithFixedDelay(new RemoveExpiredQueryResults(), 30L, 3L, TimeUnit.SECONDS);
scheduledExecService.scheduleWithFixedDelay(new Runnable() {
@@ -350,13 +358,13 @@
}
@Override
- public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
+ public void registerEvents(final Collection<ProvenanceEventRecord> events) {
persistRecord(events);
}
@Override
- public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
- final List<ProvenanceEventRecord> records = new ArrayList<>(maxRecords);
+ public List<StoredProvenanceEvent> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
+ final List<StoredProvenanceEvent> records = new ArrayList<>(maxRecords);
final List<Path> paths = getPathsForId(firstRecordId);
if (paths == null || paths.isEmpty()) {
@@ -368,7 +376,7 @@
StandardProvenanceEventRecord record;
while (records.size() < maxRecords && ((record = reader.nextRecord()) != null)) {
if (record.getEventId() >= firstRecordId) {
- records.add(record);
+ records.add(new IdEnrichedProvenanceEvent(record));
}
}
} catch (final EOFException | FileNotFoundException fnfe) {
@@ -1133,7 +1141,7 @@
return mergedFile;
}
- static File mergeJournals(final List<File> journalFiles, final File storageDir, final File mergedFile, final EventReporter eventReporter, final RingBuffer<ProvenanceEventRecord> ringBuffer) throws IOException {
+ static File mergeJournals(final List<File> journalFiles, final File storageDir, final File mergedFile, final EventReporter eventReporter, final RingBuffer<StoredProvenanceEvent> ringBuffer) throws IOException {
final long startNanos = System.nanoTime();
if (journalFiles.isEmpty()) {
return null;
@@ -1210,7 +1218,7 @@
final RecordReader reader = entry.getValue();
writer.writeRecord(record, record.getEventId());
- ringBuffer.add(record);
+ ringBuffer.add(new IdEnrichedProvenanceEvent(record));
records++;
// Remove this entry from the map
@@ -1325,8 +1333,8 @@
final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
if (latestRecords.getSize() >= query.getMaxResults()) {
- final List<ProvenanceEventRecord> latestList = latestRecords.asList();
- final List<ProvenanceEventRecord> trimmed;
+ final List<StoredProvenanceEvent> latestList = latestRecords.asList();
+ final List<StoredProvenanceEvent> trimmed;
if (latestList.size() > query.getMaxResults()) {
trimmed = latestList.subList(latestList.size() - query.getMaxResults(), latestList.size());
} else {
@@ -1335,7 +1343,7 @@
final Long maxEventId = getMaxEventId();
if (maxEventId == null) {
- result.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0L);
+ result.getResult().update(Collections.<StoredProvenanceEvent>emptyList(), 0L);
}
Long minIndexedId = indexConfig.getMinIdIndexed();
if (minIndexedId == null) {
@@ -1361,7 +1369,7 @@
querySubmissionMap.put(query.getIdentifier(), result);
if (indexDirectories.isEmpty()) {
- result.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0L);
+ result.getResult().update(Collections.<StoredProvenanceEvent>emptyList(), 0L);
} else {
for (final File indexDir : indexDirectories) {
queryExecService.submit(new QueryRunnable(query, result, indexDir, retrievalCount));
@@ -1582,7 +1590,7 @@
if (event == null) {
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
- submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+ submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
return submission;
}
@@ -1619,7 +1627,7 @@
if (event == null) {
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
- submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+ submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
return submission;
}
@@ -1661,12 +1669,21 @@
}
@Override
- public ProvenanceEventRecord getEvent(final long id) throws IOException {
- final List<ProvenanceEventRecord> records = getEvents(id, 1);
+ public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+ if ( !(location instanceof EventIdLocation) ) {
+ throw new IllegalArgumentException("Invalid StorageLocation for this repository");
+ }
+
+ return getEvent(((EventIdLocation) location).getId());
+ }
+
+ @Override
+ public StoredProvenanceEvent getEvent(final long id) throws IOException {
+ final List<StoredProvenanceEvent> records = getEvents(id, 1);
if (records.isEmpty()) {
return null;
}
- final ProvenanceEventRecord record = records.get(0);
+ final StoredProvenanceEvent record = records.get(0);
if (record.getEventId() != id) {
return null;
}
@@ -1729,7 +1746,7 @@
// get the max indexed event id
final Long maxEventId = indexConfig.getMaxIdIndexed();
if (maxEventId == null) {
- submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+ submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList(), 0);
return;
}
@@ -1743,7 +1760,7 @@
}
final long totalNumDocs = maxEventId - minIndexedId;
- final List<ProvenanceEventRecord> mostRecent = getEvents(startIndex, maxResults);
+ final List<StoredProvenanceEvent> mostRecent = getEvents(startIndex, maxResults);
submission.getResult().update(mostRecent, totalNumDocs);
} catch (final IOException ioe) {
logger.error("Failed to retrieve records from Provenance Repository: " + ioe.toString());
@@ -1818,7 +1835,7 @@
}
try {
- final Set<ProvenanceEventRecord> matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, indexDir, null, flowFileUuids);
+ final Set<StoredProvenanceEvent> matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, indexDir, null, flowFileUuids);
final StandardLineageResult result = submission.getResult();
result.update(matchingRecords);
@@ -1889,4 +1906,36 @@
return thread;
}
}
+
+ @Override
+ public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+ final List<StoredProvenanceEvent> storedEvents = new ArrayList<>(storageLocations.size());
+ for ( final StorageLocation location : storageLocations ) {
+ final StoredProvenanceEvent event = getEvent(location);
+ if ( event != null ) {
+ storedEvents.add(event);
+ }
+ }
+ return storedEvents;
+ }
+
+ @Override
+ public Long getEarliestEventTime() throws IOException {
+ final List<StoredProvenanceEvent> firstEvents = getEvents(0, 1);
+ if ( firstEvents == null || firstEvents.isEmpty() ) {
+ return null;
+ }
+
+ return firstEvents.get(0).getEventTime();
+ }
+
+ @Override
+ public ProvenanceQuerySubmission submitQuery(final String query) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(final String queryIdentifier) {
+ throw new UnsupportedOperationException();
+ }
}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
index af5fe50..be1caf0 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
@@ -28,22 +28,22 @@
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.StandardProvenanceEventRecord;
-import org.apache.nifi.provenance.serialization.RecordReader;
-import org.apache.nifi.provenance.serialization.RecordReaders;
-
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
+import org.apache.nifi.provenance.IdEnrichedProvenanceEvent;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.serialization.RecordReader;
+import org.apache.nifi.provenance.serialization.RecordReaders;
public class DocsReader {
public DocsReader(final List<File> storageDirectories) {
}
- public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
+ public Set<StoredProvenanceEvent> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
if (retrievalCount.get() >= maxResults) {
return Collections.emptySet();
}
@@ -59,13 +59,13 @@
return read(docs, allProvenanceLogFiles, retrievalCount, maxResults);
}
- public Set<ProvenanceEventRecord> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
+ public Set<StoredProvenanceEvent> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
LuceneUtil.sortDocsForRetrieval(docs);
RecordReader reader = null;
String lastStorageFilename = null;
long lastByteOffset = 0L;
- final Set<ProvenanceEventRecord> matchingRecords = new LinkedHashSet<>();
+ final Set<StoredProvenanceEvent> matchingRecords = new LinkedHashSet<>();
try {
for (final Document d : docs) {
@@ -78,7 +78,8 @@
try {
reader.skipTo(byteOffset);
final StandardProvenanceEventRecord record = reader.nextRecord();
- matchingRecords.add(record);
+ matchingRecords.add(new IdEnrichedProvenanceEvent(record));
+
if (retrievalCount.incrementAndGet() >= maxResults) {
break;
}
@@ -107,7 +108,7 @@
reader.skip(byteOffset);
final StandardProvenanceEventRecord record = reader.nextRecord();
- matchingRecords.add(record);
+ matchingRecords.add(new IdEnrichedProvenanceEvent(record));
if (retrievalCount.incrementAndGet() >= maxResults) {
break;
}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
index e2854c3..de7da44 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
@@ -26,7 +26,7 @@
import org.apache.nifi.provenance.PersistentProvenanceRepository;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.StandardQueryResult;
-
+import org.apache.nifi.provenance.StoredProvenanceEvent;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexNotFoundException;
import org.apache.lucene.search.IndexSearcher;
@@ -53,7 +53,7 @@
}
final StandardQueryResult sqr = new StandardQueryResult(provenanceQuery, 1);
- final Set<ProvenanceEventRecord> matchingRecords;
+ final Set<StoredProvenanceEvent> matchingRecords;
try (final DirectoryReader directoryReader = DirectoryReader.open(FSDirectory.open(indexDirectory))) {
final IndexSearcher searcher = new IndexSearcher(directoryReader);
@@ -65,7 +65,7 @@
TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults());
if (topDocs.totalHits == 0) {
- sqr.update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+ sqr.update(Collections.<StoredProvenanceEvent>emptyList(), 0);
return sqr;
}
@@ -76,7 +76,7 @@
return sqr;
} catch (final IndexNotFoundException e) {
// nothing has been indexed yet.
- sqr.update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+ sqr.update(Collections.<StoredProvenanceEvent>emptyList(), 0);
return sqr;
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
index 54cde15..54407ed 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
@@ -25,10 +25,6 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.nifi.provenance.PersistentProvenanceRepository;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.SearchableFields;
-
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
@@ -39,6 +35,9 @@
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
+import org.apache.nifi.provenance.PersistentProvenanceRepository;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -48,7 +47,7 @@
public static final int MAX_LINEAGE_UUIDS = 100;
private static final Logger logger = LoggerFactory.getLogger(LineageQuery.class);
- public static Set<ProvenanceEventRecord> computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final File indexDirectory, final String lineageIdentifier, final Collection<String> flowFileUuids) throws IOException {
+ public static Set<StoredProvenanceEvent> computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final File indexDirectory, final String lineageIdentifier, final Collection<String> flowFileUuids) throws IOException {
if (requireNonNull(flowFileUuids).size() > MAX_LINEAGE_UUIDS) {
throw new IllegalArgumentException(String.format("Cannot compute lineage for more than %s FlowFiles. This lineage contains %s.", MAX_LINEAGE_UUIDS, flowFileUuids.size()));
}
@@ -97,7 +96,7 @@
final long searchEnd = System.nanoTime();
final DocsReader docsReader = new DocsReader(repo.getConfiguration().getStorageDirectories());
- final Set<ProvenanceEventRecord> recs = docsReader.read(uuidQueryTopDocs, indexReader, repo.getAllLogFiles(), new AtomicInteger(0), Integer.MAX_VALUE);
+ final Set<StoredProvenanceEvent> recs = docsReader.read(uuidQueryTopDocs, indexReader, repo.getAllLogFiles(), new AtomicInteger(0), Integer.MAX_VALUE);
final long readDocsEnd = System.nanoTime();
logger.debug("Finished Lineage Query; Lucene search took {} millis, reading records took {} millis", TimeUnit.NANOSECONDS.toMillis(searchEnd - searchStart), TimeUnit.NANOSECONDS.toMillis(readDocsEnd - searchEnd));
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
index 7305872..84fd9ee 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
@@ -170,7 +170,7 @@
config.setCompressOnRollover(false);
config.setJournalCount(10);
config.setQueryThreadPoolSize(10);
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final Map<String, String> attributes = new HashMap<>();
@@ -218,12 +218,12 @@
System.out.println("Closing and re-initializing");
repo.close();
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
System.out.println("Re-initialized");
final long fetchStart = System.nanoTime();
- final List<ProvenanceEventRecord> records = repo.getEvents(0L, 1000000);
+ final List<StoredProvenanceEvent> records = repo.getEvents(0L, 1000000);
final long fetchMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - fetchStart);
assertEquals(1000000, records.size());
final long fetchRecsPerMilli = 1000000 / fetchMillis;
@@ -238,7 +238,7 @@
final RepositoryConfiguration config = createConfiguration();
config.setMaxEventFileCapacity(1L);
config.setMaxEventFileLife(1, TimeUnit.SECONDS);
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final Map<String, String> attributes = new HashMap<>();
@@ -262,9 +262,9 @@
repo.close();
Thread.sleep(500L); // Give the repo time to shutdown (i.e., close all file handles, etc.)
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
- final List<ProvenanceEventRecord> recoveredRecords = repo.getEvents(0L, 12);
+ final List<StoredProvenanceEvent> recoveredRecords = repo.getEvents(0L, 12);
assertEquals(10, recoveredRecords.size());
for (int i = 0; i < 10; i++) {
@@ -285,7 +285,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setMaxEventFileLife(2, TimeUnit.SECONDS);
config.setSearchableFields(searchableFields);
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final Map<String, String> attributes = new HashMap<>();
@@ -339,7 +339,7 @@
final RepositoryConfiguration config = createConfiguration();
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -383,7 +383,7 @@
final RepositoryConfiguration config = createConfiguration();
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
config.setCompressOnRollover(true);
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -421,7 +421,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "10000000-0000-0000-0000-000000000000";
@@ -480,7 +480,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -543,7 +543,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -625,7 +625,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -681,7 +681,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000001";
@@ -736,7 +736,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000001";
@@ -795,7 +795,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String childId = "00000000-0000-0000-0000-000000000000";
@@ -845,7 +845,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String childId = "00000000-0000-0000-0000-000000000000";
@@ -892,7 +892,7 @@
public void testCorrectProvenanceEventIdOnRestore() throws IOException {
final RepositoryConfiguration config = createConfiguration();
config.setMaxEventFileLife(1, TimeUnit.SECONDS);
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -918,7 +918,7 @@
repo.close();
- final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config);
+ final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config, 75);
secondRepo.initialize(getEventReporter());
final ProvenanceEventRecord event11 = builder.build();
@@ -938,7 +938,7 @@
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -1002,7 +1002,7 @@
final RepositoryConfiguration config = createConfiguration();
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -1070,7 +1070,7 @@
public void testMergeJournals() throws IOException, InterruptedException {
final RepositoryConfiguration config = createConfiguration();
config.setMaxEventFileLife(3, TimeUnit.SECONDS);
- repo = new PersistentProvenanceRepository(config);
+ repo = new PersistentProvenanceRepository(config, 75);
repo.initialize(getEventReporter());
final Map<String, String> attributes = new HashMap<>();
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml
index fcacb3d..33ff18e 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml
@@ -30,5 +30,9 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-volatile-provenance-repository</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-journaling-provenance-repository</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/pom.xml
index d95e29f..70cc5b4 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/pom.xml
@@ -30,6 +30,13 @@
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-data-provenance-utils</artifactId>
</dependency>
+ <!--
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-provenance-query-language</artifactId>
+ <version>0.0.2-incubating-SNAPSHOT</version>
+ </dependency>
+ -->
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-utils</artifactId>
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
index f4f9d12..bd084bc 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
@@ -24,7 +24,6 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService;
@@ -44,6 +43,7 @@
import org.apache.nifi.provenance.lineage.FlowFileLineage;
import org.apache.nifi.provenance.lineage.Lineage;
import org.apache.nifi.provenance.lineage.LineageComputationType;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
import org.apache.nifi.provenance.search.Query;
import org.apache.nifi.provenance.search.QueryResult;
import org.apache.nifi.provenance.search.QuerySubmission;
@@ -57,14 +57,15 @@
import org.apache.nifi.util.RingBuffer.IterationDirection;
public class VolatileProvenanceRepository implements ProvenanceEventRepository {
-
+ private static final int MAX_CONCURRENT_QUERIES = 10;
+
// properties
public static final String BUFFER_SIZE = "nifi.provenance.repository.buffer.size";
// default property values
public static final int DEFAULT_BUFFER_SIZE = 10000;
- private final RingBuffer<ProvenanceEventRecord> ringBuffer;
+ private final RingBuffer<StoredProvenanceEvent> ringBuffer;
private final List<SearchableField> searchableFields;
private final List<SearchableField> searchableAttributes;
private final ExecutorService queryExecService;
@@ -72,6 +73,7 @@
private final ConcurrentMap<String, AsyncQuerySubmission> querySubmissionMap = new ConcurrentHashMap<>();
private final ConcurrentMap<String, AsyncLineageSubmission> lineageSubmissionMap = new ConcurrentHashMap<>();
+ private final ConcurrentMap<String, ProvenanceQuerySubmission> provQuerySubmissionMap = new ConcurrentHashMap<>();
private final AtomicLong idGenerator = new AtomicLong(0L);
private final AtomicBoolean initialized = new AtomicBoolean(false);
@@ -119,21 +121,21 @@
@Override
public void registerEvent(final ProvenanceEventRecord event) {
final long id = idGenerator.getAndIncrement();
- ringBuffer.add(new IdEnrichedProvEvent(event, id));
+ ringBuffer.add(new IdEnrichedProvenanceEvent(event, id));
}
@Override
- public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
+ public void registerEvents(final Collection<ProvenanceEventRecord> events) {
for (final ProvenanceEventRecord event : events) {
registerEvent(event);
}
}
@Override
- public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
- return ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
+ public List<StoredProvenanceEvent> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
+ return ringBuffer.getSelectedElements(new Filter<StoredProvenanceEvent>() {
@Override
- public boolean select(final ProvenanceEventRecord value) {
+ public boolean select(final StoredProvenanceEvent value) {
return value.getEventId() >= firstRecordId;
}
}, maxRecords);
@@ -145,21 +147,22 @@
return (newest == null) ? null : newest.getEventId();
}
- public ProvenanceEventRecord getEvent(final String identifier) throws IOException {
- final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
+ public StoredProvenanceEvent getEvent(final String identifier) throws IOException {
+ final List<StoredProvenanceEvent> records = ringBuffer.getSelectedElements(new Filter<StoredProvenanceEvent>() {
@Override
- public boolean select(final ProvenanceEventRecord event) {
+ public boolean select(final StoredProvenanceEvent event) {
return identifier.equals(event.getFlowFileUuid());
}
}, 1);
+
return records.isEmpty() ? null : records.get(0);
}
@Override
- public ProvenanceEventRecord getEvent(final long id) {
- final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
+ public StoredProvenanceEvent getEvent(final long id) {
+ final List<StoredProvenanceEvent> records = ringBuffer.getSelectedElements(new Filter<StoredProvenanceEvent>() {
@Override
- public boolean select(final ProvenanceEventRecord event) {
+ public boolean select(final StoredProvenanceEvent event) {
return event.getEventId() == id;
}
}, 1);
@@ -407,13 +410,51 @@
throw new UnsupportedOperationException();
}
+
+ @Override
+ public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+ final List<StoredProvenanceEvent> events = new ArrayList<>(storageLocations.size());
+ for ( final StorageLocation location : storageLocations ) {
+ if ( !(location instanceof EventIdLocation) ) {
+ throw new IllegalArgumentException("Illegal Storage Location");
+ }
+
+ final long id = ((EventIdLocation) location).getId();
+ final StoredProvenanceEvent event = getEvent(id);
+ if ( event != null ) {
+ events.add(event);
+ }
+ }
+ return events;
+ }
+
+ @Override
+ public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+ if ( !(location instanceof EventIdLocation) ) {
+ throw new IllegalArgumentException("Illegal Storage Location");
+ }
+
+ final long id = ((EventIdLocation) location).getId();
+ return getEvent(id);
+ }
+
+ @Override
+ public Long getEarliestEventTime() throws IOException {
+ final List<StoredProvenanceEvent> events = getEvents(0L, 1);
+ if ( events.isEmpty() ) {
+ return null;
+ }
+
+ return events.get(0).getEventTime();
+ }
+
@Override
public ComputeLineageSubmission submitExpandParents(final long eventId) {
final ProvenanceEventRecord event = getEvent(eventId);
if (event == null) {
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
- submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+ submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
return submission;
}
@@ -432,9 +473,6 @@
}
}
- public Lineage expandSpawnEventChildren(final String identifier) {
- throw new UnsupportedOperationException();
- }
@Override
public ComputeLineageSubmission submitExpandChildren(final long eventId) {
@@ -442,7 +480,7 @@
if (event == null) {
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
- submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+ submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
return submission;
}
@@ -465,9 +503,9 @@
final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1);
lineageSubmissionMap.put(result.getLineageIdentifier(), result);
- final Filter<ProvenanceEventRecord> filter = new Filter<ProvenanceEventRecord>() {
+ final Filter<StoredProvenanceEvent> filter = new Filter<StoredProvenanceEvent>() {
@Override
- public boolean select(final ProvenanceEventRecord event) {
+ public boolean select(final StoredProvenanceEvent event) {
if (flowFileUuids.contains(event.getFlowFileUuid())) {
return true;
}
@@ -495,12 +533,12 @@
private static class QueryRunnable implements Runnable {
- private final RingBuffer<ProvenanceEventRecord> ringBuffer;
+ private final RingBuffer<StoredProvenanceEvent> ringBuffer;
private final Filter<ProvenanceEventRecord> filter;
private final AsyncQuerySubmission submission;
private final int maxRecords;
- public QueryRunnable(final RingBuffer<ProvenanceEventRecord> ringBuffer, final Filter<ProvenanceEventRecord> filter, final int maxRecords, final AsyncQuerySubmission submission) {
+ public QueryRunnable(final RingBuffer<StoredProvenanceEvent> ringBuffer, final Filter<ProvenanceEventRecord> filter, final int maxRecords, final AsyncQuerySubmission submission) {
this.ringBuffer = ringBuffer;
this.filter = filter;
this.submission = submission;
@@ -511,10 +549,10 @@
public void run() {
// Retrieve the most recent results and count the total number of matches
final IntegerHolder matchingCount = new IntegerHolder(0);
- final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords);
- ringBuffer.forEach(new ForEachEvaluator<ProvenanceEventRecord>() {
+ final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>(maxRecords);
+ ringBuffer.forEach(new ForEachEvaluator<StoredProvenanceEvent>() {
@Override
- public boolean evaluate(final ProvenanceEventRecord record) {
+ public boolean evaluate(final StoredProvenanceEvent record) {
if (filter.select(record)) {
if (matchingCount.incrementAndGet() <= maxRecords) {
matchingRecords.add(record);
@@ -532,20 +570,21 @@
private static class ComputeLineageRunnable implements Runnable {
- private final RingBuffer<ProvenanceEventRecord> ringBuffer;
- private final Filter<ProvenanceEventRecord> filter;
+ private final RingBuffer<StoredProvenanceEvent> ringBuffer;
+ private final Filter<StoredProvenanceEvent> filter;
private final AsyncLineageSubmission submission;
- public ComputeLineageRunnable(final RingBuffer<ProvenanceEventRecord> ringBuffer, final Filter<ProvenanceEventRecord> filter, final AsyncLineageSubmission submission) {
+ public ComputeLineageRunnable(final RingBuffer<StoredProvenanceEvent> ringBuffer, final Filter<StoredProvenanceEvent> filter, final AsyncLineageSubmission submission) {
this.ringBuffer = ringBuffer;
this.filter = filter;
this.submission = submission;
}
@Override
+ @SuppressWarnings({ "unchecked", "rawtypes" })
public void run() {
- final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(filter);
- submission.getResult().update(records);
+ final List<StoredProvenanceEvent> records = ringBuffer.getSelectedElements(filter);
+ submission.getResult().update((List) records);
}
}
@@ -577,169 +616,109 @@
}
}
- private static class IdEnrichedProvEvent implements ProvenanceEventRecord {
-
- private final ProvenanceEventRecord record;
- private final long id;
-
- public IdEnrichedProvEvent(final ProvenanceEventRecord record, final long id) {
- this.record = record;
- this.id = id;
+
+
+
+ @Override
+ public ProvenanceQuerySubmission submitQuery(final String query) {
+ throw new UnsupportedOperationException();
+ /*
+ if ( provQuerySubmissionMap.size() > MAX_CONCURRENT_QUERIES ) {
+ final List<String> toRemove = new ArrayList<>();
+ final Date now = new Date();
+
+ for ( final Map.Entry<String, ProvenanceQuerySubmission> entry : provQuerySubmissionMap.entrySet() ) {
+ if ( entry.getValue().getResult().getExpiration().after(now) ) {
+ toRemove.add(entry.getKey());
+ }
+ }
+
+ for ( final String id : toRemove ) {
+ provQuerySubmissionMap.remove(id);
+ }
+
+ if ( provQuerySubmissionMap.size() > MAX_CONCURRENT_QUERIES ) {
+ throw new IllegalStateException("There are already " + MAX_CONCURRENT_QUERIES + " outstanding queries for this Provenance Repository; cannot perform any more queries until the existing queries are expired or canceled");
+ }
}
+
+ final Iterator<StoredProvenanceEvent> eventItr = ringBuffer.asList().iterator();
+ final ProvenanceResultSet rs = ProvenanceQuery.compile(query, getSearchableFields(), getSearchableAttributes()).evaluate(eventItr);
+
+ final Date submissionTime = new Date();
+ final String queryId = UUID.randomUUID().toString();
+ final Date expiration = new Date(System.currentTimeMillis() + TimeUnit.MINUTES.toNanos(10));
+ final ProvenanceQuerySubmission submission = new ProvenanceQuerySubmission() {
+ private final AtomicBoolean canceled = new AtomicBoolean(false);
+
+ @Override
+ public String getQuery() {
+ return query;
+ }
- @Override
- public long getEventId() {
- return id;
- }
+ @Override
+ public ProvenanceQueryResult getResult() {
+ return new ProvenanceQueryResult() {
+ @Override
+ public ProvenanceResultSet getResultSet() {
+ return rs;
+ }
- @Override
- public long getEventTime() {
- return record.getEventTime();
- }
+ @Override
+ public Date getExpiration() {
+ return expiration;
+ }
- @Override
- public long getFlowFileEntryDate() {
- return record.getFlowFileEntryDate();
- }
+ @Override
+ public String getError() {
+ return null;
+ }
- @Override
- public long getLineageStartDate() {
- return record.getLineageStartDate();
- }
+ @Override
+ public int getPercentComplete() {
+ return 100;
+ }
- @Override
- public Set<String> getLineageIdentifiers() {
- return record.getLineageIdentifiers();
- }
+ @Override
+ public boolean isFinished() {
+ return true;
+ }
+ };
+ }
- @Override
- public long getFileSize() {
- return record.getFileSize();
- }
+ @Override
+ public Date getSubmissionTime() {
+ return submissionTime;
+ }
- @Override
- public Long getPreviousFileSize() {
- return record.getPreviousFileSize();
- }
+ @Override
+ public String getQueryIdentifier() {
+ return queryId;
+ }
- @Override
- public long getEventDuration() {
- return record.getEventDuration();
- }
+ @Override
+ public void cancel() {
+ canceled.set(true);
+ provQuerySubmissionMap.remove(queryId);
+ }
- @Override
- public ProvenanceEventType getEventType() {
- return record.getEventType();
- }
+ @Override
+ public boolean isCanceled() {
+ return canceled.get();
+ }
+ };
+
+ provQuerySubmissionMap.putIfAbsent(queryId, submission);
+ return submission;
+ */
+ }
- @Override
- public Map<String, String> getAttributes() {
- return record.getAttributes();
- }
-
- @Override
- public Map<String, String> getPreviousAttributes() {
- return record.getPreviousAttributes();
- }
-
- @Override
- public Map<String, String> getUpdatedAttributes() {
- return record.getUpdatedAttributes();
- }
-
- @Override
- public String getComponentId() {
- return record.getComponentId();
- }
-
- @Override
- public String getComponentType() {
- return record.getComponentType();
- }
-
- @Override
- public String getTransitUri() {
- return record.getTransitUri();
- }
-
- @Override
- public String getSourceSystemFlowFileIdentifier() {
- return record.getSourceSystemFlowFileIdentifier();
- }
-
- @Override
- public String getFlowFileUuid() {
- return record.getFlowFileUuid();
- }
-
- @Override
- public List<String> getParentUuids() {
- return record.getParentUuids();
- }
-
- @Override
- public List<String> getChildUuids() {
- return record.getChildUuids();
- }
-
- @Override
- public String getAlternateIdentifierUri() {
- return record.getAlternateIdentifierUri();
- }
-
- @Override
- public String getDetails() {
- return record.getDetails();
- }
-
- @Override
- public String getRelationship() {
- return record.getRelationship();
- }
-
- @Override
- public String getSourceQueueIdentifier() {
- return record.getSourceQueueIdentifier();
- }
-
- @Override
- public String getContentClaimSection() {
- return record.getContentClaimSection();
- }
-
- @Override
- public String getPreviousContentClaimSection() {
- return record.getPreviousContentClaimSection();
- }
-
- @Override
- public String getContentClaimContainer() {
- return record.getContentClaimContainer();
- }
-
- @Override
- public String getPreviousContentClaimContainer() {
- return record.getPreviousContentClaimContainer();
- }
-
- @Override
- public String getContentClaimIdentifier() {
- return record.getContentClaimIdentifier();
- }
-
- @Override
- public String getPreviousContentClaimIdentifier() {
- return record.getPreviousContentClaimIdentifier();
- }
-
- @Override
- public Long getContentClaimOffset() {
- return record.getContentClaimOffset();
- }
-
- @Override
- public Long getPreviousContentClaimOffset() {
- return record.getPreviousContentClaimOffset();
- }
+
+ @Override
+ public ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(final String queryIdentifier) {
+ throw new UnsupportedOperationException();
+ /*
+ return provQuerySubmissionMap.get(queryIdentifier);
+ */
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
index 3c3e401..fd27470 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
@@ -65,7 +65,7 @@
repo.registerEvent(builder.build());
}
- final List<ProvenanceEventRecord> retrieved = repo.getEvents(0L, 12);
+ final List<StoredProvenanceEvent> retrieved = repo.getEvents(0L, 12);
assertEquals(10, retrieved.size());
for (int i = 0; i < 10; i++) {
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml
index bdad811..d35ab72 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml
@@ -24,6 +24,7 @@
<modules>
<module>nifi-persistent-provenance-repository</module>
<module>nifi-volatile-provenance-repository</module>
+ <module>nifi-journaling-provenance-repository</module>
<module>nifi-provenance-repository-nar</module>
</modules>
<dependencyManagement>
@@ -38,6 +39,11 @@
<artifactId>nifi-volatile-provenance-repository</artifactId>
<version>0.0.2-incubating-SNAPSHOT</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-journaling-provenance-repository</artifactId>
+ <version>0.0.2-incubating-SNAPSHOT</version>
+ </dependency>
</dependencies>
</dependencyManagement>
</project>
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/pom.xml
index 5f6a1ba..debf202 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/pom.xml
@@ -39,6 +39,11 @@
<artifactId>nifi-processor-utils</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.nifi</groupId>
+ <artifactId>nifi-provenance-query-language</artifactId>
+ <version>0.0.2-incubating-SNAPSHOT</version>
+ </dependency>
+ <dependency>
<groupId>com.yammer.metrics</groupId>
<artifactId>metrics-ganglia</artifactId>
</dependency>
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/provenance/GenerateProvenanceReport.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/provenance/GenerateProvenanceReport.java
new file mode 100644
index 0000000..03f100b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/java/org/apache/nifi/provenance/GenerateProvenanceReport.java
@@ -0,0 +1,141 @@
+/*
+ * 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.provenance;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+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.pql.ProvenanceQuery;
+import org.apache.nifi.pql.exception.ProvenanceQueryLanguageParsingException;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
+import org.apache.nifi.reporting.AbstractReportingTask;
+import org.apache.nifi.reporting.ReportingContext;
+
+public class GenerateProvenanceReport extends AbstractReportingTask {
+
+ public static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
+ .name("Provenance Query")
+ .description("The Provenance Query to run against the repository")
+ .required(true)
+ .expressionLanguageSupported(false)
+ .addValidator(new ProvenanceQueryLanguageValidator())
+ .build();
+ public static final PropertyDescriptor DESTINATION_FILE = new PropertyDescriptor.Builder()
+ .name("Destination File")
+ .description("The file to write the results to. If not specified, the results will be written to the log")
+ .required(false)
+ .expressionLanguageSupported(false)
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+
+
+ @Override
+ protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+ final List<PropertyDescriptor> properties = new ArrayList<>();
+ properties.add(QUERY);
+ properties.add(DESTINATION_FILE);
+ return properties;
+ }
+
+
+ @Override
+ public void onTrigger(final ReportingContext context) {
+ try {
+ final long startNanos = System.nanoTime();
+ final ProvenanceQuerySubmission submission = context.getEventAccess().getProvenanceRepository().submitQuery(context.getProperty(QUERY).getValue());
+ final ProvenanceResultSet rs = submission.getResult().getResultSet();
+ final List<String> labels = rs.getLabels();
+
+ int length = 2;
+ for ( final String label : labels ) {
+ length += label.length() + 2;
+ }
+
+ final StringBuilder sb = new StringBuilder("\n");
+ for (int i=0; i < length; i++) {
+ sb.append("-");
+ }
+ sb.append("\n| ");
+
+ for ( final String label : labels ) {
+ sb.append(label).append(" |");
+ }
+ sb.append("\n");
+
+ for (int i=0; i < length; i++) {
+ sb.append("-");
+ }
+ sb.append("\n");
+
+ int rowCount = 0;
+ while (rs.hasNext()) {
+ final List<?> cols = rs.next();
+ for ( final Object col : cols ) {
+ sb.append("| ").append(col);
+ }
+ sb.append(" |\n");
+ rowCount++;
+ }
+
+ final String filename = context.getProperty(DESTINATION_FILE).getValue();
+ if ( filename == null ) {
+ getLogger().info(sb.toString());
+ } else {
+ final File file = new File(filename);
+ final File directory = file.getParentFile();
+ if ( !directory.exists() && !directory.mkdirs() ) {
+ throw new ProcessException("Cannot create directory " + directory + " to write to file");
+ }
+
+ try (final OutputStream fos = new FileOutputStream(file)) {
+ fos.write(sb.toString().getBytes(StandardCharsets.UTF_8));
+ }
+ }
+
+ final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+ getLogger().info("Successfully generated report with {} rows in the result; report generation took {} millis", new Object[] {rowCount, millis});
+ } catch (final IOException ioe) {
+ throw new ProcessException(ioe);
+ }
+ }
+
+ private static class ProvenanceQueryLanguageValidator implements Validator {
+ @Override
+ public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
+ try {
+ ProvenanceQuery.compile(input, null, null);
+ } catch (final ProvenanceQueryLanguageParsingException e) {
+ return new ValidationResult.Builder().input(input).subject(subject).valid(false).explanation(e.getMessage()).build();
+ }
+
+ return new ValidationResult.Builder().input(input).subject(subject).valid(true).build();
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask
index 1d3ec9a..a2ccdb6 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask
+++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/META-INF/services/org.apache.nifi.reporting.ReportingTask
@@ -12,7 +12,9 @@
# 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.
+
org.apache.nifi.controller.ControllerStatusReportingTask
org.apache.nifi.controller.MonitorDiskUsage
org.apache.nifi.controller.MonitorMemory
org.apache.nifi.reporting.ganglia.StandardGangliaReporter
+org.apache.nifi.provenance.GenerateProvenanceReport
\ No newline at end of file