http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b512ff12/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
index c5516aa..3fcdd77 100644
--- 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
+++ 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
@@ -20,6 +20,7 @@ import java.io.Closeable;
 
 import org.apache.nifi.provenance.ProvenanceEventRepository;
 import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
 import org.apache.nifi.provenance.search.Query;
 import org.apache.nifi.provenance.search.QuerySubmission;
 
@@ -34,6 +35,19 @@ public interface QueryManager extends Closeable {
     QuerySubmission submitQuery(Query query);
     
     /**
+     * Registers the given query submission with this manager so that it can 
be retrieved later.
+     * @param submission
+     */
+    void registerSubmission(ProvenanceQuerySubmission submission);
+    
+    /**
+     * Retrieves the ProvenanceQuerySubmission with the given identifier, if 
it exists, else returns <code>null</code>.
+     * @param id
+     * @return
+     */
+    ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(String id);
+    
+    /**
      * Returns the QueryResult associated with the given identifier, if the
      * query has finished processing. If the query has not yet finished 
running,
      * returns <code>null</code>.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b512ff12/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
index 0753e9e..aebeade 100644
--- 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
+++ 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -52,6 +53,8 @@ import 
org.apache.nifi.provenance.journaling.toc.StandardTocReader;
 import org.apache.nifi.provenance.journaling.toc.TocReader;
 import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
 import org.apache.nifi.provenance.lineage.LineageComputationType;
+import org.apache.nifi.provenance.query.ProvenanceQueryResult;
+import org.apache.nifi.provenance.query.ProvenanceQuerySubmission;
 import org.apache.nifi.provenance.search.Query;
 import org.apache.nifi.provenance.search.QuerySubmission;
 import org.slf4j.Logger;
@@ -66,6 +69,7 @@ public class StandardQueryManager implements QueryManager {
     private final JournalingRepositoryConfig config;
     private final ConcurrentMap<String, AsyncQuerySubmission> 
querySubmissionMap = new ConcurrentHashMap<>();
     private final ConcurrentMap<String, AsyncLineageSubmission> 
lineageSubmissionMap = new ConcurrentHashMap<>();
+    private final ConcurrentMap<String, ProvenanceQuerySubmission> 
provenanceQuerySubmissionMap = new ConcurrentHashMap<>();
     
     public StandardQueryManager(final IndexManager indexManager, final 
ExecutorService executor, final JournalingRepositoryConfig config, final int 
maxConcurrentQueries) {
         this.config = config;
@@ -150,8 +154,8 @@ public class StandardQueryManager implements QueryManager {
         final AsyncQuerySubmission submission = new 
AsyncQuerySubmission(query, indexManager.getNumberOfIndices()) {
             @Override
             public void cancel() {
-                super.cancel();
                 querySubmissionMap.remove(query.getIdentifier());
+                super.cancel();
             }
         };
         
@@ -403,6 +407,39 @@ public class StandardQueryManager implements QueryManager {
     
     
     @Override
+    public void registerSubmission(final ProvenanceQuerySubmission submission) 
{
+        final ProvenanceQuerySubmission cancelable = new 
ProvenanceQuerySubmission() {
+            @Override
+            public String getQuery() { return submission.getQuery(); }
+
+            @Override
+            public ProvenanceQueryResult getResult() { return 
submission.getResult(); }
+
+            @Override
+            public Date getSubmissionTime() { return 
submission.getSubmissionTime(); }
+
+            @Override
+            public String getQueryIdentifier() { return 
submission.getQueryIdentifier(); }
+
+            @Override
+            public void cancel() {
+                
provenanceQuerySubmissionMap.remove(submission.getQueryIdentifier());
+                submission.cancel();
+            }
+
+            @Override
+            public boolean isCanceled() { return submission.isCanceled(); }
+        };
+        
+        
provenanceQuerySubmissionMap.putIfAbsent(submission.getQueryIdentifier(), 
cancelable);
+    }
+    
+    @Override
+    public ProvenanceQuerySubmission retrieveProvenanceQuerySubmission(final 
String id) {
+        return provenanceQuerySubmissionMap.get(id);
+    }
+    
+    @Override
     public void close() throws IOException {
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b512ff12/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
index 30e100a..e51f5b7 100644
--- 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
+++ 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
@@ -17,19 +17,25 @@
 package org.apache.nifi.provenance.journaling;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.provenance.SearchableFields;
 import org.apache.nifi.provenance.StandardProvenanceEventRecord;
@@ -40,12 +46,14 @@ import 
org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
 import org.apache.nifi.provenance.lineage.LineageNode;
 import org.apache.nifi.provenance.lineage.LineageNodeType;
 import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
+import org.apache.nifi.provenance.query.ProvenanceResultSet;
 import org.apache.nifi.provenance.search.Query;
 import org.apache.nifi.provenance.search.QueryResult;
 import org.apache.nifi.provenance.search.QuerySubmission;
 import org.apache.nifi.provenance.search.SearchTerms;
 import org.apache.nifi.provenance.search.SearchableField;
 import org.apache.nifi.util.file.FileUtils;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -286,7 +294,7 @@ public class TestJournalingProvenanceRepository {
     
     
     
-    @Test(timeout=10000000)
+    @Test(timeout=10000)
     public void testSearchByUUID() throws IOException, InterruptedException {
         final JournalingRepositoryConfig config = new 
JournalingRepositoryConfig();
         final Map<String, File> containers = new HashMap<>();
@@ -327,6 +335,303 @@ public class TestJournalingProvenanceRepository {
             final StoredProvenanceEvent event = matches.get(0);
             assertEquals(5, event.getEventId());
             assertEquals("00000000-0000-0000-0000-000000000005", 
event.getFlowFileUuid());
+            System.out.println("Query time was " + result.getQueryTime() + " 
millis");
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
+    @Test()
+    public void testSearchByUUIDTextQuery() throws IOException, 
InterruptedException {
+        final JournalingRepositoryConfig config = new 
JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + 
UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(1);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new 
JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            
+            final Map<String, String> attributes = new HashMap<>();
+            
+            final long start = System.nanoTime();
+            for (int i=0; i < 10; i++) {
+                attributes.put("i", String.valueOf(i));
+                repo.registerEvent(TestUtil.generateEvent(i, attributes));
+            }
+            final long registerFinish = System.nanoTime();
+            
+            // Ensure that we get the Events, even though we are querying for 
Event.Time because selectMatchingEvents
+            // only takes into account the WHERE clause
+            final Iterator<? extends StoredProvenanceEvent> itr = 
repo.selectMatchingEvents("SELECT Event.Time WHERE Event.uuid = 
'00000000-0000-0000-0000-000000000005'", new AtomicLong(0L));
+            assertTrue(itr.hasNext());
+            final StoredProvenanceEvent event = itr.next();
+            assertNotNull(event);
+            assertFalse(itr.hasNext());
+            
+            final long searchFinish = System.nanoTime();
+            assertEquals(5, event.getEventId());
+            assertEquals("00000000-0000-0000-0000-000000000005", 
event.getFlowFileUuid());
+            
+            System.out.println("Register records: " + 
TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+            System.out.println("Query records: " + 
TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+        } catch (final Exception e) {
+            e.printStackTrace();
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
+    
+    @Test()
+    public void testSearchFieldTextQuery() throws IOException, 
InterruptedException {
+        final JournalingRepositoryConfig config = new 
JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + 
UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + 
UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(3);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new 
JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            
+            final Map<String, String> attributes = new HashMap<>();
+            
+            final long start = System.nanoTime();
+            for (int i=0; i < 10; i++) {
+                attributes.put("i", String.valueOf(i));
+                repo.registerEvent(TestUtil.generateEvent(i, attributes));
+            }
+            final long registerFinish = System.nanoTime();
+            
+            final ProvenanceResultSet rs = repo.query("SELECT Event.uuid WHERE 
Event.uuid = '00000000-0000-0000-0000-000000000005'");
+            assertTrue(rs.hasNext());
+            final List<?> cols = rs.next();
+            assertEquals(1, cols.size());
+            assertEquals(String.class, rs.getReturnType().get(0));
+            final Object firstCol = cols.get(0);
+            assertEquals("00000000-0000-0000-0000-000000000005", firstCol);
+            assertFalse(rs.hasNext());
+            
+            final long searchFinish = System.nanoTime();
+            System.out.println("Register records: " + 
TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+            System.out.println("Query records: " + 
TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
+    @Test()
+    public void testSearchEventTextQuery() throws IOException, 
InterruptedException {
+        final JournalingRepositoryConfig config = new 
JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + 
UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + 
UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(3);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new 
JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            
+            final Map<String, String> attributes = new HashMap<>();
+            
+            final long start = System.nanoTime();
+            for (int i=0; i < 10; i++) {
+                attributes.put("i", String.valueOf(i));
+                repo.registerEvent(TestUtil.generateEvent(i, attributes));
+            }
+            final long registerFinish = System.nanoTime();
+            
+            final ProvenanceResultSet rs = repo.query("SELECT Event WHERE 
Event.uuid = '00000000-0000-0000-0000-000000000005'");
+            assertTrue(rs.hasNext());
+            final List<?> cols = rs.next();
+            assertEquals(1, cols.size());
+            
assertTrue(ProvenanceEventRecord.class.isAssignableFrom(rs.getReturnType().get(0)));
+            final ProvenanceEventRecord firstCol = (ProvenanceEventRecord) 
cols.get(0);
+            assertEquals("00000000-0000-0000-0000-000000000005", 
firstCol.getFlowFileUuid());
+            assertEquals("5", firstCol.getAttribute("i"));
+            assertEquals("5", firstCol.getAttributes().get("i"));
+            assertEquals("5", firstCol.getUpdatedAttributes().get("i"));
+            assertFalse(rs.hasNext());
+            
+            final long searchFinish = System.nanoTime();
+            System.out.println("Register records: " + 
TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+            System.out.println("Query records: " + 
TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
+    
+    @Test()
+    public void testSearchEventTextQueryAgainstMany() throws IOException, 
InterruptedException {
+        final JournalingRepositoryConfig config = new 
JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + 
UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + 
UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(3);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new 
JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            
+            final Map<String, String> attributes = new HashMap<>();
+            
+            final long start = System.nanoTime();
+            final List<ProvenanceEventRecord> events = new ArrayList<>(1000);
+            for (int i=0; i < 100000; i++) {
+                attributes.put("i", String.valueOf(i));
+                final ProvenanceEventRecord event = TestUtil.generateEvent(i, 
attributes);
+                events.add(event);
+                if ( events.size() % 1000 == 0 ) {
+                    repo.registerEvents(events);
+                    events.clear();
+                }
+            }
+            final long registerFinish = System.nanoTime();
+            
+            final ProvenanceResultSet rs = repo.query("SELECT Event WHERE 
Event.uuid = '00000000-0000-0000-0000-000000000005'");
+            assertTrue(rs.hasNext());
+            final List<?> cols = rs.next();
+            assertEquals(1, cols.size());
+            
assertTrue(ProvenanceEventRecord.class.isAssignableFrom(rs.getReturnType().get(0)));
+            final ProvenanceEventRecord firstCol = (ProvenanceEventRecord) 
cols.get(0);
+            assertEquals("00000000-0000-0000-0000-000000000005", 
firstCol.getFlowFileUuid());
+            assertFalse(rs.hasNext());
+            
+            final long searchFinish = System.nanoTime();
+            System.out.println("Register records: " + 
TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+            System.out.println("Query records: " + 
TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
+    
+    @Test()
+    public void testSearchManyEventsTextQueryAgainstMany() throws IOException, 
InterruptedException {
+        final JournalingRepositoryConfig config = new 
JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + 
UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + 
UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(3);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new 
JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            
+            final Map<String, String> attributes = new HashMap<>();
+            
+            final long start = System.nanoTime();
+            final List<ProvenanceEventRecord> events = new ArrayList<>(1000);
+            for (int i=0; i < 100000; i++) {
+                attributes.put("i", String.valueOf(i));
+                final ProvenanceEventRecord event = TestUtil.generateEvent(i, 
attributes);
+                events.add(event);
+                if ( events.size() % 1000 == 0 ) {
+                    repo.registerEvents(events);
+                    events.clear();
+                }
+            }
+            final long registerFinish = System.nanoTime();
+            
+            final ProvenanceResultSet rs = repo.query("SELECT Event");
+            for (int i=0; i < 100000; i++) {
+                assertTrue(rs.hasNext());
+                final List<?> cols = rs.next();
+                assertEquals(1, cols.size());
+            }
+            assertFalse(rs.hasNext());
+            
+            final long searchFinish = System.nanoTime();
+            System.out.println("Register records: " + 
TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+            System.out.println("Query records: " + 
TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+        } catch (final Exception e) {
+            e.printStackTrace();
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
+    
+    @Test()
+    public void testAggregateQueryAgainstMany() throws IOException, 
InterruptedException {
+        final JournalingRepositoryConfig config = new 
JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + 
UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + 
UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(3);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new 
JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            
+            final Map<String, String> attributes = new HashMap<>();
+            
+            final long start = System.nanoTime();
+            final List<ProvenanceEventRecord> events = new ArrayList<>(1000);
+            for (int i=0; i < 100000; i++) {
+                attributes.put("i", String.valueOf(i));
+                final ProvenanceEventRecord event = TestUtil.generateEvent(i, 
attributes);
+                events.add(event);
+                if ( events.size() % 1000 == 0 ) {
+                    repo.registerEvents(events);
+                    events.clear();
+                }
+            }
+            final long registerFinish = System.nanoTime();
+            
+            final ProvenanceResultSet rs = repo.query("SELECT SUM(Event.Size), 
AVG(Event.Size)");
+            assertTrue(rs.hasNext());
+            assertEquals(Long.class, rs.getReturnType().get(0));
+            assertEquals(Double.class, rs.getReturnType().get(1));
+            final List<?> cols = rs.next();
+            assertEquals(100000L, cols.get(0));
+            assertEquals(1D, cols.get(1));
+            assertFalse(rs.hasNext());
+            
+            final long searchFinish = System.nanoTime();
+            System.out.println("Register records: " + 
TimeUnit.NANOSECONDS.toMillis(registerFinish - start) + " millis");
+            System.out.println("Query records: " + 
TimeUnit.NANOSECONDS.toMillis(searchFinish - registerFinish) + " millis");
+        } catch (final Exception e) {
+            e.printStackTrace();
+            Assert.fail(e.toString());
         } finally {
             for ( final File file : containers.values() ) {
                 FileUtils.deleteFile(file, true);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b512ff12/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
index 6d05f7a..88201ae 100644
--- 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
+++ 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
@@ -40,7 +40,7 @@ public class TestUtil {
             .setEventTime(System.currentTimeMillis())
             .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
             .setLineageStartDate(System.currentTimeMillis() - 2000L)
-            .setCurrentContentClaim(null, null, null, null, 0L)
+            .setCurrentContentClaim(null, null, null, null, 1L)
             .setAttributes(null, attributes == null ? Collections.<String, 
String>emptyMap() : attributes)
             .build();
         

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b512ff12/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
index e611aaa..874727d 100644
--- 
a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
+++ 
b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
@@ -52,7 +52,7 @@ public class TestEventIndexWriter {
         
         final File indexDir = new File("target/" + 
UUID.randomUUID().toString());
         
-        try (final LuceneIndexWriter indexWriter = new 
LuceneIndexWriter(indexDir, config)) {
+        try (final LuceneIndexWriter indexWriter = new LuceneIndexWriter(null, 
indexDir, config)) {
             final ProvenanceEventRecord event = TestUtil.generateEvent(23L);
             final JournaledStorageLocation location = new 
JournaledStorageLocation("container", "section", 1L, 2, 23L);
             final JournaledProvenanceEvent storedEvent = new 
JournaledProvenanceEvent(event, location);

Reply via email to