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:13 UTC

[07/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-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java
deleted file mode 100644
index d273096..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.java
deleted file mode 100644
index 8fdce05..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/MetricDescriptor.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java
deleted file mode 100644
index f1bb946..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/StatusHistory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java
deleted file mode 100644
index 551ceb2..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/StatusSnapshot.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.java
deleted file mode 100644
index 8000b3a..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ValueMapper.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.java
deleted file mode 100644
index 0427da7..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/history/ValueReducer.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/events/EventReporter.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java b/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java
deleted file mode 100644
index d645d60..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java b/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java
index 0f4d2d0..7d0e27e 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFile.java
@@ -68,7 +68,7 @@ public interface FlowFile extends Comparable<FlowFile> {
      * @return the time at which the FlowFile was most recently added to a
      * FlowFile queue, or {@code null} if the FlowFile has never been enqueued.
      * This value will always be populated before it is passed to a
-     * {@link FlowFilePrioritizer}
+     * FlowFilePrioritizer
      */
     Long getLastQueueDate();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.java b/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.java
deleted file mode 100644
index 684f454..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/flowfile/FlowFilePrioritizer.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java
deleted file mode 100644
index 23d6b3d..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceAuthorizableFactory.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
index 235a1fa..14991ad 100644
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
+++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
@@ -19,16 +19,6 @@ package org.apache.nifi.provenance;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.nifi.authorization.AccessDeniedException;
-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 org.apache.nifi.web.ResourceNotFoundException;
-
 /**
  * This Repository houses Provenance Events. The repository is responsible for
  * managing the life-cycle of the events, providing access to the events that it
@@ -38,18 +28,6 @@ import org.apache.nifi.web.ResourceNotFoundException;
 public interface 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;
-
-    /**
      * Returns a {@link ProvenanceEventBuilder} that is capable of building
      * {@link ProvenanceEventRecord}s
      *
@@ -91,21 +69,6 @@ public interface ProvenanceEventRepository {
      */
     List<ProvenanceEventRecord> getEvents(long firstRecordId, final int maxRecords) 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 largest ID of any event that is queryable in the repository.
@@ -114,57 +77,6 @@ public interface ProvenanceEventRepository {
     Long getMaxEventId();
 
     /**
-     * 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);
-
-    /**
      * Retrieves the Provenance Event with the given ID. The event will be returned only
      * if the given user is authorized to access the event. Otherwise, an
      * AccessDeniedException or ResourceNotFoundException will be thrown, as appropriate
@@ -173,41 +85,8 @@ public interface ProvenanceEventRepository {
      * @return the Provenance Event Record with the given ID, if it exists, or
      *         {@code null} otherwise
      * @throws IOException if failure while retrieving event
-     * @throws AccessDeniedException if the user does not have access to the component
-     * @throws ResourceNotFoundException if the component that the event belongs to cannot be found
-     */
-    ProvenanceEventRecord getEvent(long id, NiFiUser user) throws IOException;
-
-    /**
-     * 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);
+    ProvenanceEventRecord getEvent(long id) throws IOException;
 
     /**
      * Closes the repository, freeing any resources
@@ -216,15 +95,5 @@ public interface ProvenanceEventRepository {
      */
     void close() throws IOException;
 
-    /**
-     * @return a list of all fields that can be searched via the
-     * {@link #submitQuery(nifi.provenance.search.Query)} method
-     */
-    List<SearchableField> getSearchableFields();
 
-    /**
-     * @return a list of all FlowFile attributes that can be searched via the
-     * {@link #submitQuery(nifi.provenance.search.Query)} method
-     */
-    List<SearchableField> getSearchableAttributes();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java
deleted file mode 100644
index e754ff7..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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
-     */
-    public List<LineageNode> getNodes();
-
-    /**
-     * @return all links for the graph
-     */
-    public 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-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.java
deleted file mode 100644
index 8a11523..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageSubmission.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java
deleted file mode 100644
index ff5fee7..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/Lineage.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.java
deleted file mode 100644
index be74324..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageComputationType.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java
deleted file mode 100644
index dba56f3..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageEdge.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java
deleted file mode 100644
index 5d0956e..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNode.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.java
deleted file mode 100644
index 67e0d61..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/LineageNodeType.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java b/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java
deleted file mode 100644
index f490496..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/lineage/ProvenanceEventLineageNode.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/search/Query.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/search/Query.java b/nifi-api/src/main/java/org/apache/nifi/provenance/search/Query.java
deleted file mode 100644
index 3519c14..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/search/Query.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java b/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
deleted file mode 100644
index 0079433..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.java b/nifi-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.java
deleted file mode 100644
index 45efe4b..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/search/QuerySubmission.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java b/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java
deleted file mode 100644
index 573dbd7..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchTerm.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.java b/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.java
deleted file mode 100644
index aad4d7b..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchTerms.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java b/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java
deleted file mode 100644
index 85c6154..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchableField.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java b/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java
deleted file mode 100644
index 51ece58..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/provenance/search/SearchableFieldType.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java b/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
deleted file mode 100644
index c3a34b2..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/remote/RemoteDestination.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java b/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
index 695ed7f..be0e3f4 100644
--- a/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
+++ b/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinRepository.java
@@ -25,8 +25,8 @@ import java.util.List;
  */
 public interface BulletinRepository {
 
-    public static final int MAX_BULLETINS_PER_COMPONENT = 5;
-    public static final int MAX_BULLETINS_FOR_CONTROLLER = 10;
+    int MAX_BULLETINS_PER_COMPONENT = 5;
+    int MAX_BULLETINS_FOR_CONTROLLER = 10;
 
     /**
      * Adds a Bulletin to the repository.

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java b/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java
deleted file mode 100644
index ee5f417..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/web/ComponentDescriptor.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/web/ComponentDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/web/ComponentDescriptor.java
deleted file mode 100644
index 9412ee1..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/web/ComponentDescriptor.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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;
-
-import java.util.Map;
-
-public class ComponentDescriptor {
-
-    private final String name;
-    private final String displayName;
-    private final String description;
-    private final String defaultValue;
-    private final Map<String,String> allowableValues;
-
-    private ComponentDescriptor(Builder builder){
-        this.name = builder.name;
-        this.displayName = builder.displayName;
-        this.description = builder.description;
-        this.defaultValue = builder.defaultValue;
-        this.allowableValues = builder.allowableValues;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public String getDisplayName() {
-        return displayName;
-    }
-
-    public String getDescription() {
-        return description;
-    }
-
-    public String getDefaultValue() {
-        return defaultValue;
-    }
-
-    public Map<String,String> getAllowableValues() {
-        return allowableValues;
-    }
-
-    public static final class Builder{
-        private String name;
-        private String displayName;
-        private String description;
-        private String defaultValue;
-        private Map<String,String> allowableValues;
-
-        public Builder name(String name){
-            this.name = name;
-            return this;
-        }
-
-        public Builder displayName(String displayName){
-            this.displayName = displayName;
-            return this;
-        }
-
-        public  Builder description(String description){
-            this.description = description;
-            return this;
-        }
-
-        public Builder defaultValue(String defaultValue){
-            this.defaultValue = defaultValue;
-            return this;
-        }
-
-        public Builder allowableValues(Map<String,String> allowableValues){
-            this.allowableValues = allowableValues;
-            return this;
-        }
-
-        public ComponentDescriptor build(){
-            return new ComponentDescriptor(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java b/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java
deleted file mode 100644
index 9d163e4..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * 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;
-
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * Details about a given component. Contains configuration and current
- * validation errors.
- */
-public class ComponentDetails {
-
-    private final String id;
-    private final String name;
-    private final String type;
-    private final String state;
-    private final String annotationData;
-    private final Map<String, String> properties;
-    private final Map<String, ComponentDescriptor> descriptors;
-
-    private final Collection<String> validationErrors;
-
-    private ComponentDetails(final Builder builder) {
-        this.id = builder.id;
-        this.name = builder.name;
-        this.type = builder.type;
-        this.state = builder.state;
-        this.annotationData = builder.annotationData;
-        this.properties = builder.properties;
-        this.descriptors = builder.descriptors;
-        this.validationErrors = builder.validationErrors;
-
-    }
-
-    /**
-     * @return component id
-     */
-    public String getId() {
-        return id;
-    }
-
-    /**
-     * @return component name
-     */
-    public String getName() {
-        return name;
-    }
-
-    /**
-     * @return component type
-     */
-    public String getType() {
-        return type;
-    }
-
-    /**
-     * @return component state
-     */
-    public String getState() {
-        return state;
-    }
-
-    /**
-     * @return component's annotation data
-     */
-    public String getAnnotationData() {
-        return annotationData;
-    }
-
-    /**
-     * @return Mapping of component properties
-     */
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-
-    /**
-     * @return Mapping of component descriptors
-     */
-    public Map<String,ComponentDescriptor> getDescriptors(){
-        return descriptors;
-    }
-
-    /**
-     * @return Current validation errors for the component
-     */
-    public Collection<String> getValidationErrors() {
-        return validationErrors;
-    }
-
-    public static final class Builder {
-
-        private String id;
-        private String name;
-        private String type;
-        private String state;
-        private String annotationData;
-        private Map<String, String> properties;
-        private Map<String,ComponentDescriptor> descriptors;
-
-        private Collection<String> validationErrors;
-
-        public Builder id(final String id) {
-            this.id = id;
-            return this;
-        }
-
-        public Builder name(final String name) {
-            this.name = name;
-            return this;
-        }
-
-        public Builder type(final String type) {
-            this.type = type;
-            return this;
-        }
-
-        public Builder state(final String state) {
-            this.state = state;
-            return this;
-        }
-
-        public Builder annotationData(final String annotationData) {
-            this.annotationData = annotationData;
-            return this;
-        }
-
-        public Builder properties(final Map<String, String> properties) {
-            this.properties = properties;
-            return this;
-        }
-
-        public Builder validateErrors(final Collection<String> validationErrors) {
-            this.validationErrors = validationErrors;
-            return this;
-        }
-
-        public Builder descriptors(final Map<String,ComponentDescriptor> descriptors){
-            this.descriptors = descriptors;
-            return this;
-        }
-
-        public ComponentDetails build() {
-            return new ComponentDetails(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java b/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java
deleted file mode 100644
index 96f2abf..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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 action that represents the configuration of a component.
- */
-public class ConfigurationAction {
-
-    private final String id;
-    private final String name;
-    private final String type;
-    private final String field;
-    private final String previousValue;
-    private final String value;
-
-    private ConfigurationAction(final Builder builder) {
-        this.id = builder.id;
-        this.name = builder.name;
-        this.type = builder.type;
-        this.field = builder.field;
-        this.previousValue = builder.previousValue;
-        this.value = builder.value;
-    }
-
-    /**
-     * @return id of the component being modified
-     */
-    public String getId() {
-        return id;
-    }
-
-    /**
-     * @return name of the component being modified
-     */
-    public String getName() {
-        return name;
-    }
-
-    /**
-     * @return type of the component being modified
-     */
-    public String getType() {
-        return type;
-    }
-
-    /**
-     * @return the name of the field, property, etc that has been modified
-     */
-    public String getField() {
-        return field;
-    }
-
-    /**
-     * @return the previous value
-     */
-    public String getPreviousValue() {
-        return previousValue;
-    }
-
-    /**
-     * @return the new value
-     */
-    public String getValue() {
-        return value;
-    }
-
-    public static class Builder {
-
-        private String id;
-        private String name;
-        private String type;
-        private String field;
-        private String previousValue;
-        private String value;
-
-        public Builder id(final String id) {
-            this.id = id;
-            return this;
-        }
-
-        public Builder name(final String name) {
-            this.name = name;
-            return this;
-        }
-
-        public Builder type(final String type) {
-            this.type = type;
-            return this;
-        }
-
-        public Builder field(final String field) {
-            this.field = field;
-            return this;
-        }
-
-        public Builder previousValue(final String previousValue) {
-            this.previousValue = previousValue;
-            return this;
-        }
-
-        public Builder value(final String value) {
-            this.value = value;
-            return this;
-        }
-
-        public ConfigurationAction build() {
-            return new ConfigurationAction(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/web/InvalidRevisionException.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/web/InvalidRevisionException.java b/nifi-api/src/main/java/org/apache/nifi/web/InvalidRevisionException.java
deleted file mode 100644
index 8e04f69..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/web/InvalidRevisionException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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;
-
-/**
- * Exception indicating that the client has included an old revision in their
- * request.
- */
-@SuppressWarnings("serial")
-public class InvalidRevisionException extends RuntimeException {
-
-    public InvalidRevisionException(String message) {
-        super(message);
-    }
-
-    public InvalidRevisionException(String message, Throwable cause) {
-        super(message, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java b/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java
deleted file mode 100644
index 5082af2..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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;
-
-import org.apache.nifi.controller.ControllerService;
-
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * NiFi web context providing limited access to dataflow configuration for
- * component custom UIs.
- */
-public interface NiFiWebConfigurationContext {
-
-    /**
-     * @param serviceIdentifier of the controller service
-     * @param componentId the id of the component that is referencing the controller service
-     * @return the ControllerService for the specified identifier. If a
-     * corresponding service cannot be found, null is returned. If this NiFi is
-     * clustered, the only services available will be those those availability
-     * is NCM only
-     */
-    ControllerService getControllerService(String serviceIdentifier, String componentId);
-
-    /**
-     * Provides a mechanism for custom UIs to save actions to appear in NiFi
-     * configuration history. Note all fields within each Action must be
-     * populated. Null values will result in a failure to insert the audit
-     * record. Since the saving to these actions is separate from the actual
-     * configuration change, a failure to insert here will just generate a
-     * warning log message. The recording of these actions typically happens
-     * after a configuration change is applied. Since those changes have already
-     * been applied to the flow, we cannot revert them because of a failure to
-     * insert an audit record.
-     *
-     * @param requestContext context of the request
-     * @param actions to save
-     * @throws IllegalArgumentException When the requestContext isn't fully
-     * populated or isn't appropriate for the given request
-     */
-    void saveActions(NiFiWebRequestContext requestContext, Collection<ConfigurationAction> actions);
-
-    /**
-     * @return the current user identity. The value may be a DN, an email, a username, or any string that identities the user. Returns null if no user is found
-     */
-    String getCurrentUserIdentity();
-
-    /**
-     * Sets the annotation data for the underlying component.
-     *
-     * @param configurationContext config context
-     * @param annotationData the data
-     * @param properties component properties
-     * @return the configuration for the underlying component
-     * @throws ResourceNotFoundException if the underlying component does not
-     * exit
-     * @throws InvalidRevisionException if a revision other than the current
-     * revision is given
-     * @throws ClusterRequestException if the annotation data was unable to be
-     * set for the underlying component. This exception will only be thrown when
-     * operating in a cluster.
-     * @throws IllegalArgumentException When the requestContext isn't fully
-     * populated or isn't appropriate for the given request
-     */
-    ComponentDetails updateComponent(NiFiWebConfigurationRequestContext configurationContext, String annotationData, Map<String, String> properties)
-            throws ResourceNotFoundException, InvalidRevisionException, ClusterRequestException;
-
-
-    /**
-     * Gets the details for the underlying component (including configuration,
-     * validation errors, and annotation data).
-     *
-     * @param requestContext context of request
-     * @return the configuration for the underlying component
-     * @throws ResourceNotFoundException if the underlying component does not
-     * exit
-     * @throws ClusterRequestException if the underlying component was unable to
-     * be retrieved from the cluster. This exception will only be thrown when
-     * operating in a cluster.
-     * @throws IllegalArgumentException When the requestContext isn't fully
-     * populated or isn't appropriate for the given request
-     */
-    ComponentDetails getComponentDetails(NiFiWebRequestContext requestContext) throws ResourceNotFoundException, ClusterRequestException;
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d1129706/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java b/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java
deleted file mode 100644
index c75d9dc..0000000
--- a/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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;
-
-/**
- * Contextual details required to make a configuration request from a UI
- * extension.
- */
-public interface NiFiWebConfigurationRequestContext extends NiFiWebRequestContext {
-
-    /**
-     * The revision to include in the request.
-     *
-     * @return the revision
-     */
-    Revision getRevision();
-
-}