You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2016/07/14 22:33:09 UTC

[03/10] nifi git commit: NIFI-1896 This closes #650. Refactored nifi-api into nifi-framework-api and other locations. The nifi-api is specific to that which is needed for intended extension points.

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.java
new file mode 100644
index 0000000..09202c0
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.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.controller.repository;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+
+/**
+ * Represents an abstraction of a FlowFile that can be used to track changing
+ * state of a FlowFile so that transactionality with repositories can exist
+ */
+public interface RepositoryRecord {
+
+    /**
+     * @return FlowFileQueue to which the FlowFile is to be transferred
+     */
+    FlowFileQueue getDestination();
+
+    /**
+     * @return FlowFileQueue from which the record was pulled
+     */
+    FlowFileQueue getOriginalQueue();
+
+    /**
+     * @return type of update that this record encapsulates
+     */
+    RepositoryRecordType getType();
+
+    /**
+     * @return current ContentClaim for the FlowFile
+     */
+    ContentClaim getCurrentClaim();
+
+    /**
+     * @return original ContentClaim for the FlowFile before any changes were made
+     */
+    ContentClaim getOriginalClaim();
+
+    /**
+     * @return byte offset into the Content Claim where this FlowFile's content
+     * begins
+     */
+    long getCurrentClaimOffset();
+
+    /**
+     * @return FlowFile being encapsulated by this record
+     */
+    FlowFileRecord getCurrent();
+
+    /**
+     * @return Whether or not the FlowFile's attributes have changed since the FlowFile
+     * was pulled from its queue (or created)
+     */
+    boolean isAttributesChanged();
+
+    /**
+     * @return <code>true</code> if the FlowFile is not viable and should be
+     * aborted (e.g., the content of the FlowFile cannot be found)
+     */
+    boolean isMarkedForAbort();
+
+    /**
+     * @return If the FlowFile is swapped out of the Java heap space, provides the
+     * location of the swap file, or <code>null</code> if the FlowFile is not
+     * swapped out
+     */
+    String getSwapLocation();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.java
new file mode 100644
index 0000000..ff8dc50
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.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.controller.repository;
+
+/**
+ *
+ */
+public enum RepositoryRecordType {
+
+    UPDATE, CREATE, DELETE, CONTENTMISSING, SWAP_IN, SWAP_OUT;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapContents.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapContents.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapContents.java
new file mode 100644
index 0000000..2fb7ba4
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapContents.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.controller.repository;
+
+import java.util.List;
+
+/**
+ * When FlowFiles are constructed from a Swap File, there is information in addition to
+ * the FlowFiles themselves that get stored and recovered. SwapContents provides a
+ * mechanism to encapsulate all of the information contained within a Swap File in a
+ * single object
+ */
+public interface SwapContents {
+
+    /**
+     * @return a summary of information included in a Swap File
+     */
+    SwapSummary getSummary();
+
+    /**
+     * @return the FlowFiles that are contained within a Swap File
+     */
+    List<FlowFileRecord> getFlowFiles();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapManagerInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapManagerInitializationContext.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapManagerInitializationContext.java
new file mode 100644
index 0000000..0e30784
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapManagerInitializationContext.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.controller.repository;
+
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.events.EventReporter;
+
+public interface SwapManagerInitializationContext {
+
+    /**
+     * @return the {@link FlowFileRepository} that should be updated when FlowFiles are swapped in and out
+     */
+    FlowFileRepository getFlowFileRepository();
+
+    /**
+     * @return the {@link ResourceClaimManager} that is necessary to provide to the FlowFileRepository when
+     *         performing swapping actions
+     */
+    ResourceClaimManager getResourceClaimManager();
+
+    /**
+     * @return an {@link EventReporter} that can be used to report events to users
+     */
+    EventReporter getEventReporter();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapSummary.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapSummary.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapSummary.java
new file mode 100644
index 0000000..ad01ccd
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/SwapSummary.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.controller.repository;
+
+import java.util.List;
+
+import org.apache.nifi.controller.queue.QueueSize;
+import org.apache.nifi.controller.repository.claim.ResourceClaim;
+
+/**
+ * <p>
+ * Provides a summary of the information that is stored in a FlowFile swap file.
+ * </p>
+ */
+public interface SwapSummary {
+    /**
+     * @return a QueueSize that represents the number of FlowFiles are in the swap file and their
+     *         aggregate content size
+     */
+    QueueSize getQueueSize();
+
+    /**
+     * @return the largest ID of any of the FlowFiles that are contained in the swap file
+     */
+    Long getMaxFlowFileId();
+
+    /**
+     * Returns a List of all ResoruceClaims that are referenced by the FlowFiles in the swap file.
+     * This List may well contain the same ResourceClaim many times. This indicates that many FlowFiles
+     * reference the same ResourceClaim.
+     *
+     * @return a List of all ResourceClaims that are referenced by the FlowFiles in the swap file
+     */
+    List<ResourceClaim> getResourceClaims();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaim.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaim.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaim.java
new file mode 100644
index 0000000..5c1d76b
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaim.java
@@ -0,0 +1,47 @@
+/*
+ * 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.controller.repository.claim;
+
+/**
+ * <p>
+ * A reference to a section of a {@link ResourceClaim}, which may or may not encompass
+ * the entire ResourceClaim. Multiple FlowFiles may reference the same content by both
+ * having the same content claim.
+ * </p>
+ *
+ * <p>
+ * Must be thread safe
+ * </p>
+ */
+public interface ContentClaim extends Comparable<ContentClaim> {
+
+    /**
+     * @return the ResourceClaim that this ContentClaim references
+     */
+    ResourceClaim getResourceClaim();
+
+    /**
+     * @return the offset into the ResourceClaim where the content for this
+     * claim begins
+     */
+    long getOffset();
+
+    /**
+     * @return the length of this ContentClaim
+     */
+    long getLength();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaim.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaim.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaim.java
new file mode 100644
index 0000000..77e7f84
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaim.java
@@ -0,0 +1,54 @@
+/*
+ * 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.controller.repository.claim;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+
+/**
+ * <p>
+ * Represents a resource that can be provided by a {@link ContentRepository}
+ * </p>
+ *
+ * <p>
+ * MUST BE THREAD-SAFE!
+ * </p>
+ */
+public interface ResourceClaim extends Comparable<ResourceClaim> {
+
+    /**
+     * @return the unique identifier for this claim
+     */
+    String getId();
+
+    /**
+     * @return the container identifier in which this claim is held
+     */
+    String getContainer();
+
+    /**
+     * @return the section within a given container the claim is held
+     */
+    String getSection();
+
+    /**
+     * @return Indicates whether or not the Claim is loss-tolerant. If so, we will
+     *         attempt to keep the content but will not sacrifice a great deal of
+     *         performance to do so
+     */
+    boolean isLossTolerant();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaimManager.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaimManager.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaimManager.java
new file mode 100644
index 0000000..01f4c65
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/claim/ResourceClaimManager.java
@@ -0,0 +1,135 @@
+/*
+ * 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.controller.repository.claim;
+
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Responsible for managing all ResourceClaims that are used in the application
+ */
+public interface ResourceClaimManager {
+
+    /**
+     * Creates a new Resource Claim with the given id, container, section, and
+     * loss tolerance.
+     *
+     * @param id of claim
+     * @param container of claim
+     * @param section of claim
+     * @param lossTolerant of claim
+     * @return new claim
+     */
+    ResourceClaim newResourceClaim(String container, String section, String id, boolean lossTolerant);
+
+    /**
+     * @param claim to obtain reference count for
+     * @return the number of FlowFiles that hold a claim to a particular piece
+     * of FlowFile content
+     */
+    int getClaimantCount(ResourceClaim claim);
+
+    /**
+     * Decreases by 1 the count of how many FlowFiles hold a claim to a
+     * particular piece of FlowFile content and returns the new count
+     *
+     * @param claim to decrement claimants on
+     * @return new claimaint count
+     */
+    int decrementClaimantCount(ResourceClaim claim);
+
+    /**
+     * Increases by 1 the count of how many FlowFiles hold a claim to a
+     * particular piece of FlowFile content and returns the new count
+     *
+     * @param claim to increment claims on
+     * @return new claimant count
+     */
+    int incrementClaimantCount(ResourceClaim claim);
+
+    /**
+     * Increases by 1 the count of how many FlowFiles hold a claim to a
+     * particular piece of FlowFile content and returns the new count.
+     *
+     * If it is known that the Content Claim whose count is being incremented is
+     * a newly created ResourceClaim, this method should be called with a value
+     * of {@code true} as the second argument, as it may allow the manager to
+     * optimize its tasks, knowing that the Content Claim cannot be referenced
+     * by any other component
+     *
+     * @param claim to increment
+     * @param newClaim provides a hint that no other process can have access to this
+     *            claim right now
+     * @return new claim count
+     */
+    int incrementClaimantCount(ResourceClaim claim, boolean newClaim);
+
+    /**
+     * Indicates that the given ResourceClaim can now be destroyed by the
+     * appropriate Content Repository. This should be done only after it is
+     * guaranteed that the FlowFile Repository has been synchronized with its
+     * underlying storage component. This way, we avoid the following sequence
+     * of events:
+     * <ul>
+     * <li>FlowFile Repository is updated to indicate that FlowFile F no longer depends on ResourceClaim C</li>
+     * <li>ResourceClaim C is no longer needed and is destroyed</li>
+     * <li>The Operating System crashes or there is a power failure</li>
+     * <li>Upon restart, the FlowFile Repository was not synchronized with its underlying storage mechanism and as such indicates that FlowFile F needs ResourceClaim C.</li>
+     * <li>Since ResourceClaim C has already been destroyed, it is inaccessible, and FlowFile F's Content is not found, so the FlowFile is removed, resulting in data loss.</li>
+     * </ul>
+     *
+     * <p>
+     * Using this method of marking the ResourceClaim as destructable only when the FlowFile repository has been synced with the underlying storage mechanism, we can ensure that on restart, we will
+     * not point to this unneeded claim. As such, it is now safe to destroy the contents.
+     * </p>
+     *
+     * @param claim to mark as now destructable
+     */
+    void markDestructable(ResourceClaim claim);
+
+    /**
+     * Drains up to {@code maxElements} Content Claims from the internal queue
+     * of destructable content claims to the given {@code destination} so that
+     * they can be destroyed.
+     *
+     * @param destination to drain to
+     * @param maxElements max items to drain
+     */
+    void drainDestructableClaims(Collection<ResourceClaim> destination, int maxElements);
+
+    /**
+     * Drains up to {@code maxElements} Content Claims from the internal queue
+     * of destructable content claims to the given {@code destination} so that
+     * they can be destroyed. If no ResourceClaim is ready to be destroyed at
+     * this time, will wait up to the specified amount of time before returning.
+     * If, after the specified amount of time, there is still no ResourceClaim
+     * ready to be destroyed, the method will return without having added
+     * anything to the given {@code destination}.
+     *
+     * @param destination to drain to
+     * @param maxElements max items to drain
+     * @param timeout maximum time to wait
+     * @param unit unit of time to wait
+     */
+    void drainDestructableClaims(Collection<ResourceClaim> destination, int maxElements, long timeout, TimeUnit unit);
+
+    /**
+     * Clears the manager's memory of any and all ResourceClaims that it knows
+     * about
+     */
+    void purge();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java
new file mode 100644
index 0000000..d273096
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java
@@ -0,0 +1,120 @@
+/*
+ * 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.controller.status.history;
+
+import java.util.Date;
+
+import org.apache.nifi.controller.status.ProcessGroupStatus;
+
+/**
+ * A repository for storing and retrieving components' historical status
+ * information
+ */
+public interface ComponentStatusRepository {
+
+    /**
+     * Captures the status information provided in the given report
+     *
+     * @param rootGroupStatus status of root group
+     */
+    void capture(ProcessGroupStatus rootGroupStatus);
+
+    /**
+     * Captures the status information provided in the given report, providing a
+     * timestamp that indicates the time at which the status report was
+     * generated. This can be used to replay historical values.
+     *
+     * @param rootGroupStatus status
+     * @param timestamp timestamp of capture
+     */
+    void capture(ProcessGroupStatus rootGroupStatus, Date timestamp);
+
+    /**
+     * @return the Date at which the latest capture was performed
+     */
+    Date getLastCaptureDate();
+
+    /**
+     * @param connectionId the ID of the Connection for which the Status is
+     * desired
+     * @param start the earliest date for which status information should be
+     * returned; if <code>null</code>, the start date should be assumed to be
+     * the beginning of time
+     * @param end the latest date for which status information should be
+     * returned; if <code>null</code>, the end date should be assumed to be the
+     * current time
+     * @param preferredDataPoints the preferred number of data points to return.
+     * If the date range is large, the total number of data points could be far
+     * too many to process. Therefore, this parameter allows the requestor to
+     * indicate how many samples to return.
+     * @return a {@link StatusHistory} that provides the status information
+     * about the Connection with the given ID during the given time period
+     */
+    StatusHistory getConnectionStatusHistory(String connectionId, Date start, Date end, int preferredDataPoints);
+
+    /**
+     * @param processGroupId of group to get status of
+     * @param start the earliest date for which status information should be
+     * returned; if <code>null</code>, the start date should be assumed to be
+     * the beginning of time
+     * @param end the latest date for which status information should be
+     * returned; if <code>null</code>, the end date should be assumed to be the
+     * current time
+     * @param preferredDataPoints the preferred number of data points to return.
+     * If the date range is large, the total number of data points could be far
+     * too many to process. Therefore, this parameter allows the requestor to
+     * indicate how many samples to return.
+     * @return a {@link StatusHistory} that provides the status information
+     * about the Process Group with the given ID during the given time period
+     */
+    StatusHistory getProcessGroupStatusHistory(String processGroupId, Date start, Date end, int preferredDataPoints);
+
+    /**
+     * @param processorId to get status of
+     * @param start the earliest date for which status information should be
+     * returned; if <code>null</code>, the start date should be assumed to be
+     * the beginning of time
+     * @param end the latest date for which status information should be
+     * returned; if <code>null</code>, the end date should be assumed to be the
+     * current time
+     * @param preferredDataPoints the preferred number of data points to return.
+     * If the date range is large, the total number of data points could be far
+     * too many to process. Therefore, this parameter allows the requestor to
+     * indicate how many samples to return.
+     * @return a {@link StatusHistory} that provides the status information
+     * about the Processor with the given ID during the given time period
+     */
+    StatusHistory getProcessorStatusHistory(String processorId, Date start, Date end, int preferredDataPoints);
+
+    /**
+     * @param remoteGroupId to get history of
+     * @param start the earliest date for which status information should be
+     * returned; if <code>null</code>, the start date should be assumed to be
+     * the beginning of time
+     * @param end the latest date for which status information should be
+     * returned; if <code>null</code>, the end date should be assumed to be the
+     * current time
+     * @param preferredDataPoints the preferred number of data points to return.
+     * If the date range is large, the total number of data points could be far
+     * too many to process. Therefore, this parameter allows the requestor to
+     * indicate how many samples to return.
+     * @return a {@link StatusHistory} that provides the status information
+     * about the Remote Process Group with the given ID during the given time
+     * period
+     */
+    StatusHistory getRemoteProcessGroupStatusHistory(String remoteGroupId, Date start, Date end, int preferredDataPoints);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.java
new file mode 100644
index 0000000..8fdce05
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.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.controller.status.history;
+
+/**
+ * Describes a particular metric that is derived from a Status History
+ *
+ * @param <T> type of metric
+ */
+public interface MetricDescriptor<T> {
+
+    public enum Formatter {
+
+        COUNT,
+        DURATION,
+        DATA_SIZE
+    };
+
+    /**
+     * Specifies how the values should be formatted
+     *
+     * @return formatter for values
+     */
+    Formatter getFormatter();
+
+    /**
+     * @return a human-readable description of the field
+     */
+    String getDescription();
+
+    /**
+     * @return a human-readable label for the field
+     */
+    String getLabel();
+
+    /**
+     * @return the name of a field
+     */
+    String getField();
+
+    /**
+     * @return a {@link ValueMapper} that can be used to extract a value for the
+     * status history
+     */
+    ValueMapper<T> getValueFunction();
+
+    /**
+     * @return a {@link ValueReducer} that can reduce multiple StatusSnapshots
+     * into a single Long value
+     */
+    ValueReducer<StatusSnapshot, Long> getValueReducer();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java
new file mode 100644
index 0000000..f1bb946
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.controller.status.history;
+
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Represents a collection of historical status values for a component
+ */
+public interface StatusHistory {
+
+    /**
+     * @return a Date indicating when this report was generated
+     */
+    Date getDateGenerated();
+
+    /**
+     * @return a Map of component field names and their values. The order in
+     * which these values are displayed is dependent on the natural ordering of
+     * the Map returned
+     */
+    Map<String, String> getComponentDetails();
+
+    /**
+     * @return List of snapshots for a given component
+     */
+    List<StatusSnapshot> getStatusSnapshots();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java
new file mode 100644
index 0000000..551ceb2
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java
@@ -0,0 +1,42 @@
+/*
+ * 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.controller.status.history;
+
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * A StatusSnapshot represents a Component's status report at some point in time
+ */
+public interface StatusSnapshot {
+
+    /**
+     * @return the point in time for which the status values were obtained
+     */
+    Date getTimestamp();
+
+    /**
+     * @return a Map of MetricDescriptor to value
+     */
+    Map<MetricDescriptor<?>, Long> getStatusMetrics();
+
+    /**
+     * @return a {@link ValueReducer} that is capable of merging multiple
+     * StatusSnapshot objects into a single one
+     */
+    ValueReducer<StatusSnapshot, StatusSnapshot> getValueReducer();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.java
new file mode 100644
index 0000000..8000b3a
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.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.controller.status.history;
+
+public interface ValueMapper<S> {
+
+    Long getValue(S status);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.java
new file mode 100644
index 0000000..0427da7
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.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.controller.status.history;
+
+import java.util.List;
+
+public interface ValueReducer<T, R> {
+
+    R reduce(List<T> values);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/events/EventReporter.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/events/EventReporter.java b/nifi-framework-api/src/main/java/org/apache/nifi/events/EventReporter.java
new file mode 100644
index 0000000..d645d60
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/events/EventReporter.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.events;
+
+import java.io.Serializable;
+
+import org.apache.nifi.reporting.Severity;
+
+/**
+ * Implementations MUST be thread-safe
+ */
+public interface EventReporter extends Serializable {
+
+    void reportEvent(Severity severity, String category, String message);
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.java b/nifi-framework-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.java
new file mode 100644
index 0000000..684f454
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.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.flowfile;
+
+import java.util.Comparator;
+
+/**
+ * Provides a mechanism to prioritize flow file objects based on their
+ * attributes. The actual flow file content will not be available for comparison
+ * so if features of that content are necessary for prioritization it should be
+ * extracted to be used as an attribute of the flow file.
+ *
+ */
+public interface FlowFilePrioritizer extends Comparator<FlowFile> {
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java
new file mode 100644
index 0000000..23d6b3d
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.authorization.resource.Authorizable;
+import org.apache.nifi.web.ResourceNotFoundException;
+
+public interface ProvenanceAuthorizableFactory {
+
+    /**
+     * Generates an Authorizable object for the Provenance events of the component with the given ID
+     *
+     * @param componentId the ID of the component to which the Provenance events belong
+     *
+     * @return the Authorizable that can be use to authorize access to provenance events
+     * @throws ResourceNotFoundException if no component can be found with the given ID
+     */
+    Authorizable createProvenanceAuthorizable(String componentId);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java
new file mode 100644
index 0000000..7ac1a65
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java
@@ -0,0 +1,160 @@
+/*
+ * 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.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.search.SearchableField;
+
+import java.io.IOException;
+import java.util.List;
+
+public interface ProvenanceRepository extends ProvenanceEventRepository {
+
+    /**
+     * Performs any initialization needed. This should be called only by the
+     * framework.
+     *
+     * @param eventReporter   to report to
+     * @param authorizer      the authorizer to use for authorizing individual events
+     * @param resourceFactory the resource factory to use for generating Provenance Resource objects for authorization purposes
+     * @throws java.io.IOException if unable to initialize
+     */
+    void initialize(EventReporter eventReporter, Authorizer authorizer, ProvenanceAuthorizableFactory resourceFactory) throws IOException;
+
+
+    ProvenanceEventRecord getEvent(long id, NiFiUser user) throws IOException;
+
+    /**
+     * Returns a List of all <code>ProvenanceEventRecord</code>s in the
+     * repository starting with the given ID. The first ID in the repository
+     * will always be 0 or higher. Each event that is found will be authorized
+     * against the given NiFiUser. If the user does not have authorization for
+     * the event, the event will not be returned.
+     *
+     * @param firstRecordId id of the first record to retrieve
+     * @param maxRecords    maximum number of records to retrieve
+     * @param user          the NiFi user that the events should be authorized against
+     * @return records
+     * @throws java.io.IOException if error reading from repository
+     */
+    List<ProvenanceEventRecord> getEvents(long firstRecordId, final int maxRecords, NiFiUser user) throws IOException;
+
+    /**
+     * @return the {@link ProvenanceEventRepository} backing this ProvenanceRepository
+     */
+    ProvenanceEventRepository getProvenanceEventRepository();
+
+    /**
+     * 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 to submit
+     * @param user the NiFi User to authorize the events against
+     *
+     * @return an identifier that can be used to fetch the results at a later
+     *         time
+     */
+    QuerySubmission submitQuery(Query query, NiFiUser user);
+
+    /**
+     * @param queryIdentifier of the query
+     * @param user the user who is retrieving the query
+     *
+     * @return 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>
+     */
+    QuerySubmission retrieveQuerySubmission(String queryIdentifier, NiFiUser user);
+
+    /**
+     * 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. If the given user does not
+     * have authorization to view one of the events in the lineage, a placeholder
+     * event will be used instead that provides none of the event details except
+     * for the identifier of the component that emitted the Provenance Event. It is
+     * necessary to include this node in the lineage view so that the lineage makes
+     * sense, rather than showing disconnected graphs when the user is not authorized
+     * for all components' provenance events.
+     *
+     * @param flowFileUuid the UUID of the FlowFile for which the Lineage should
+     *            be calculated
+     * @param user the NiFi User to authorize events against
+     *
+     * @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, NiFiUser user);
+
+    /**
+     * @param lineageIdentifier identifier of lineage to compute
+     * @param user the user who is retrieving the lineage submission
+     *
+     * @return the {@link ComputeLineageSubmission} associated with the given
+     *         identifier
+     */
+    ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier, NiFiUser user);
+
+    /**
+     * Submits a request to expand the parents of the event with the given id. If the given user
+     * is not authorized to access any event, a placeholder will be used instead that contains only
+     * the ID of the component that emitted the event.
+     *
+     * @param eventId the one-up id of the Event to expand
+     * @param user the NiFi user to authorize events against
+     * @return a submission which can be checked for status
+     *
+     * @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(long eventId, NiFiUser user);
+
+    /**
+     * Submits a request to expand the children of the event with the given id. If the given user
+     * is not authorized to access any event, a placeholder will be used instead that contains only
+     * the ID of the component that emitted the event.
+     *
+     * @param eventId the one-up id of the Event
+     * @param user the NiFi user to authorize events against
+     *
+     * @return a submission which can be checked for status
+     *
+     * @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(long eventId, NiFiUser user);
+
+    /**
+     * @return a list of all fields that can be searched via the
+     * {@link ProvenanceRepository#submitQuery(Query, NiFiUser)} method
+     */
+    List<SearchableField> getSearchableFields();
+
+    /**
+     * @return a list of all FlowFile attributes that can be searched via the
+     * {@link ProvenanceRepository#submitQuery(Query, NiFiUser)} method
+     */
+    List<SearchableField> getSearchableAttributes();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java
new file mode 100644
index 0000000..4d0f991
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.lineage;
+
+import java.util.Date;
+import java.util.List;
+
+/**
+ *
+ */
+public interface ComputeLineageResult {
+
+    /**
+     * @return all nodes for the graph
+     */
+    List<LineageNode> getNodes();
+
+    /**
+     * @return all links for the graph
+     */
+    List<LineageEdge> getEdges();
+
+    /**
+     * @return the date at which this AsynchronousLineageResult will expire
+     */
+    Date getExpiration();
+
+    /**
+     * @return If an error occurred while computing the lineage, this will return the
+     * serialized error; otherwise, returns <code>null</code>
+     */
+    String getError();
+
+    /**
+     * @return an integer between 0 and 100 (inclusive) that indicates what
+     * percentage of completion the computation has reached
+     */
+    int getPercentComplete();
+
+    /**
+     * @return Indicates whether or not the lineage has finished running
+     */
+    boolean isFinished();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.java
new file mode 100644
index 0000000..8a11523
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.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.lineage;
+
+import java.util.Collection;
+import java.util.Date;
+
+public interface ComputeLineageSubmission {
+
+    /**
+     * @return the {@link ComputeLineageResult} that contains the results. The
+     * results may be partial if a call to
+     * {@link ComputeLineageResult#isFinished()} returns <code>false</code>
+     */
+    ComputeLineageResult getResult();
+
+    /**
+     * @return the date at which this lineage was submitted
+     */
+    Date getSubmissionTime();
+
+    /**
+     * @return the identity of the user who submitted the request
+     */
+    String getSubmitterIdentity();
+
+    /**
+     * @return the generated identifier for this lineage result
+     */
+    String getLineageIdentifier();
+
+    /**
+     * Cancels the lineage computation
+     */
+    void cancel();
+
+    /**
+     * @return <code>true</code> if {@link #cancel()} has been called,
+     * <code>false</code> otherwise
+     */
+    boolean isCanceled();
+
+    /**
+     * @return the type of Lineage Computation that was submitted
+     */
+    LineageComputationType getLineageComputationType();
+
+    /**
+     * @return If the Lineage Computation Type of this submission is
+     * {@link LineageComputationType.EXPAND_CHILDREN} or
+     * {@link LineageComputationType.EXPAND_PARENTS}, indicates the ID event
+     * that is to be expanded; otherwise, returns <code>null</code>
+     */
+    Long getExpandedEventId();
+
+    /**
+     * @return all FlowFile UUID's that are encapsulated in this lineage
+     * computation submission
+     */
+    Collection<String> getLineageFlowFileUuids();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java
new file mode 100644
index 0000000..ff5fee7
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java
@@ -0,0 +1,37 @@
+/*
+ * 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.lineage;
+
+import java.util.List;
+
+/**
+ * A Data Structure for representing a Directed Graph that depicts the lineage
+ * of a FlowFile and all events that occurred for the FlowFile
+ */
+public interface Lineage {
+
+    /**
+     * @return all nodes for the graph
+     */
+    public List<LineageNode> getNodes();
+
+    /**
+     * @return all links for the graph
+     */
+    public List<LineageEdge> getEdges();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.java
new file mode 100644
index 0000000..be74324
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.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.lineage;
+
+/**
+ *
+ */
+public enum LineageComputationType {
+
+    FLOWFILE_LINEAGE,
+    EXPAND_PARENTS,
+    EXPAND_CHILDREN;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java
new file mode 100644
index 0000000..dba56f3
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java
@@ -0,0 +1,26 @@
+/*
+ * 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.lineage;
+
+public interface LineageEdge {
+
+    String getUuid();
+
+    LineageNode getSource();
+
+    LineageNode getDestination();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java
new file mode 100644
index 0000000..5d0956e
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.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.lineage;
+
+public interface LineageNode {
+
+    /**
+     * @return the type of the LineageNode
+     */
+    LineageNodeType getNodeType();
+
+    /**
+     * @return the UUID of the FlowFile for which this Node was created
+     */
+    String getFlowFileUuid();
+
+    /**
+     * @return the UUID for this LineageNode
+     */
+    String getIdentifier();
+
+    /**
+     * @return the timestamp that corresponds to this Node. The meaning of the
+     * timestamp may differ between implementations. For example, a
+     * {@link ProvenanceEventLineageNode}'s timestamp indicates the time at
+     * which the event occurred. However, for a Node that reperesents a
+     * FlowFile, for example, the timestamp may represent the time at which the
+     * FlowFile was created
+     */
+    long getTimestamp();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.java
new file mode 100644
index 0000000..67e0d61
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.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.lineage;
+
+public enum LineageNodeType {
+
+    FLOWFILE_NODE,
+    PROVENANCE_EVENT_NODE;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java
new file mode 100644
index 0000000..f490496
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java
@@ -0,0 +1,32 @@
+/*
+ * 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.lineage;
+
+import java.util.List;
+
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+public interface ProvenanceEventLineageNode extends LineageNode {
+
+    ProvenanceEventType getEventType();
+
+    long getEventIdentifier();
+
+    List<String> getParentUuids();
+
+    List<String> getChildUuids();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.java
new file mode 100644
index 0000000..3519c14
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.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.search;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Objects;
+
+
+public class Query {
+
+    private final String identifier;
+    private final List<SearchTerm> searchTerms = new ArrayList<>();
+    private Date startDate;
+    private Date endDate;
+    private String minFileSize;
+    private String maxFileSize;
+    private int maxResults = 1000;
+
+    public Query(final String identifier) {
+        this.identifier = Objects.requireNonNull(identifier);
+    }
+
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    public void addSearchTerm(final SearchTerm searchTerm) {
+        searchTerms.add(searchTerm);
+    }
+
+    public List<SearchTerm> getSearchTerms() {
+        return Collections.unmodifiableList(searchTerms);
+    }
+
+    public Date getStartDate() {
+        return startDate;
+    }
+
+    public void setStartDate(Date startDate) {
+        this.startDate = startDate;
+    }
+
+    public Date getEndDate() {
+        return endDate;
+    }
+
+    public void setEndDate(Date endDate) {
+        this.endDate = endDate;
+    }
+
+    public int getMaxResults() {
+        return maxResults;
+    }
+
+    public void setMaxResults(int maxResults) {
+        this.maxResults = maxResults;
+    }
+
+    public void setMinFileSize(final String fileSize) {
+        this.minFileSize = fileSize;
+    }
+
+    public String getMinFileSize() {
+        return minFileSize;
+    }
+
+    public void setMaxFileSize(final String fileSize) {
+        this.maxFileSize = fileSize;
+    }
+
+    public String getMaxFileSize() {
+        return maxFileSize;
+    }
+
+    @Override
+    public String toString() {
+        return "Query[ " + searchTerms + " ]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
new file mode 100644
index 0000000..0079433
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.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.search;
+
+import java.util.Date;
+import java.util.List;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface QueryResult {
+
+    /**
+     * @return the Provenance events that match the query (up to the limit
+     * specified in the query)
+     */
+    List<ProvenanceEventRecord> getMatchingEvents();
+
+    /**
+     * @return the total number of Provenance Events that hit
+     */
+    long getTotalHitCount();
+
+    /**
+     * @return the number of milliseconds the query took to run
+     */
+    long getQueryTime();
+
+    /**
+     * @return the date at which this QueryResult will expire
+     */
+    Date getExpiration();
+
+    /**
+     * @return If an error occurred while computing the lineage, this will return the
+     * serialized error; otherwise, returns <code>null</code>
+     */
+    String getError();
+
+    /**
+     * @return an integer between 0 and 100 (inclusive) that indicates what
+     * percentage of completion the query has reached
+     */
+    int getPercentComplete();
+
+    /**
+     * @return Indicates whether or not the query has finished running
+     */
+    boolean isFinished();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.java
new file mode 100644
index 0000000..45efe4b
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.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.search;
+
+import java.util.Date;
+
+public interface QuerySubmission {
+
+    /**
+     * @return the query that this submission pertains to
+     */
+    Query getQuery();
+
+    /**
+     * @return 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>
+     */
+    QueryResult getResult();
+
+    /**
+     * @return the date at which this query was submitted
+     */
+    Date getSubmissionTime();
+
+    /**
+     * @return the generated identifier for this query result
+     */
+    String getQueryIdentifier();
+
+    /**
+     * Cancels the query
+     */
+    void cancel();
+
+    /**
+     * @return <code>true</code> if {@link #cancel()} has been called,
+     * <code>false</code> otherwise
+     */
+    boolean isCanceled();
+
+    /**
+     * @return the identity of the user who submitted the query
+     */
+    String getSubmitterIdentity();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java
new file mode 100644
index 0000000..573dbd7
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java
@@ -0,0 +1,24 @@
+/*
+ * 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.search;
+
+public interface SearchTerm {
+
+    SearchableField getSearchableField();
+
+    String getValue();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.java
new file mode 100644
index 0000000..aad4d7b
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.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.provenance.search;
+
+public class SearchTerms {
+
+    public static SearchTerm newSearchTerm(final SearchableField field, final String value) {
+        return new SearchTerm() {
+            @Override
+            public SearchableField getSearchableField() {
+                return field;
+            }
+
+            @Override
+            public String getValue() {
+                return value;
+            }
+
+            @Override
+            public String toString() {
+                return getValue();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java
new file mode 100644
index 0000000..85c6154
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java
@@ -0,0 +1,52 @@
+/*
+ * 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.search;
+
+/**
+ * A SearchableField represents a field in a Provenance Event that can be
+ * searched
+ */
+public interface SearchableField {
+
+    /**
+     * @return the identifier that is used to refer to this field
+     */
+    String getIdentifier();
+
+    /**
+     * @return the name of the field that is used when searching the repository
+     */
+    String getSearchableFieldName();
+
+    /**
+     * @return the "friendly" name or "display name" of the field, which may be
+     * more human-readable than the searchable field name
+     */
+    String getFriendlyName();
+
+    /**
+     * @return the type of the data stored in this field
+     */
+    SearchableFieldType getFieldType();
+
+    /**
+     * @return <code>true</code> if this field represents a FlowFile attribute,
+     * <code>false</code> if the field represents a Provenance Event detail,
+     * such as Source System URI
+     */
+    boolean isAttribute();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java
new file mode 100644
index 0000000..51ece58
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java
@@ -0,0 +1,28 @@
+/*
+ * 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.search;
+
+/**
+ *
+ */
+public enum SearchableFieldType {
+
+    STRING,
+    DATE,
+    DATA_SIZE,
+    LONG;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java b/nifi-framework-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
new file mode 100644
index 0000000..c3a34b2
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/remote/RemoteDestination.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.remote;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A model object for referring to a remote destination (i.e., a Port) for
+ * site-to-site communications
+ */
+public interface RemoteDestination {
+
+    /**
+     * @return the identifier of the remote destination
+     */
+    String getIdentifier();
+
+    /**
+     * @return the human-readable name of the remote destination
+     */
+    String getName();
+
+    /**
+     * @param timeUnit to yield
+     * @return the amount of time that system should pause sending to a
+     * particular node if unable to send data to or receive data from this
+     * endpoint
+     */
+    long getYieldPeriod(TimeUnit timeUnit);
+
+    /**
+     * @return whether or not compression should be used when transferring data
+     * to or receiving data from the remote endpoint
+     */
+    boolean isUseCompression();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-framework-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java b/nifi-framework-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java
new file mode 100644
index 0000000..ee5f417
--- /dev/null
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/web/ClusterRequestException.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.web;
+
+/**
+ * An general error occurred when attempting to communicate with the cluster.
+ */
+public class ClusterRequestException extends RuntimeException {
+
+    public ClusterRequestException(Throwable cause) {
+        super(cause);
+    }
+
+    public ClusterRequestException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ClusterRequestException(String message) {
+        super(message);
+    }
+
+    public ClusterRequestException() {
+    }
+
+}