You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2015/02/27 21:47:28 UTC

[01/12] incubator-nifi git commit: implemented ability to persist and recover records

Repository: incubator-nifi
Updated Branches:
  refs/heads/journaling-prov-repo [created] f23f36d73


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
new file mode 100644
index 0000000..eca664e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.toc;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+
+public class TocJournalReader implements Closeable {
+
+    private final TocReader tocReader;
+    private final JournalReader reader;
+    
+    private final String containerName;
+    private final String sectionName;
+    private final String journalId;
+    
+    private int blockIndex;
+    private long nextBlockOffset;
+    
+    
+    public TocJournalReader(final String containerName, final String sectionName, final String journalId, final File journalFile) throws IOException {
+        this.containerName = containerName;
+        this.sectionName = sectionName;
+        this.journalId = journalId;
+        
+        final File tocFile = new File(journalFile.getParentFile(), journalFile.getName() + ".toc");
+        tocReader = new StandardTocReader(tocFile);
+        reader = new StandardJournalReader(journalFile);
+        
+        blockIndex = 0;
+        nextBlockOffset = tocReader.getBlockOffset(1);
+    }
+    
+    @Override
+    public void close() throws IOException {
+        IOException suppressed = null;
+        try {
+            tocReader.close();
+        } catch (final IOException ioe) {
+            suppressed = ioe;
+        }
+        
+        try {
+            reader.close();
+        } catch (final IOException ioe) {
+            if ( suppressed != null ) {
+                ioe.addSuppressed(suppressed);
+            }
+            throw ioe;
+        }
+        
+        if ( suppressed != null ) {
+            throw suppressed;
+        }
+    }
+    
+    public JournaledProvenanceEvent nextJournaledEvent() throws IOException {
+        ProvenanceEventRecord event = reader.nextEvent();
+        if ( event == null ) {
+            return null;
+        }
+        
+        final JournaledStorageLocation location = new JournaledStorageLocation(containerName, sectionName, 
+                journalId, blockIndex, event.getEventId());
+        
+        // Check if we've gone beyond the offset of the next block. If so, write
+        // out a new block in the TOC.
+        final long newPosition = reader.getPosition();
+        if ( newPosition > nextBlockOffset && nextBlockOffset > 0 ) {
+            blockIndex++;
+            nextBlockOffset = tocReader.getBlockOffset(blockIndex + 1);
+        }
+        
+        return new JournaledProvenanceEvent(event, location);
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
new file mode 100644
index 0000000..9f6a264
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.toc;
+
+import java.io.Closeable;
+
+/**
+ * <p>
+ * Reads a Table of Contents (.toc file) for a corresponding Journal File. We use a Table of Contents
+ * to map a Block Index to an offset into the Journal file where that Block begins. We do this so that
+ * we can then persist a Block Index for an event and then compress the Journal later. This way, we can
+ * get good compression by compressing a large batch of events at once, and this way we can also look up
+ * an event in a Journal that has not been compressed by looking in the Table of Contents or lookup the
+ * event in a Journal post-compression by simply rewriting the TOC while we compress the data.
+ * </p>
+ */
+public interface TocReader extends Closeable {
+
+    /**
+     * Indicates whether or not the corresponding Journal file is compressed
+     * @return
+     */
+    boolean isCompressed();
+
+    /**
+     * Returns the byte offset into the Journal File for the Block with the given index.
+     * @param blockIndex
+     * @return
+     */
+    long getBlockOffset(int blockIndex);
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocWriter.java
new file mode 100644
index 0000000..b44b55b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocWriter.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.toc;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Writes a .toc file
+ */
+public interface TocWriter extends Closeable {
+
+    /**
+     * Adds the given block offset as the next Block Offset in the Table of Contents
+     * @param offset
+     * @throws IOException
+     */
+    void addBlockOffset(long offset) throws IOException;
+    
+    /**
+     * Returns the index of the current Block
+     * @return
+     */
+    int getCurrentBlockIndex();
+    
+    /**
+     * Returns the file that is currently being written to
+     * @return
+     */
+    File getFile();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/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
new file mode 100644
index 0000000..45b7338
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestUtil.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling;
+
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+
+public class TestUtil {
+    public static ProvenanceEventRecord generateEvent(final long id) {
+        // Create prov event to add to the stream
+        final ProvenanceEventRecord event = new StandardProvenanceEventRecord.Builder()
+            .setEventType(ProvenanceEventType.CREATE)
+            .setFlowFileUUID("00000000-0000-0000-0000-" + pad(String.valueOf(id), 12, '0'))
+            .setComponentType("Unit Test")
+            .setComponentId(UUID.randomUUID().toString())
+            .setEventTime(System.currentTimeMillis())
+            .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
+            .setLineageStartDate(System.currentTimeMillis() - 2000L)
+            .setCurrentContentClaim(null, null, null, null, 0L)
+            .build();
+        
+        return event;
+    }
+    
+    public static String pad(final String value, final int charCount, final char padding) {
+        if ( value.length() >= charCount ) {
+            return value;
+        }
+        
+        final StringBuilder sb = new StringBuilder();
+        for (int i=value.length(); i < charCount; i++) {
+            sb.append(padding);
+        }
+        sb.append(value);
+        
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/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
new file mode 100644
index 0000000..dfaeb1a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/index/TestEventIndexWriter.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.TestUtil;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.SearchTerms;
+import org.apache.nifi.provenance.search.SearchableField;
+import org.junit.Test;
+
+public class TestEventIndexWriter {
+
+    @Test
+    public void testIndexAndFetch() throws IOException {
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        config.setSearchableAttributes(Arrays.asList(new SearchableField[] {
+                SearchableFields.newSearchableAttribute("test.1")
+        }));
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        final File indexDir = new File("target/" + UUID.randomUUID().toString());
+        
+        final File journalFile = new File("target/" + UUID.randomUUID().toString());
+        try (final LuceneIndexWriter indexWriter = new LuceneIndexWriter(indexDir, config)) {
+            
+            final ProvenanceEventRecord event = TestUtil.generateEvent(23L);
+            final JournaledStorageLocation location = new JournaledStorageLocation("container", "section", "journalId", 2, 23L);
+            final JournaledProvenanceEvent storedEvent = new JournaledProvenanceEvent(event, location);
+            indexWriter.index(Collections.singleton(storedEvent));
+            
+            final Query query = new Query("123");
+            query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.FlowFileUUID, "00000000-0000-0000-0000-000000000023"));
+            
+            try (final EventIndexSearcher searcher = indexWriter.newIndexSearcher()) {
+                final SearchResult searchResult = searcher.search(query);
+                final List<JournaledStorageLocation> locations = searchResult.getLocations();
+                assertNotNull(locations);
+                assertEquals(1, locations.size());
+                
+                final JournaledStorageLocation found = locations.get(0);
+                assertNotNull(found);
+                assertEquals("container", found.getContainerName());
+                assertEquals("section", found.getSectionName());
+                assertEquals("journalId", found.getJournalId());
+                assertEquals(2, found.getBlockIndex());
+                assertEquals(23L, found.getEventId());
+            }
+        } finally {
+            journalFile.delete();
+        }
+        
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestJournalReadWrite.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestJournalReadWrite.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestJournalReadWrite.java
new file mode 100644
index 0000000..f2266e2
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestJournalReadWrite.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.journals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.TestUtil;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestJournalReadWrite {
+
+    @Test
+    public void testReadWrite100Blocks() throws IOException {
+        testReadWrite100Blocks(true);
+        testReadWrite100Blocks(false);
+    }
+    
+    private void testReadWrite100Blocks(final boolean compressed) throws IOException {
+        final long journalId = 1L;
+        final File journalFile = new File("target/1.journal");
+        final StandardEventSerializer serializer = new StandardEventSerializer();
+        
+        try {
+            try (final StandardJournalWriter writer = new StandardJournalWriter(journalId, journalFile, compressed, serializer)) {
+                for (int block=0; block < 100; block++) {
+                    writer.beginNewBlock();
+                    
+                    for (int i=0; i < 5; i++) {
+                        final ProvenanceEventRecord event = TestUtil.generateEvent(i);
+                        writer.write(Collections.singleton(event), i);
+                    }
+                    
+                    final List<ProvenanceEventRecord> events = new ArrayList<>();
+                    for (int i=0; i < 90; i++) {
+                        events.add(TestUtil.generateEvent(i + 5));
+                    }
+                    writer.write(events, 5);
+                    
+                    for (int i=0; i < 5; i++) {
+                        final ProvenanceEventRecord event = TestUtil.generateEvent(i);
+                        writer.write(Collections.singleton(event), 95 + i);
+                    }
+                    
+                    writer.finishBlock();
+                }
+            }
+            
+            try (final StandardJournalReader reader = new StandardJournalReader(journalFile)) {
+                for (int block=0; block < 100; block++) {
+                    for (int i=0; i < 100; i++) {
+                        final ProvenanceEventRecord record = reader.nextEvent();
+                        Assert.assertNotNull(record);
+                        Assert.assertEquals((long) i, record.getEventId());
+                    }
+                }
+            }
+        } finally {
+            journalFile.delete();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalReader.java
new file mode 100644
index 0000000..e1ecf7d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalReader.java
@@ -0,0 +1,516 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.journals;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.remote.io.CompressionOutputStream;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestStandardJournalReader {
+
+    private ByteArrayOutputStream baos;
+    private DataOutputStream dos;
+    
+    @Before
+    public void setup() throws IOException {
+        // Create a BAOS to write the record to.
+        baos = new ByteArrayOutputStream();
+        dos = new DataOutputStream(baos);
+        
+        // Write out header: codec name and serialization version
+        dos.writeUTF(StandardEventSerializer.CODEC_NAME);
+        dos.writeInt(0);
+    }
+    
+    
+    @Test
+    public void testReadFirstEventUncompressed() throws IOException {
+        dos.writeBoolean(false);
+        writeRecord(88L);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            final ProvenanceEventRecord restored = reader.nextEvent();
+            assertNotNull(restored);
+            assertEquals(88L, restored.getEventId());
+            assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+            assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    
+    @Test
+    public void testReadManyUncompressed() throws IOException {
+        dos.writeBoolean(false);
+        writeRecords(0, 1024, false);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            for (int i=0; i < 1024; i++) {
+                final ProvenanceEventRecord restored = reader.nextEvent();
+                assertNotNull(restored);
+                assertEquals((long) i, restored.getEventId());
+                assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+                assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+            }
+            
+            assertNull(reader.nextEvent());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    @Test
+    public void testReadFirstEventWithBlockOffsetUncompressed() throws IOException {
+        dos.writeBoolean(false);
+        writeRecords(0, 10, false);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, false);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+            assertNotNull(restored);
+            assertEquals(10L, restored.getEventId());
+            assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+            assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    @Test
+    public void testReadSubsequentEventWithBlockOffsetUncompressed() throws IOException {
+        dos.writeBoolean(false);
+        writeRecords(0, 10, false);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, false);
+
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+            assertNotNull(restored);
+            assertEquals(10L, restored.getEventId());
+            assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+            assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    @Test
+    public void testReadMultipleEventsWithBlockOffsetUncompressed() throws IOException {
+        dos.writeBoolean(false);
+        writeRecords(0, 10, false);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, false);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            for (int i=0; i < 2; i++) {
+                final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+                assertNotNull(event10);
+                assertEquals(10L, event10.getEventId());
+                assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+                assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+                
+                final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+                assertNotNull(event13);
+                assertEquals(13L, event13.getEventId());
+                assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+                assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+            }
+        } finally {
+            file.delete();
+        }
+    }
+    
+    
+    @Test
+    public void testReadFirstEventCompressed() throws IOException {
+        dos.writeBoolean(true);
+        writeRecords(88L, 1, true);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            final ProvenanceEventRecord restored = reader.nextEvent();
+            assertNotNull(restored);
+            assertEquals(88L, restored.getEventId());
+            assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+            assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    @Test
+    public void testReadManyCompressed() throws IOException {
+        dos.writeBoolean(true);
+        writeRecords(0, 1024, true);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            for (int i=0; i < 1024; i++) {
+                final ProvenanceEventRecord restored = reader.nextEvent();
+                assertNotNull(restored);
+                assertEquals((long) i, restored.getEventId());
+                assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+                assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+            }
+            
+            assertNull(reader.nextEvent());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    
+    @Test
+    public void testReadFirstEventWithBlockOffsetCompressed() throws IOException {
+        dos.writeBoolean(true);
+        writeRecords(0, 10, true);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, true);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+            assertNotNull(restored);
+            assertEquals(10L, restored.getEventId());
+            assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+            assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    @Test
+    public void testReadSubsequentEventWithBlockOffsetCompressed() throws IOException {
+        dos.writeBoolean(true);
+        writeRecords(0, 10, true);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, true);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            final ProvenanceEventRecord restored = reader.getEvent(secondBlockOffset, 10L);
+            assertNotNull(restored);
+            assertEquals(10L, restored.getEventId());
+            assertEquals(ProvenanceEventType.CREATE, restored.getEventType());
+            assertEquals("00000000-0000-0000-0000-000000000000", restored.getFlowFileUuid());
+        } finally {
+            file.delete();
+        }
+    }
+    
+    @Test
+    public void testReadMultipleEventsWithBlockOffsetCompressed() throws IOException {
+        dos.writeBoolean(true);
+        writeRecords(0, 10, true);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, true);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            for (int i=0; i < 2; i++) {
+                final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+                assertNotNull(event10);
+                assertEquals(10L, event10.getEventId());
+                assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+                assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+                
+                final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+                assertNotNull(event13);
+                assertEquals(13L, event13.getEventId());
+                assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+                assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+            }
+        } finally {
+            file.delete();
+        }
+    }
+    
+    
+    @Test
+    public void testReadEventWithBlockOffsetThenPreviousBlockOffsetUncompressed() throws IOException {
+        dos.writeBoolean(false);
+        final int firstBlockOffset = baos.size();
+        writeRecords(0, 10, false);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, false);
+
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            for (int j=0; j < 2; j++) {
+                for (int i=0; i < 2; i++) {
+                    final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+                    assertNotNull(event10);
+                    assertEquals(10L, event10.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+                    
+                    final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+                    assertNotNull(event13);
+                    assertEquals(13L, event13.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+                }
+                
+                for (int i=0; i < 2; i++) {
+                    final ProvenanceEventRecord event2 = reader.getEvent(firstBlockOffset, 2L);
+                    assertNotNull(event2);
+                    assertEquals(2L, event2.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event2.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event2.getFlowFileUuid());
+                    
+                    final ProvenanceEventRecord event6 = reader.getEvent(firstBlockOffset, 6L);
+                    assertNotNull(event6);
+                    assertEquals(6L, event6.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event6.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event6.getFlowFileUuid());
+                }
+            }
+        } finally {
+            file.delete();
+        }
+    }
+    
+    
+    @Test
+    public void testReadEventWithBlockOffsetThenPreviousBlockOffsetCompressed() throws IOException {
+        dos.writeBoolean(true);
+        final int firstBlockOffset = baos.size();
+        writeRecords(0, 10, true);
+        
+        final int secondBlockOffset = baos.size();
+        writeRecords(10, 10, true);
+        
+        // write data to a file so that we can read it with the journal reader
+        final File dir = new File("target/testData");
+        final File file = new File(dir, UUID.randomUUID().toString() + ".journal");
+        dir.mkdirs();
+        
+        try (final FileOutputStream fos = new FileOutputStream(file)) {
+            baos.writeTo(fos);
+        }
+        
+        // read the record and verify its contents
+        try (final StandardJournalReader reader = new StandardJournalReader(file)) {
+            for (int j=0; j < 2; j++) {
+                for (int i=0; i < 2; i++) {
+                    final ProvenanceEventRecord event10 = reader.getEvent(secondBlockOffset, 10L);
+                    assertNotNull(event10);
+                    assertEquals(10L, event10.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event10.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event10.getFlowFileUuid());
+                    
+                    final ProvenanceEventRecord event13 = reader.getEvent(secondBlockOffset, 13L);
+                    assertNotNull(event13);
+                    assertEquals(13L, event13.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event13.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event13.getFlowFileUuid());
+                }
+                
+                for (int i=0; i < 2; i++) {
+                    final ProvenanceEventRecord event2 = reader.getEvent(firstBlockOffset, 2L);
+                    assertNotNull(event2);
+                    assertEquals(2L, event2.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event2.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event2.getFlowFileUuid());
+                    
+                    final ProvenanceEventRecord event6 = reader.getEvent(firstBlockOffset, 6L);
+                    assertNotNull(event6);
+                    assertEquals(6L, event6.getEventId());
+                    assertEquals(ProvenanceEventType.CREATE, event6.getEventType());
+                    assertEquals("00000000-0000-0000-0000-000000000000", event6.getFlowFileUuid());
+                }
+            }
+        } finally {
+            file.delete();
+        }
+    }
+    
+
+    
+    
+    private void writeRecord(final long id) throws IOException {
+        writeRecord(id, dos);
+    }
+    
+    private void writeRecords(final long startId, final int numRecords, final boolean compressed) throws IOException {
+        if ( compressed ) {
+            final CompressionOutputStream compressedOut = new CompressionOutputStream(dos);
+            for (long id = startId; id < startId + numRecords; id++) {
+                writeRecord(id, new DataOutputStream(compressedOut));
+            }
+            compressedOut.close();
+        } else {
+            for (long id = startId; id < startId + numRecords; id++) {
+                writeRecord(id, dos);
+            }
+        }
+    }
+    
+    private void writeRecord(final long id, final DataOutputStream dos) throws IOException {
+        // Create prov event to add to the stream
+        final ProvenanceEventRecord event = new StandardProvenanceEventRecord.Builder()
+            .setEventType(ProvenanceEventType.CREATE)
+            .setFlowFileUUID("00000000-0000-0000-0000-000000000000")
+            .setComponentType("Unit Test")
+            .setComponentId(UUID.randomUUID().toString())
+            .setEventTime(System.currentTimeMillis())
+            .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
+            .setLineageStartDate(System.currentTimeMillis() - 2000L)
+            .setCurrentContentClaim(null, null, null, null, 0L)
+            .build();
+        
+        // Serialize the prov event
+        final ByteArrayOutputStream serializationStream = new ByteArrayOutputStream();
+        final StandardEventSerializer serializer = new StandardEventSerializer();
+        serializer.serialize(event, new DataOutputStream(serializationStream));
+        
+        // Write out to our stream the event length, followed by the id, and then the serialized event
+        final int recordLen = 8 + serializationStream.size();
+        
+        dos.writeInt(recordLen);
+        dos.writeLong(id);
+        serializationStream.writeTo(dos);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
new file mode 100644
index 0000000..d5eab8e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.journals;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Collections;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.journaling.TestUtil;
+import org.apache.nifi.provenance.journaling.io.StandardEventDeserializer;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.remote.io.CompressionInputStream;
+import org.junit.Test;
+
+public class TestStandardJournalWriter {
+
+    @Test
+    public void testOneBlockOneRecordWriteCompressed() throws IOException {
+        final File journalFile = new File("target/" + UUID.randomUUID().toString());
+        
+        final StandardEventSerializer serializer = new StandardEventSerializer();
+        try {
+            try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, true, serializer)) {
+                writer.beginNewBlock();
+                writer.write(Collections.singleton(TestUtil.generateEvent(1L)), 1L);
+                writer.finishBlock();
+            }
+        
+            final byte[] data = Files.readAllBytes(journalFile.toPath());
+            final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+            final DataInputStream dis = new DataInputStream(bais);
+
+            final String codecName = dis.readUTF();
+            assertEquals(StandardEventSerializer.CODEC_NAME, codecName);
+            
+            final int version = dis.readInt();
+            assertEquals(1, version);
+            
+            // compression flag
+            assertEquals(true, dis.readBoolean());
+            
+            // read block start
+            final CompressionInputStream decompressedIn = new CompressionInputStream(bais);
+            final StandardEventDeserializer deserializer = new StandardEventDeserializer();
+            
+            final DataInputStream decompressedDis = new DataInputStream(decompressedIn);
+            final int eventLength = decompressedDis.readInt();
+            assertEquals(131, eventLength);
+            final ProvenanceEventRecord event = deserializer.deserialize(decompressedDis, 0);
+            assertEquals(1, event.getEventId());
+            assertEquals(ProvenanceEventType.CREATE, event.getEventType());
+            
+            assertEquals(-1, decompressedIn.read());
+        } finally {
+            journalFile.delete();
+        }
+    }
+    
+    @Test
+    public void testManyBlocksOneRecordWriteCompressed() throws IOException {
+        final File journalFile = new File("target/" + UUID.randomUUID().toString());
+        
+        final StandardEventSerializer serializer = new StandardEventSerializer();
+        try {
+            try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, true, serializer)) {
+                for (int i=0; i < 1024; i++) {
+                    writer.beginNewBlock();
+                    writer.write(Collections.singleton(TestUtil.generateEvent(1L)), 1L);
+                    writer.finishBlock();
+                }
+            }
+        
+            final byte[] data = Files.readAllBytes(journalFile.toPath());
+            final ByteArrayInputStream bais = new ByteArrayInputStream(data);
+            final DataInputStream dis = new DataInputStream(bais);
+
+            final String codecName = dis.readUTF();
+            assertEquals(StandardEventSerializer.CODEC_NAME, codecName);
+            
+            final int version = dis.readInt();
+            assertEquals(1, version);
+            
+            // compression flag
+            assertEquals(true, dis.readBoolean());
+            
+            // read block start
+            for (int i=0; i < 1024; i++) {
+                final CompressionInputStream decompressedIn = new CompressionInputStream(bais);
+                final StandardEventDeserializer deserializer = new StandardEventDeserializer();
+                
+                final DataInputStream decompressedDis = new DataInputStream(decompressedIn);
+                final int eventLength = decompressedDis.readInt();
+                assertEquals(131, eventLength);
+                final ProvenanceEventRecord event = deserializer.deserialize(decompressedDis, 0);
+                assertEquals(1, event.getEventId());
+                assertEquals(ProvenanceEventType.CREATE, event.getEventType());
+                
+                if ( i == 1023 ) {
+                    assertEquals(-1, decompressedIn.read());
+                }
+            }
+        } finally {
+            journalFile.delete();
+        }
+    }
+    
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocReader.java
new file mode 100644
index 0000000..d5c4037
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocReader.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.toc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.UUID;
+
+import org.junit.Test;
+
+public class TestStandardTocReader {
+
+    @Test
+    public void testDetectsCompression() throws IOException {
+        final File file = new File("target/" + UUID.randomUUID().toString());
+        try (final OutputStream out = new FileOutputStream(file)) {
+            out.write(0);
+            out.write(0);
+        }
+        
+        try {
+            try(final StandardTocReader reader = new StandardTocReader(file)) {
+                assertFalse(reader.isCompressed());
+            }
+        } finally {
+            file.delete();
+        }
+        
+        
+        try (final OutputStream out = new FileOutputStream(file)) {
+            out.write(0);
+            out.write(1);
+        }
+        
+        try {
+            try(final StandardTocReader reader = new StandardTocReader(file)) {
+                assertTrue(reader.isCompressed());
+            }
+        } finally {
+            file.delete();
+        }
+    }
+    
+    
+    @Test
+    public void testGetBlockIndex() throws IOException {
+        final File file = new File("target/" + UUID.randomUUID().toString());
+        try (final OutputStream out = new FileOutputStream(file);
+             final DataOutputStream dos = new DataOutputStream(out)) {
+            out.write(0);
+            out.write(0);
+            
+            for (int i=0; i < 1024; i++) {
+                dos.writeLong(i * 1024L);
+            }
+        }
+        
+        try {
+            try(final StandardTocReader reader = new StandardTocReader(file)) {
+                assertFalse(reader.isCompressed());
+                
+                for (int i=0; i < 1024; i++) {
+                    assertEquals(i * 1024, reader.getBlockOffset(i));
+                }
+            }
+        } finally {
+            file.delete();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
index f4f9d12..777130e 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
@@ -64,7 +64,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
     // default property values
     public static final int DEFAULT_BUFFER_SIZE = 10000;
 
-    private final RingBuffer<ProvenanceEventRecord> ringBuffer;
+    private final RingBuffer<StoredProvenanceEvent> ringBuffer;
     private final List<SearchableField> searchableFields;
     private final List<SearchableField> searchableAttributes;
     private final ExecutorService queryExecService;
@@ -123,17 +123,17 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
     }
 
     @Override
-    public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
+    public void registerEvents(final Collection<ProvenanceEventRecord> events) {
         for (final ProvenanceEventRecord event : events) {
             registerEvent(event);
         }
     }
 
     @Override
-    public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
-        return ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
+    public List<StoredProvenanceEvent> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
+        return ringBuffer.getSelectedElements(new Filter<StoredProvenanceEvent>() {
             @Override
-            public boolean select(final ProvenanceEventRecord value) {
+            public boolean select(final StoredProvenanceEvent value) {
                 return value.getEventId() >= firstRecordId;
             }
         }, maxRecords);
@@ -145,21 +145,22 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         return (newest == null) ? null : newest.getEventId();
     }
 
-    public ProvenanceEventRecord getEvent(final String identifier) throws IOException {
-        final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
+    public StoredProvenanceEvent getEvent(final String identifier) throws IOException {
+        final List<StoredProvenanceEvent> records = ringBuffer.getSelectedElements(new Filter<StoredProvenanceEvent>() {
             @Override
-            public boolean select(final ProvenanceEventRecord event) {
+            public boolean select(final StoredProvenanceEvent event) {
                 return identifier.equals(event.getFlowFileUuid());
             }
         }, 1);
+        
         return records.isEmpty() ? null : records.get(0);
     }
 
     @Override
-    public ProvenanceEventRecord getEvent(final long id) {
-        final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(new Filter<ProvenanceEventRecord>() {
+    public StoredProvenanceEvent getEvent(final long id) {
+        final List<StoredProvenanceEvent> records = ringBuffer.getSelectedElements(new Filter<StoredProvenanceEvent>() {
             @Override
-            public boolean select(final ProvenanceEventRecord event) {
+            public boolean select(final StoredProvenanceEvent event) {
                 return event.getEventId() == id;
             }
         }, 1);
@@ -407,6 +408,44 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         throw new UnsupportedOperationException();
     }
 
+
+    @Override
+    public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+        final List<StoredProvenanceEvent> events = new ArrayList<>(storageLocations.size());
+        for ( final StorageLocation location : storageLocations ) {
+            if ( !(location instanceof IdLocation) ) {
+                throw new IllegalArgumentException("Illegal Storage Location");
+            }
+            
+            final long id = ((IdLocation) location).getId();
+            final StoredProvenanceEvent event = getEvent(id);
+            if ( event != null ) {
+                events.add(event);
+            }
+        }
+        return events;
+    }
+
+    @Override
+    public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+        if ( !(location instanceof IdLocation) ) {
+            throw new IllegalArgumentException("Illegal Storage Location");
+        }
+        
+        final long id = ((IdLocation) location).getId();
+        return getEvent(id);
+    }
+
+    @Override
+    public Long getEarliestEventTime() throws IOException {
+        final List<StoredProvenanceEvent> events = getEvents(0L, 1);
+        if ( events.isEmpty() ) {
+            return null;
+        }
+        
+        return events.get(0).getEventTime();
+    }
+    
     @Override
     public ComputeLineageSubmission submitExpandParents(final long eventId) {
         final ProvenanceEventRecord event = getEvent(eventId);
@@ -432,9 +471,6 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         }
     }
 
-    public Lineage expandSpawnEventChildren(final String identifier) {
-        throw new UnsupportedOperationException();
-    }
 
     @Override
     public ComputeLineageSubmission submitExpandChildren(final long eventId) {
@@ -465,9 +501,9 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         final AsyncLineageSubmission result = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, 1);
         lineageSubmissionMap.put(result.getLineageIdentifier(), result);
 
-        final Filter<ProvenanceEventRecord> filter = new Filter<ProvenanceEventRecord>() {
+        final Filter<StoredProvenanceEvent> filter = new Filter<StoredProvenanceEvent>() {
             @Override
-            public boolean select(final ProvenanceEventRecord event) {
+            public boolean select(final StoredProvenanceEvent event) {
                 if (flowFileUuids.contains(event.getFlowFileUuid())) {
                     return true;
                 }
@@ -495,12 +531,12 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
 
     private static class QueryRunnable implements Runnable {
 
-        private final RingBuffer<ProvenanceEventRecord> ringBuffer;
+        private final RingBuffer<StoredProvenanceEvent> ringBuffer;
         private final Filter<ProvenanceEventRecord> filter;
         private final AsyncQuerySubmission submission;
         private final int maxRecords;
 
-        public QueryRunnable(final RingBuffer<ProvenanceEventRecord> ringBuffer, final Filter<ProvenanceEventRecord> filter, final int maxRecords, final AsyncQuerySubmission submission) {
+        public QueryRunnable(final RingBuffer<StoredProvenanceEvent> ringBuffer, final Filter<ProvenanceEventRecord> filter, final int maxRecords, final AsyncQuerySubmission submission) {
             this.ringBuffer = ringBuffer;
             this.filter = filter;
             this.submission = submission;
@@ -511,10 +547,10 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         public void run() {
             // Retrieve the most recent results and count the total number of matches
             final IntegerHolder matchingCount = new IntegerHolder(0);
-            final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>(maxRecords);
-            ringBuffer.forEach(new ForEachEvaluator<ProvenanceEventRecord>() {
+            final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>(maxRecords);
+            ringBuffer.forEach(new ForEachEvaluator<StoredProvenanceEvent>() {
                 @Override
-                public boolean evaluate(final ProvenanceEventRecord record) {
+                public boolean evaluate(final StoredProvenanceEvent record) {
                     if (filter.select(record)) {
                         if (matchingCount.incrementAndGet() <= maxRecords) {
                             matchingRecords.add(record);
@@ -532,20 +568,21 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
 
     private static class ComputeLineageRunnable implements Runnable {
 
-        private final RingBuffer<ProvenanceEventRecord> ringBuffer;
-        private final Filter<ProvenanceEventRecord> filter;
+        private final RingBuffer<StoredProvenanceEvent> ringBuffer;
+        private final Filter<StoredProvenanceEvent> filter;
         private final AsyncLineageSubmission submission;
 
-        public ComputeLineageRunnable(final RingBuffer<ProvenanceEventRecord> ringBuffer, final Filter<ProvenanceEventRecord> filter, final AsyncLineageSubmission submission) {
+        public ComputeLineageRunnable(final RingBuffer<StoredProvenanceEvent> ringBuffer, final Filter<StoredProvenanceEvent> filter, final AsyncLineageSubmission submission) {
             this.ringBuffer = ringBuffer;
             this.filter = filter;
             this.submission = submission;
         }
 
         @Override
+        @SuppressWarnings({ "unchecked", "rawtypes" })
         public void run() {
-            final List<ProvenanceEventRecord> records = ringBuffer.getSelectedElements(filter);
-            submission.getResult().update(records);
+            final List<StoredProvenanceEvent> records = ringBuffer.getSelectedElements(filter);
+            submission.getResult().update((List) records);
         }
     }
 
@@ -577,7 +614,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         }
     }
 
-    private static class IdEnrichedProvEvent implements ProvenanceEventRecord {
+    private static class IdEnrichedProvEvent implements StoredProvenanceEvent {
 
         private final ProvenanceEventRecord record;
         private final long id;
@@ -741,5 +778,23 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         public Long getPreviousContentClaimOffset() {
             return record.getPreviousContentClaimOffset();
         }
+
+        @Override
+        public StorageLocation getStorageLocation() {
+            return new IdLocation(getEventId());
+        }
     }
+    
+    private static class IdLocation implements StorageLocation {
+        private final long id;
+        
+        public IdLocation(final long id) {
+            this.id = id;
+        }
+        
+        public long getId() {
+            return id;
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
index 3c3e401..fd27470 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
@@ -65,7 +65,7 @@ public class TestVolatileProvenanceRepository {
             repo.registerEvent(builder.build());
         }
 
-        final List<ProvenanceEventRecord> retrieved = repo.getEvents(0L, 12);
+        final List<StoredProvenanceEvent> retrieved = repo.getEvents(0L, 12);
 
         assertEquals(10, retrieved.size());
         for (int i = 0; i < 10; i++) {


[06/12] incubator-nifi git commit: bug fixes and additional pieces of repo implemented

Posted by ma...@apache.org.
bug fixes and additional pieces of repo implemented


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/b95e7569
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/b95e7569
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/b95e7569

Branch: refs/heads/journaling-prov-repo
Commit: b95e7569f75c7beb2fd214d3304ae9630f4a8545
Parents: a68bef6
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Feb 12 19:19:37 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Feb 12 19:19:37 2015 -0500

----------------------------------------------------------------------
 .../apache/nifi/provenance/EventIdLocation.java |  31 +++
 .../provenance/IdEnrichedProvenanceEvent.java   | 175 ++++++++++++++++
 .../nifi/provenance/StandardLineageResult.java  |   2 +-
 .../org/apache/nifi/util/file/FileUtils.java    |   1 -
 .../MockProvenanceEventRepository.java          |  41 +++-
 .../manager/impl/ClusteredEventAccess.java      |  32 ++-
 .../repository/StandardProcessSession.java      | 164 ++++++---------
 .../nifi/controller/tasks/ExpireFlowFiles.java  |  27 ++-
 .../repository/TestStandardProcessSession.java  |  13 +-
 .../nifi/web/controller/ControllerFacade.java   |   6 +-
 .../pom.xml                                     |   5 +
 .../JournalingProvenanceRepository.java         |  31 +--
 .../config/JournalingRepositoryConfig.java      |   2 +-
 .../journals/StandardJournalReader.java         |   2 +-
 .../partition/JournalingPartition.java          |   6 +
 .../partition/QueuingPartitionManager.java      |  39 +++-
 .../journaling/query/QueryManager.java          |  42 ++++
 .../journaling/query/StandardQueryManager.java  | 144 ++++++++++++++
 ...he.nifi.provenance.ProvenanceEventRepository |  15 ++
 .../TestJournalingProvenanceRepository.java     | 144 ++++++++++++++
 .../nifi/provenance/journaling/TestUtil.java    |   8 +
 .../PersistentProvenanceRepository.java         |  80 ++++++--
 .../nifi/provenance/lucene/DocsReader.java      |  21 +-
 .../nifi/provenance/lucene/IndexSearch.java     |   8 +-
 .../nifi/provenance/lucene/LineageQuery.java    |  11 +-
 .../TestPersistentProvenanceRepository.java     |  44 ++---
 .../nifi-provenance-repository-nar/pom.xml      |   4 +
 .../VolatileProvenanceRepository.java           | 198 +------------------
 .../nifi-provenance-repository-bundle/pom.xml   |   6 +
 29 files changed, 909 insertions(+), 393 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/EventIdLocation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/EventIdLocation.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/EventIdLocation.java
new file mode 100644
index 0000000..9cc6c4d
--- /dev/null
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/EventIdLocation.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance;
+
+import org.apache.nifi.provenance.StorageLocation;
+
+public class EventIdLocation implements StorageLocation {
+    private final long id;
+
+    public EventIdLocation(final long id) {
+        this.id = id;
+    }
+    
+    public long getId() {
+        return id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/IdEnrichedProvenanceEvent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/IdEnrichedProvenanceEvent.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/IdEnrichedProvenanceEvent.java
new file mode 100644
index 0000000..4ef0e5d
--- /dev/null
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/IdEnrichedProvenanceEvent.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+
+public class IdEnrichedProvenanceEvent implements StoredProvenanceEvent {
+
+    private final ProvenanceEventRecord event;
+    private final long id;
+    
+    public IdEnrichedProvenanceEvent(final ProvenanceEventRecord event) {
+        this(event, event.getEventId());
+    }
+    
+    public IdEnrichedProvenanceEvent(final ProvenanceEventRecord event, final long id) {
+        this.event = event;
+        this.id = id;
+    }
+    
+    @Override
+    public StorageLocation getStorageLocation() {
+        return new EventIdLocation(id);
+    }
+
+    public long getEventId() {
+        return id;
+    }
+
+    public long getEventTime() {
+        return event.getEventTime();
+    }
+
+    public long getFlowFileEntryDate() {
+        return event.getFlowFileEntryDate();
+    }
+
+    public long getLineageStartDate() {
+        return event.getLineageStartDate();
+    }
+
+    public Set<String> getLineageIdentifiers() {
+        return event.getLineageIdentifiers();
+    }
+
+    public long getFileSize() {
+        return event.getFileSize();
+    }
+
+    public Long getPreviousFileSize() {
+        return event.getPreviousFileSize();
+    }
+
+    public long getEventDuration() {
+        return event.getEventDuration();
+    }
+
+    public ProvenanceEventType getEventType() {
+        return event.getEventType();
+    }
+
+    public Map<String, String> getAttributes() {
+        return event.getAttributes();
+    }
+
+    public Map<String, String> getPreviousAttributes() {
+        return event.getPreviousAttributes();
+    }
+
+    public Map<String, String> getUpdatedAttributes() {
+        return event.getUpdatedAttributes();
+    }
+
+    public String getComponentId() {
+        return event.getComponentId();
+    }
+
+    public String getComponentType() {
+        return event.getComponentType();
+    }
+
+    public String getTransitUri() {
+        return event.getTransitUri();
+    }
+
+    public String getSourceSystemFlowFileIdentifier() {
+        return event.getSourceSystemFlowFileIdentifier();
+    }
+
+    public String getFlowFileUuid() {
+        return event.getFlowFileUuid();
+    }
+
+    public List<String> getParentUuids() {
+        return event.getParentUuids();
+    }
+
+    public List<String> getChildUuids() {
+        return event.getChildUuids();
+    }
+
+    public String getAlternateIdentifierUri() {
+        return event.getAlternateIdentifierUri();
+    }
+
+    public String getDetails() {
+        return event.getDetails();
+    }
+
+    public String getRelationship() {
+        return event.getRelationship();
+    }
+
+    public String getSourceQueueIdentifier() {
+        return event.getSourceQueueIdentifier();
+    }
+
+    public String getContentClaimSection() {
+        return event.getContentClaimSection();
+    }
+
+    public String getPreviousContentClaimSection() {
+        return event.getPreviousContentClaimSection();
+    }
+
+    public String getContentClaimContainer() {
+        return event.getContentClaimContainer();
+    }
+
+    public String getPreviousContentClaimContainer() {
+        return event.getPreviousContentClaimContainer();
+    }
+
+    public String getContentClaimIdentifier() {
+        return event.getContentClaimIdentifier();
+    }
+
+    public String getPreviousContentClaimIdentifier() {
+        return event.getPreviousContentClaimIdentifier();
+    }
+
+    public Long getContentClaimOffset() {
+        return event.getContentClaimOffset();
+    }
+
+    public Long getPreviousContentClaimOffset() {
+        return event.getPreviousContentClaimOffset();
+    }
+
+    @Override
+    public String toString() {
+        return event.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
index afb56e8..0f454bd 100644
--- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java
@@ -178,7 +178,7 @@ public class StandardLineageResult implements ComputeLineageResult {
         }
     }
 
-    public void update(final Collection<ProvenanceEventRecord> records) {
+    public void update(final Collection<StoredProvenanceEvent> records) {
         writeLock.lock();
         try {
             relevantRecords.addAll(records);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
index 41a0557..71dbc79 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
@@ -16,7 +16,6 @@
  */
 package org.apache.nifi.util.file;
 
-import java.io.BufferedInputStream;
 import java.io.Closeable;
 import java.io.File;
 import java.io.FileInputStream;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java
index 241041a..c4caa71 100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceEventRepository.java
@@ -18,6 +18,7 @@ package org.apache.nifi.provenance;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
@@ -30,11 +31,11 @@ import org.apache.nifi.provenance.search.SearchableField;
 
 public class MockProvenanceEventRepository implements ProvenanceEventRepository {
 
-    private final List<ProvenanceEventRecord> records = new ArrayList<>();
+    private final List<StoredProvenanceEvent> records = new ArrayList<>();
     private final AtomicLong idGenerator = new AtomicLong(0L);
 
     @Override
-    public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
+    public void registerEvents(final Collection<ProvenanceEventRecord> events) {
         for (final ProvenanceEventRecord event : events) {
             registerEvent(event);
         }
@@ -50,7 +51,7 @@ public class MockProvenanceEventRepository implements ProvenanceEventRepository
         }
         newRecord.setEventId(idGenerator.getAndIncrement());
 
-        records.add(newRecord);
+        records.add(new IdEnrichedProvenanceEvent(newRecord));
     }
 
     @Override
@@ -58,7 +59,7 @@ public class MockProvenanceEventRepository implements ProvenanceEventRepository
     }
 
     @Override
-    public List<ProvenanceEventRecord> getEvents(long firstRecordId, int maxRecords) throws IOException {
+    public List<StoredProvenanceEvent> getEvents(long firstRecordId, int maxRecords) throws IOException {
         if (firstRecordId > records.size()) {
             return Collections.emptyList();
         }
@@ -92,7 +93,7 @@ public class MockProvenanceEventRepository implements ProvenanceEventRepository
     }
 
     @Override
-    public ProvenanceEventRecord getEvent(long id) throws IOException {
+    public StoredProvenanceEvent getEvent(long id) throws IOException {
         if (id > records.size()) {
             return null;
         }
@@ -128,4 +129,34 @@ public class MockProvenanceEventRepository implements ProvenanceEventRepository
     public ProvenanceEventBuilder eventBuilder() {
         return new StandardProvenanceEventRecord.Builder();
     }
+    
+    @Override
+    public Long getEarliestEventTime() throws IOException {
+        final StoredProvenanceEvent event = getEvent(0);
+        if ( event == null ) {
+            return null;
+        }
+        
+        return event.getEventTime();
+    }
+    
+    @Override
+    public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+        if ( location instanceof EventIdLocation ) {
+            return getEvent( ((EventIdLocation) location).getId() );
+        }
+        throw new IllegalArgumentException("Invalid StorageLocation");
+    }
+    
+    @Override
+    public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+        final List<StoredProvenanceEvent> events = new ArrayList<>(storageLocations.size());
+        for ( final StorageLocation location : storageLocations ) {
+            final StoredProvenanceEvent event = getEvent(location);
+            if ( event != null ) {
+                events.add(event);
+            }
+        }
+        return events;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
index 2015530..7780d04 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java
@@ -18,6 +18,8 @@ package org.apache.nifi.cluster.manager.impl;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.nifi.controller.status.ProcessGroupStatus;
@@ -25,6 +27,8 @@ import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.provenance.ProvenanceEventBuilder;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
 import org.apache.nifi.provenance.search.Query;
 import org.apache.nifi.provenance.search.QuerySubmission;
@@ -59,12 +63,12 @@ public class ClusteredEventAccess implements EventAccess {
             }
 
             @Override
-            public ProvenanceEventRecord getEvent(long eventId) throws IOException {
+            public StoredProvenanceEvent getEvent(long eventId) throws IOException {
                 return null;
             }
 
             @Override
-            public List<ProvenanceEventRecord> getEvents(long startEventId, int maxEvents) throws IOException {
+            public List<StoredProvenanceEvent> getEvents(long startEventId, int maxEvents) throws IOException {
                 return new ArrayList<>();
             }
 
@@ -88,10 +92,6 @@ public class ClusteredEventAccess implements EventAccess {
             }
 
             @Override
-            public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
-            }
-
-            @Override
             public ComputeLineageSubmission retrieveLineageSubmission(final String submissionId) {
                 return null;
             }
@@ -130,6 +130,26 @@ public class ClusteredEventAccess implements EventAccess {
             public void initialize(EventReporter eventReporter) throws IOException {
 
             }
+            
+            @Override
+            public Long getEarliestEventTime() throws IOException {
+                return null;
+            }
+            
+            @Override
+            public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+                return null;
+            }
+            
+            @Override
+            public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+                return Collections.emptyList();
+            }
+            
+            @Override
+            public void registerEvents(final Collection<ProvenanceEventRecord> events) throws IOException {
+                
+            }
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index dcb461c..899fccc 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -28,11 +28,9 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
@@ -53,9 +51,6 @@ import org.apache.nifi.controller.repository.io.LimitedInputStream;
 import org.apache.nifi.controller.repository.io.LongHolder;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.stream.io.BufferedOutputStream;
-import org.apache.nifi.stream.io.NonCloseableInputStream;
-import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.processor.FlowFileFilter;
 import org.apache.nifi.processor.ProcessSession;
@@ -74,6 +69,9 @@ import org.apache.nifi.provenance.ProvenanceEventRepository;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.provenance.ProvenanceReporter;
 import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.NonCloseableInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -299,7 +297,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         resetReadClaim();
 
         final long updateProvenanceStart = System.nanoTime();
-        updateProvenanceRepo(checkpoint);
+        try {
+            updateProvenanceRepo(checkpoint);
+        } catch (final IOException ioe) {
+            rollback();
+            throw new ProcessException("Provenance Repository failed to update", ioe);
+        }
 
         final long claimRemovalStart = System.nanoTime();
         final long updateProvenanceNanos = claimRemovalStart - updateProvenanceStart;
@@ -497,7 +500,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         eventTypes.add(eventType);
     }
     
-    private void updateProvenanceRepo(final Checkpoint checkpoint) {
+    private void updateProvenanceRepo(final Checkpoint checkpoint) throws IOException {
         // Update Provenance Repository
         final ProvenanceEventRepository provenanceRepo = context.getProvenanceRepository();
 
@@ -641,46 +644,16 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         }
 
         final List<ProvenanceEventRecord> autoTermEvents = checkpoint.autoTerminatedEvents;
-        final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
-            final Iterator<ProvenanceEventRecord> recordsToSubmitIterator = recordsToSubmit.iterator();
-            final Iterator<ProvenanceEventRecord> autoTermIterator = autoTermEvents == null ? null : autoTermEvents.iterator();
-
-            @Override
-            public Iterator<ProvenanceEventRecord> iterator() {
-                return new Iterator<ProvenanceEventRecord>() {
-                    @Override
-                    public boolean hasNext() {
-                        return recordsToSubmitIterator.hasNext() || (autoTermIterator != null && autoTermIterator.hasNext());
-                    }
-
-                    @Override
-                    public ProvenanceEventRecord next() {
-                        if (recordsToSubmitIterator.hasNext()) {
-                            final ProvenanceEventRecord rawEvent = recordsToSubmitIterator.next();
-
-                            // Update the Provenance Event Record with all of the info that we know about the event.
-                            // For SEND events, we do not want to update the FlowFile info on the Event, because the event should
-                            // reflect the FlowFile as it was sent to the remote system. However, for other events, we want to use
-                            // the representation of the FlowFile as it is committed, as this is the only way in which it really
-                            // exists in our system -- all other representations are volatile representations that have not been
-                            // exposed.
-                            return enrich(rawEvent, flowFileRecordMap, checkpoint.records, rawEvent.getEventType() != ProvenanceEventType.SEND);
-                        } else if (autoTermIterator != null && autoTermIterator.hasNext()) {
-                            return enrich(autoTermIterator.next(), flowFileRecordMap, checkpoint.records, true);
-                        }
-
-                        throw new NoSuchElementException();
-                    }
-
-                    @Override
-                    public void remove() {
-                        throw new UnsupportedOperationException();
-                    }
-                };
-            }
-        };
-
-        provenanceRepo.registerEvents(iterable);
+        
+        final List<ProvenanceEventRecord> enrichedEvents = new ArrayList<>();
+        for ( final ProvenanceEventRecord record : recordsToSubmit ) {
+            enrichedEvents.add(enrich(record, flowFileRecordMap, checkpoint.records, record.getEventType() != ProvenanceEventType.SEND));
+        }
+        for ( final ProvenanceEventRecord record : autoTermEvents ) {
+            enrichedEvents.add(enrich(record, flowFileRecordMap, checkpoint.records, true));
+        }
+        
+        provenanceRepo.registerEvents(enrichedEvents);
     }
 
     private void updateEventContentClaims(final ProvenanceEventBuilder builder, final FlowFile flowFile, final StandardRepositoryRecord repoRecord) {
@@ -1140,7 +1113,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             final Connection conn = connections.get(context.getNextIncomingConnectionIndex() % connections.size());
             final Set<FlowFileRecord> expired = new HashSet<>();
             final FlowFileRecord flowFile = conn.getFlowFileQueue().poll(expired);
-            removeExpired(expired, conn);
+            
+            try {
+                removeExpired(expired, conn);
+            } catch (final IOException ioe) {
+                throw new ProcessException("Failed to update repositories to remove expired FlowFiles", ioe);
+            }
 
             if (flowFile != null) {
                 registerDequeuedRecord(flowFile, conn);
@@ -1201,7 +1179,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             for (final Connection conn : connections) {
                 final Set<FlowFileRecord> expired = new HashSet<>();
                 final List<FlowFileRecord> newlySelected = poller.poll(conn.getFlowFileQueue(), expired);
-                removeExpired(expired, conn);
+                try {
+                    removeExpired(expired, conn);
+                } catch (final IOException ioe) {
+                    throw new ProcessException("Failed to update repositories to remove expired FlowFiles", ioe);
+                }
 
                 if (newlySelected.isEmpty() && expired.isEmpty()) {
                     continue;
@@ -1571,7 +1553,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         }
     }
 
-    public void expireFlowFiles() {
+    public void expireFlowFiles() throws IOException {
         final Set<FlowFileRecord> expired = new HashSet<>();
         final FlowFileFilter filter = new FlowFileFilter() {
             @Override
@@ -1589,7 +1571,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         }
     }
 
-    private void removeExpired(final Set<FlowFileRecord> flowFiles, final Connection connection) {
+    private void removeExpired(final Set<FlowFileRecord> flowFiles, final Connection connection) throws IOException {
         if (flowFiles.isEmpty()) {
             return;
         }
@@ -1612,7 +1594,31 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
         final Map<String, FlowFileRecord> recordIdMap = new HashMap<>();
         for (final FlowFileRecord flowFile : flowFiles) {
             recordIdMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile);
+        }
+        
+        final Set<ProvenanceEventRecord> expiredEvents = expiredReporter.getEvents();
+        final List<ProvenanceEventRecord> events = new ArrayList<>(expiredEvents.size());
+        for ( final ProvenanceEventRecord event : expiredEvents ) {
+            final StandardProvenanceEventRecord.Builder enriched = new StandardProvenanceEventRecord.Builder().fromEvent(event);
+            final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
+            if (record == null) {
+                continue;
+            }
+
+            final ContentClaim claim = record.getContentClaim();
+            if (claim != null) {
+                enriched.setCurrentContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+                enriched.setPreviousContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+            }
 
+            enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
+            events.add(enriched.build());
+        }
+        
+        context.getProvenanceRepository().registerEvents(events);
+        context.getFlowFileRepository().updateRepository(expiredRecords);
+        
+        for ( final FlowFileRecord flowFile : flowFiles ) {
             final StandardRepositoryRecord record = new StandardRepositoryRecord(connection.getFlowFileQueue(), flowFile);
             record.markForDelete();
             expiredRecords.add(record);
@@ -1623,53 +1629,6 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
             final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
             LOG.info("{} terminated by {} due to FlowFile expiration; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
         }
-
-        try {
-            final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
-                @Override
-                public Iterator<ProvenanceEventRecord> iterator() {
-                    final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator();
-                    final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<ProvenanceEventRecord>() {
-                        @Override
-                        public boolean hasNext() {
-                            return expiredEventIterator.hasNext();
-                        }
-
-                        @Override
-                        public ProvenanceEventRecord next() {
-                            final ProvenanceEventRecord event = expiredEventIterator.next();
-                            final StandardProvenanceEventRecord.Builder enriched = new StandardProvenanceEventRecord.Builder().fromEvent(event);
-                            final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
-                            if (record == null) {
-                                return null;
-                            }
-
-                            final ContentClaim claim = record.getContentClaim();
-                            if (claim != null) {
-                                enriched.setCurrentContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
-                                enriched.setPreviousContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
-                            }
-
-                            enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
-                            return enriched.build();
-                        }
-
-                        @Override
-                        public void remove() {
-                            throw new UnsupportedOperationException();
-                        }
-                    };
-
-                    return enrichingIterator;
-                }
-            };
-
-            context.getProvenanceRepository().registerEvents(iterable);
-            context.getFlowFileRepository().updateRepository(expiredRecords);
-        } catch (final IOException e) {
-            LOG.error("Failed to update FlowFile Repository to record expired records due to {}", e);
-        }
-
     }
 
     private InputStream getInputStream(final FlowFile flowFile, final ContentClaim claim, final long offset) throws ContentNotFoundException {
@@ -2438,7 +2397,14 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
 
         final ProvenanceEventRecord dropEvent = provenanceReporter.drop(suspectRecord.getCurrent(), nfe.getMessage() == null ? "Content Not Found" : nfe.getMessage());
         if (dropEvent != null) {
-            context.getProvenanceRepository().registerEvent(dropEvent);
+            try {
+                context.getProvenanceRepository().registerEvent(dropEvent);
+            } catch (final IOException ioe) {
+                LOG.error("{} Failed to register DROP Provenance event for {} when handling ContentNotFound error due to {}", this, suspectRecord.getCurrent(), ioe.toString());
+                if ( LOG.isDebugEnabled() ) {
+                    LOG.error("", ioe);
+                }
+            }
         }
 
         if (missingClaim == registeredClaim) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
index a351a68..d0020b5 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
@@ -16,9 +16,12 @@
  */
 package org.apache.nifi.controller.tasks;
 
+import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import javax.print.attribute.standard.Severity;
+
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
@@ -29,9 +32,12 @@ import org.apache.nifi.controller.repository.ProcessContext;
 import org.apache.nifi.controller.repository.StandardProcessSession;
 import org.apache.nifi.controller.repository.StandardProcessSessionFactory;
 import org.apache.nifi.controller.scheduling.ProcessContextFactory;
+import org.apache.nifi.events.BulletinFactory;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This task runs through all Connectable Components and goes through its
@@ -39,7 +45,8 @@ import org.apache.nifi.util.FormatUtils;
  * desired side effect of expiring old FlowFiles.
  */
 public class ExpireFlowFiles implements Runnable {
-
+    private static final Logger logger = LoggerFactory.getLogger(ExpireFlowFiles.class);
+    
     private final FlowController flowController;
     private final ProcessContextFactory contextFactory;
 
@@ -51,7 +58,19 @@ public class ExpireFlowFiles implements Runnable {
     @Override
     public void run() {
         final ProcessGroup rootGroup = flowController.getGroup(flowController.getRootGroupId());
-        expireFlowFiles(rootGroup);
+        
+        try {
+            expireFlowFiles(rootGroup);
+        } catch (final Exception e) {
+            logger.error("Failed to expire FlowFiles due to {}", e.toString());
+            if ( logger.isDebugEnabled() ) {
+                logger.error("", e);
+            }
+            
+            flowController.getBulletinRepository().addBulletin(BulletinFactory.createBulletin(
+                    "FlowFile Expiration", Severity.ERROR.getName(), "Could not expire FlowFiles due to " + e));
+            
+        }
     }
 
     private StandardProcessSession createSession(final Connectable connectable) {
@@ -60,7 +79,7 @@ public class ExpireFlowFiles implements Runnable {
         return sessionFactory.createSession();
     }
 
-    private void expireFlowFiles(final Connectable connectable) {
+    private void expireFlowFiles(final Connectable connectable) throws IOException {
         // determine if the incoming connections for this Connectable have Expiration configured.
         boolean expirationConfigured = false;
         for (final Connection incomingConn : connectable.getIncomingConnections()) {
@@ -80,7 +99,7 @@ public class ExpireFlowFiles implements Runnable {
         session.commit();
     }
 
-    private void expireFlowFiles(final ProcessGroup group) {
+    private void expireFlowFiles(final ProcessGroup group) throws IOException {
         for (final ProcessorNode procNode : group.getProcessors()) {
             expireFlowFiles(procNode);
         }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
index 1ff63c5..7ae156c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java
@@ -65,6 +65,7 @@ import org.apache.nifi.provenance.MockProvenanceEventRepository;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventRepository;
 import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -357,7 +358,7 @@ public class TestStandardProcessSession {
         
         session.commit();
         
-        final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 1000);
+        final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 1000);
 
         // We should have a JOIN and 2 ATTRIBUTE_MODIFIED's
         assertEquals(3, events.size());
@@ -412,7 +413,7 @@ public class TestStandardProcessSession {
         session.remove(orig);
         session.commit();
 
-        final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 1000);
+        final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 1000);
         assertEquals(2, events.size());
 
         final ProvenanceEventRecord firstRecord = events.get(0);
@@ -838,7 +839,7 @@ public class TestStandardProcessSession {
         session.transfer(newFlowFile, new Relationship.Builder().name("A").build());
         session.commit();
         
-        final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+        final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
         assertFalse(events.isEmpty());
         assertEquals(1, events.size());
         
@@ -857,7 +858,7 @@ public class TestStandardProcessSession {
         session.transfer(newFlowFile, new Relationship.Builder().name("A").build());
         session.commit();
         
-        final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+        final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
         assertFalse(events.isEmpty());
         assertEquals(1, events.size());
         
@@ -883,7 +884,7 @@ public class TestStandardProcessSession {
         session.transfer(existingFlowFile, new Relationship.Builder().name("A").build());
         session.commit();
         
-        final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+        final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
         assertFalse(events.isEmpty());
         assertEquals(1, events.size());
         
@@ -904,7 +905,7 @@ public class TestStandardProcessSession {
         session.transfer(existingFlowFile, new Relationship.Builder().name("A").build());
         session.commit();
         
-        final List<ProvenanceEventRecord> events = provenanceRepo.getEvents(0L, 10000);
+        final List<StoredProvenanceEvent> events = provenanceRepo.getEvents(0L, 10000);
         assertFalse(events.isEmpty());
         assertEquals(1, events.size());
         

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index b009581..56db464 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -697,9 +697,9 @@ public class ControllerFacade implements ControllerServiceProvider {
             resultsDto.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime()));
 
             // get the oldest available event time
-            final List<ProvenanceEventRecord> firstEvent = provenanceRepository.getEvents(0, 1);
-            if (!firstEvent.isEmpty()) {
-                resultsDto.setOldestEvent(new Date(firstEvent.get(0).getEventTime()));
+            final Long oldestEventTime = provenanceRepository.getEarliestEventTime();
+            if (oldestEventTime != null) {
+                resultsDto.setOldestEvent(new Date(oldestEventTime));
             }
 
             provenanceDto.setResults(resultsDto);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
index 5997281..4e9e9fb 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
@@ -36,5 +36,10 @@
 			<groupId>org.apache.lucene</groupId>
 			<artifactId>lucene-queryparser</artifactId>
 		</dependency>
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-simple</artifactId>
+			<scope>test</scope>
+		</dependency>
 	</dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
index 2130e73..7911d73 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
@@ -55,6 +55,8 @@ import org.apache.nifi.provenance.journaling.partition.PartitionAction;
 import org.apache.nifi.provenance.journaling.partition.PartitionManager;
 import org.apache.nifi.provenance.journaling.partition.QueuingPartitionManager;
 import org.apache.nifi.provenance.journaling.partition.VoidPartitionAction;
+import org.apache.nifi.provenance.journaling.query.QueryManager;
+import org.apache.nifi.provenance.journaling.query.StandardQueryManager;
 import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
 import org.apache.nifi.provenance.journaling.toc.TocReader;
 import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
@@ -67,15 +69,17 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class JournalingProvenanceRepository implements ProvenanceEventRepository {
+    public static final String BLOCK_SIZE = "nifi.provenance.block.size";
+    
     private static final Logger logger = LoggerFactory.getLogger(JournalingProvenanceRepository.class);
     
     private final JournalingRepositoryConfig config;
-    private final PartitionManager partitionManager;
     private final AtomicLong idGenerator = new AtomicLong(0L);
-    
-    private EventReporter eventReporter;    // effectively final
     private final ExecutorService executor;
     
+    private EventReporter eventReporter;    // effectively final
+    private PartitionManager partitionManager;  // effectively final
+    private QueryManager queryManager;    // effectively final
     
     public JournalingProvenanceRepository() throws IOException {
         this(createConfig());
@@ -84,7 +88,6 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     public JournalingProvenanceRepository(final JournalingRepositoryConfig config) throws IOException {
         this.config = config;
         this.executor = Executors.newFixedThreadPool(config.getThreadPoolSize());
-        this.partitionManager = new QueuingPartitionManager(config, executor);
     }
     
     
@@ -110,7 +113,8 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
         final boolean compressOnRollover = Boolean.parseBoolean(properties.getProperty(NiFiProperties.PROVENANCE_COMPRESS_ON_ROLLOVER));
         final String indexedFieldString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_FIELDS);
         final String indexedAttrString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_ATTRIBUTES);
-
+        final int blockSize = properties.getIntegerProperty(BLOCK_SIZE, 1000);
+        
         final Boolean alwaysSync = Boolean.parseBoolean(properties.getProperty("nifi.provenance.repository.always.sync", "false"));
 
         final List<SearchableField> searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true);
@@ -137,7 +141,8 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
         config.setMaxStorageCapacity(maxStorageBytes);
         config.setThreadPoolSize(queryThreads);
         config.setPartitionCount(journalCount);
-
+        config.setBlockSize(blockSize);
+        
         if (shardSize != null) {
             config.setDesiredIndexSize(DataUnit.parseDataSize(shardSize, DataUnit.B).longValue());
         }
@@ -150,6 +155,9 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     @Override
     public synchronized void initialize(final EventReporter eventReporter) throws IOException {
         this.eventReporter = eventReporter;
+        
+        this.partitionManager = new QueuingPartitionManager(config, executor);
+        this.queryManager = new StandardQueryManager(partitionManager, config, 10);
     }
 
     @Override
@@ -328,14 +336,12 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     
     @Override
     public QuerySubmission submitQuery(final Query query) {
-        // TODO Auto-generated method stub
-        return null;
+        return queryManager.submitQuery(query);
     }
 
     @Override
     public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
-        // TODO Auto-generated method stub
-        return null;
+        return queryManager.retrieveQuerySubmission(queryIdentifier);
     }
 
     @Override
@@ -364,7 +370,10 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
 
     @Override
     public void close() throws IOException {
-        partitionManager.shutdown();
+        if ( partitionManager != null ) {
+            partitionManager.shutdown();
+        }
+        
         executor.shutdown();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
index 6dd7be9..8998932 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
@@ -34,7 +34,7 @@ public class JournalingRepositoryConfig {
     private long journalCapacity = 1024L * 1024L * 5L;   // 5 MB
     private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB
     private int partitionCount = 16;
-    private int blockSize = 100;
+    private int blockSize = 5000;
 
     private List<SearchableField> searchableFields = new ArrayList<>();
     private List<SearchableField> searchableAttributes = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
index 82ef39b..2ec5131 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
@@ -163,7 +163,7 @@ public class StandardJournalReader implements JournalReader {
             }
         }
         
-        throw new IOException("Could not find event with ID " + eventId);
+        throw new IOException("Could not find event with ID " + eventId + " in " + this);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
index 51f84a2..651c41e 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
@@ -214,6 +214,7 @@ public class JournalingPartition implements Partition {
         final File journalFile = new File(journalsDir, firstEventId + JOURNAL_FILE_EXTENSION);
         journalWriter = new StandardJournalWriter(firstEventId, journalFile, false, new StandardEventSerializer());
         tocWriter = new StandardTocWriter(QueryUtils.getTocFile(journalFile), false);
+        tocWriter.addBlockOffset(journalWriter.getSize());
         numEventsAtEndOfLastBlock = 0;
     }
     
@@ -421,4 +422,9 @@ public class JournalingPartition implements Partition {
     public Long getEarliestEventTime() throws IOException {
         return earliestEventTime;
     }
+    
+    @Override
+    public String toString() {
+        return "Partition[section=" + sectionName + "]";
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
index 4ac0fc6..51d90e2 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
@@ -35,9 +35,13 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
 import org.apache.nifi.util.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class QueuingPartitionManager implements PartitionManager {
-
+    
+    private static final Logger logger = LoggerFactory.getLogger(QueuingPartitionManager.class);
+    
     private final JournalingRepositoryConfig config;
     private final BlockingQueue<Partition> partitionQueue;
     private final JournalingPartition[] partitionArray;
@@ -180,6 +184,39 @@ public class QueuingPartitionManager implements PartitionManager {
     
     @Override
     public void withEachPartition(final VoidPartitionAction action, final boolean async) {
+        // TODO: Do not use blacklisted partitions.
+        final Map<Partition, Future<?>> futures = new HashMap<>(partitionArray.length);
+        for ( final Partition partition : partitionArray ) {
+            final Runnable runnable = new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        action.perform(partition);
+                    } catch (final Throwable t) {
+                        logger.error("Failed to perform action against " + partition + " due to " + t);
+                        if ( logger.isDebugEnabled() ) {
+                            logger.error("", t);
+                        }
+                    }
+                }
+            };
+            
+            final Future<?> future = executor.submit(runnable);
+            futures.put(partition, future);
+        }
         
+        if ( !async ) {
+            for ( final Map.Entry<Partition, Future<?>> entry : futures.entrySet() ) {
+                try {
+                    // throw any exception thrown by runnable
+                    entry.getValue().get();
+                } catch (final ExecutionException ee) {
+                    final Throwable cause = ee.getCause();
+                    throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
+                } catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/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
new file mode 100644
index 0000000..4edc6ad
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/QueryManager.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.query;
+
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+
+public interface QueryManager {
+    /**
+     * Submits an asynchronous request to process the given query, returning an
+     * identifier that can be used to fetch the results at a later time
+     *
+     * @param query
+     * @return
+     */
+    QuerySubmission submitQuery(Query query);
+    
+    /**
+     * Returns the QueryResult associated with the given identifier, if the
+     * query has finished processing. If the query has not yet finished running,
+     * returns <code>null</code>.
+     *
+     * @param queryIdentifier
+     *
+     * @return
+     */
+    QuerySubmission retrieveQuerySubmission(String queryIdentifier);
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/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
new file mode 100644
index 0000000..4cce231
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/query/StandardQueryManager.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.query;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.nifi.provenance.AsyncQuerySubmission;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.QueryUtils;
+import org.apache.nifi.provenance.journaling.index.SearchResult;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.partition.Partition;
+import org.apache.nifi.provenance.journaling.partition.PartitionManager;
+import org.apache.nifi.provenance.journaling.partition.VoidPartitionAction;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StandardQueryManager implements QueryManager {
+    private static final Logger logger = LoggerFactory.getLogger(StandardQueryManager.class);
+    
+    private final int maxConcurrentQueries;
+    private final JournalingRepositoryConfig config;
+    private final PartitionManager partitionManager;
+    private final ConcurrentMap<String, AsyncQuerySubmission> querySubmissionMap = new ConcurrentHashMap<>();
+    
+    public StandardQueryManager(final PartitionManager partitionManager, final JournalingRepositoryConfig config, final int maxConcurrentQueries) {
+        this.config = config;
+        this.maxConcurrentQueries = maxConcurrentQueries;
+        this.partitionManager = partitionManager;
+    }
+    
+    @Override
+    public QuerySubmission submitQuery(final Query query) {
+        final int numQueries = querySubmissionMap.size();
+        if (numQueries > maxConcurrentQueries) {
+            throw new IllegalStateException("Cannot process query because there are currently " + numQueries + " queries whose results have not been deleted (likely due to poorly behaving clients not issuing DELETE requests). Please try again later.");
+        }
+
+        if (query.getEndDate() != null && query.getStartDate() != null && query.getStartDate().getTime() > query.getEndDate().getTime()) {
+            throw new IllegalArgumentException("Query End Time cannot be before Query Start Time");
+        }
+
+        if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) {
+            final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
+
+            querySubmissionMap.put(query.getIdentifier(), result);
+            return result;
+        }
+
+        final AtomicInteger retrievalCount = new AtomicInteger(query.getMaxResults());
+        final AsyncQuerySubmission submission = new AsyncQuerySubmission(query, config.getPartitionCount()) {
+            @Override
+            public void cancel() {
+                super.cancel();
+                querySubmissionMap.remove(query.getIdentifier());
+            }
+        };
+        
+        querySubmissionMap.put(query.getIdentifier(), submission);
+
+        partitionManager.withEachPartition(new VoidPartitionAction() {
+            @SuppressWarnings({ "rawtypes", "unchecked" })
+            @Override
+            public void perform(final Partition partition) throws IOException {
+                logger.debug("Running {} against {}", query, partition);
+                
+                try (final EventIndexSearcher searcher = partition.newIndexSearcher()) {
+                    final SearchResult searchResult = searcher.search(query);
+                    logger.debug("{} has {} hits against {} over {} files", query, searchResult.getTotalCount(), partition, searchResult.getLocations().size());
+                    
+                    final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>();
+                    final Map<File, List<JournaledStorageLocation>> locationMap = QueryUtils.orderLocations((List) searchResult.getLocations(), config);
+                    for ( final Map.Entry<File, List<JournaledStorageLocation>> entry : locationMap.entrySet() ) {
+                        final File journalFile = entry.getKey();
+                        final List<JournaledStorageLocation> locations = entry.getValue();
+                        
+                        if ( retrievalCount.get() <= 0 ) {
+                            break;
+                        }
+                        
+                        try (final JournalReader reader = new StandardJournalReader(journalFile);
+                             final TocReader tocReader = new StandardTocReader(QueryUtils.getTocFile(journalFile))) {
+                            
+                            for ( final JournaledStorageLocation location : locations ) {
+                                final long blockOffset = tocReader.getBlockOffset(location.getBlockIndex());
+                                final ProvenanceEventRecord event = reader.getEvent(blockOffset, location.getEventId());
+                                matchingRecords.add(new JournaledProvenanceEvent(event, location));
+                                
+                                final int recordsLeft = retrievalCount.decrementAndGet();
+                                if ( recordsLeft <= 0 ) {
+                                    break;
+                                }
+                            }
+                        }
+                    }
+                    
+                    logger.debug("Finished executing {} against {}", query, partition);
+                    submission.getResult().update(matchingRecords, searchResult.getTotalCount());
+                } catch (final Exception e) {
+                    submission.getResult().setError("Failed to query " + partition + " due to " + e.toString());
+                    throw e;
+                }
+            }
+        }, true);
+        
+        return submission;
+    }
+
+    @Override
+    public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
+        return querySubmissionMap.get(queryIdentifier);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceEventRepository
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceEventRepository b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceEventRepository
new file mode 100644
index 0000000..e224c51
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceEventRepository
@@ -0,0 +1,15 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+org.apache.nifi.provenance.journaling.JournalingProvenanceRepository
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/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
new file mode 100644
index 0000000..a547a8a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/TestJournalingProvenanceRepository.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+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.BeforeClass;
+import org.junit.Test;
+
+public class TestJournalingProvenanceRepository {
+
+    
+    @BeforeClass
+    public static void setupLogging() {
+        System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance.journaling", "DEBUG");
+    }
+    
+    @Test
+    public void testStoreAndRetrieve() throws IOException {
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        config.setPartitionCount(3);
+        
+        try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            final Map<String, String> attributes = new HashMap<>();
+            
+            for (int i=0; i < 10; i++) {
+                attributes.put("i", String.valueOf(i));
+                repo.registerEvent(TestUtil.generateEvent(i, attributes));
+            }
+            
+            // retrieve records one at a time.
+            for (int i=0; i < 10; i++) {
+                final StoredProvenanceEvent event = repo.getEvent(i);
+                assertNotNull(event);
+                assertEquals((long) i, event.getEventId());
+                assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+            }
+            
+            final List<StoredProvenanceEvent> events = repo.getEvents(0, 1000);
+            assertNotNull(events);
+            assertEquals(10, events.size());
+            for (int i=0; i < 10; i++) {
+                final StoredProvenanceEvent event = events.get(i);
+                assertNotNull(event);
+                assertEquals((long) i, event.getEventId());
+                assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+            }
+        } finally {
+            for ( final File file : containers.values() ) {
+                if ( file.exists() ) {
+                    FileUtils.deleteFile(file, true);
+                }
+            }
+        }
+    }
+    
+    
+    @Test(timeout=10000000)
+    public void testSearchByUUID() throws IOException, InterruptedException {
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(3);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+            
+            final Map<String, String> attributes = new HashMap<>();
+            
+            for (int i=0; i < 10; i++) {
+                attributes.put("i", String.valueOf(i));
+                repo.registerEvent(TestUtil.generateEvent(i, attributes));
+            }
+            
+            final Query query = new Query("query");
+            query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.FlowFileUUID, "00000000-0000-0000-0000-000000000005"));
+            final QuerySubmission submission = repo.submitQuery(query);
+            assertNotNull(submission);
+            
+            final QueryResult result = submission.getResult();
+            while ( !result.isFinished() ) {
+                Thread.sleep(50L);
+            }
+            
+            assertNull(result.getError());
+            final List<StoredProvenanceEvent> matches = result.getMatchingEvents();
+            assertNotNull(matches);
+            assertEquals(1, matches.size());
+            
+            final StoredProvenanceEvent event = matches.get(0);
+            assertEquals(5, event.getEventId());
+            assertEquals("00000000-0000-0000-0000-000000000005", event.getFlowFileUuid());
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/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 45b7338..6d05f7a 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
@@ -16,6 +16,8 @@
  */
 package org.apache.nifi.provenance.journaling;
 
+import java.util.Collections;
+import java.util.Map;
 import java.util.UUID;
 
 import org.apache.nifi.provenance.ProvenanceEventRecord;
@@ -23,7 +25,12 @@ import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.provenance.StandardProvenanceEventRecord;
 
 public class TestUtil {
+    
     public static ProvenanceEventRecord generateEvent(final long id) {
+        return generateEvent(id, Collections.<String, String>emptyMap());
+    }
+    
+    public static ProvenanceEventRecord generateEvent(final long id, final Map<String, String> attributes) {
         // Create prov event to add to the stream
         final ProvenanceEventRecord event = new StandardProvenanceEventRecord.Builder()
             .setEventType(ProvenanceEventType.CREATE)
@@ -34,6 +41,7 @@ public class TestUtil {
             .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
             .setLineageStartDate(System.currentTimeMillis() - 2000L)
             .setCurrentContentClaim(null, null, null, null, 0L)
+            .setAttributes(null, attributes == null ? Collections.<String, String>emptyMap() : attributes)
             .build();
         
         return event;


[02/12] incubator-nifi git commit: implemented ability to persist and recover records

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventSerializer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventSerializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventSerializer.java
new file mode 100644
index 0000000..fae427e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventSerializer.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.io;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+public class StandardEventSerializer implements Serializer {
+    public static final String CODEC_NAME = "StandardProvCodec";
+    
+    @Override
+    public int getVersion() {
+        return 1;
+    }
+    
+    @Override
+    public String getCodecName() {
+        return CODEC_NAME;
+    }
+    
+    @Override
+    public void serialize(final ProvenanceEventRecord event, final DataOutputStream out) throws IOException {
+        final ProvenanceEventType recordType = event.getEventType();
+
+        out.writeUTF(event.getEventType().name());
+        out.writeLong(event.getEventTime());
+        out.writeLong(event.getFlowFileEntryDate());
+        out.writeLong(event.getEventDuration());
+
+        writeUUIDs(out, event.getLineageIdentifiers());
+        out.writeLong(event.getLineageStartDate());
+
+        writeNullableString(out, event.getComponentId());
+        writeNullableString(out, event.getComponentType());
+        writeUUID(out, event.getFlowFileUuid());
+        writeNullableString(out, event.getDetails());
+
+        // Write FlowFile attributes
+        final Map<String, String> attrs = event.getPreviousAttributes();
+        out.writeInt(attrs.size());
+        for (final Map.Entry<String, String> entry : attrs.entrySet()) {
+            writeLongString(out, entry.getKey());
+            writeLongString(out, entry.getValue());
+        }
+
+        final Map<String, String> attrUpdates = event.getUpdatedAttributes();
+        out.writeInt(attrUpdates.size());
+        for (final Map.Entry<String, String> entry : attrUpdates.entrySet()) {
+            writeLongString(out, entry.getKey());
+            writeLongNullableString(out, entry.getValue());
+        }
+
+        // If Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'. 
+        if (event.getContentClaimSection() != null && event.getContentClaimContainer() != null && event.getContentClaimIdentifier() != null) {
+            out.writeBoolean(true);
+            out.writeUTF(event.getContentClaimContainer());
+            out.writeUTF(event.getContentClaimSection());
+            out.writeUTF(event.getContentClaimIdentifier());
+            if (event.getContentClaimOffset() == null) {
+                out.writeLong(0L);
+            } else {
+                out.writeLong(event.getContentClaimOffset());
+            }
+            out.writeLong(event.getFileSize());
+        } else {
+            out.writeBoolean(false);
+        }
+
+        // If Previous Content Claim Info is present, write out a 'TRUE' followed by claim info. Else, write out 'false'.
+        if (event.getPreviousContentClaimSection() != null && event.getPreviousContentClaimContainer() != null && event.getPreviousContentClaimIdentifier() != null) {
+            out.writeBoolean(true);
+            out.writeUTF(event.getPreviousContentClaimContainer());
+            out.writeUTF(event.getPreviousContentClaimSection());
+            out.writeUTF(event.getPreviousContentClaimIdentifier());
+            if (event.getPreviousContentClaimOffset() == null) {
+                out.writeLong(0L);
+            } else {
+                out.writeLong(event.getPreviousContentClaimOffset());
+            }
+
+            if (event.getPreviousFileSize() == null) {
+                out.writeLong(0L);
+            } else {
+                out.writeLong(event.getPreviousFileSize());
+            }
+        } else {
+            out.writeBoolean(false);
+        }
+
+        // write out the identifier of the destination queue.
+        writeNullableString(out, event.getSourceQueueIdentifier());
+
+        // Write type-specific info
+        if (recordType == ProvenanceEventType.FORK || recordType == ProvenanceEventType.JOIN || recordType == ProvenanceEventType.CLONE || recordType == ProvenanceEventType.REPLAY) {
+            writeUUIDs(out, event.getParentUuids());
+            writeUUIDs(out, event.getChildUuids());
+        } else if (recordType == ProvenanceEventType.RECEIVE) {
+            writeNullableString(out, event.getTransitUri());
+            writeNullableString(out, event.getSourceSystemFlowFileIdentifier());
+        } else if (recordType == ProvenanceEventType.SEND) {
+            writeNullableString(out, event.getTransitUri());
+        } else if (recordType == ProvenanceEventType.ADDINFO) {
+            writeNullableString(out, event.getAlternateIdentifierUri());
+        } else if (recordType == ProvenanceEventType.ROUTE) {
+            writeNullableString(out, event.getRelationship());
+        }
+    }
+
+    private void writeNullableString(final DataOutputStream out, final String toWrite) throws IOException {
+        if (toWrite == null) {
+            out.writeBoolean(false);
+        } else {
+            out.writeBoolean(true);
+            out.writeUTF(toWrite);
+        }
+    }
+
+    private void writeLongNullableString(final DataOutputStream out, final String toWrite) throws IOException {
+        if (toWrite == null) {
+            out.writeBoolean(false);
+        } else {
+            out.writeBoolean(true);
+            writeLongString(out, toWrite);
+        }
+    }
+
+    private void writeLongString(final DataOutputStream out, final String value) throws IOException {
+        final byte[] bytes = value.getBytes("UTF-8");
+        out.writeInt(bytes.length);
+        out.write(bytes);
+    }
+    
+    static void writeUUID(final DataOutputStream out, final String uuid) throws IOException {
+        final UUID uuidObj = UUID.fromString(uuid);
+        out.writeLong(uuidObj.getMostSignificantBits());
+        out.writeLong(uuidObj.getLeastSignificantBits());
+    }
+
+    static void writeUUIDs(final DataOutputStream out, final Collection<String> list) throws IOException {
+        if (list == null) {
+            out.writeInt(0);
+        } else {
+            out.writeInt(list.size());
+            for (final String value : list) {
+                writeUUID(out, value);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
new file mode 100644
index 0000000..535d1dd
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.journals;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface JournalReader extends Closeable {
+
+    /**
+     * Retrieve a specific event from the journal, given the offset of the Block and the ID of the event
+     * @param blockOffset
+     * @param eventId
+     * @return
+     * @throws IOException
+     */
+    ProvenanceEventRecord getEvent(long blockOffset, long eventId) throws IOException;
+    
+    /**
+     * Retrieve the next event in the journal, or <code>null</code> if no more events exist
+     * @return
+     * @throws IOException
+     */
+    ProvenanceEventRecord nextEvent() throws IOException;
+
+    /**
+     * Returns the current byte offset into the Journal from which the next event (if any) will be read
+     * @return
+     */
+    long getPosition();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalWriter.java
new file mode 100644
index 0000000..5108f49
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalWriter.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.journals;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+/**
+ * Responsible for writing events to an append-only journal, or write-ahead-log. Events are written in "Blocks."
+ * These Blocks are used so that if we are compressing data, we can compress individual Blocks. This allows us
+ * to store a "Block Index" so that we can quickly lookup the start of a Block when reading the data to quickly
+ * obtain the data that we need.
+ */
+public interface JournalWriter extends Closeable {
+    
+    /**
+     * Returns the identifier of this journal. The identifier is unique per 'section' of the repository
+     * @return
+     */
+    long getJournalId();
+    
+    /**
+     * Writes the given events to the journal and assigns the events sequential ID's starting with the
+     * ID given
+     * 
+     * @param records
+     * @param firstRecordId
+     * @return
+     * @throws IOException
+     */
+    void write(Collection<ProvenanceEventRecord> events, long firstEventId) throws IOException;
+    
+    /**
+     * Returns the File that the Journal is writing to
+     */
+    File getJournalFile();
+    
+    /**
+     * Synchronizes changes to the underlying file system
+     * @throws IOException
+     */
+    void sync() throws IOException;
+    
+    /**
+     * Returns the size of the journal
+     * @return
+     */
+    long getSize();
+    
+    /**
+     * Returns the number of events that have been written to this journal
+     * @return
+     */
+    int getEventCount();
+    
+    /**
+     * Returns the amount of time that has elapsed since the point at which the writer was created.
+     * @param timeUnit
+     * @return
+     */
+    long getAge(TimeUnit timeUnit);
+    
+    /**
+     * Marks the end of a Block in the output file. If the previous Block has been finished and no new
+     * Block has been started, this method will return silently without doing anything.
+     * @throws IOException
+     */
+    void finishBlock() throws IOException;
+    
+    /**
+     * Starts a new Block in the output file. If a Block has already been started, this method throws
+     * an IllegalStateException
+     * 
+     * @throws IOException
+     */
+    void beginNewBlock() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
new file mode 100644
index 0000000..82ef39b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.journals;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.Deserializer;
+import org.apache.nifi.provenance.journaling.io.Deserializers;
+import org.apache.nifi.remote.io.CompressionInputStream;
+import org.apache.nifi.stream.io.ByteCountingInputStream;
+import org.apache.nifi.stream.io.LimitingInputStream;
+import org.apache.nifi.stream.io.MinimumLengthInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Standard implementation of {@link JournalReader}. This reader reads data that is written
+ * in the format specified by {@link StandardJournalWriter}
+ */
+public class StandardJournalReader implements JournalReader {
+    private static final Logger logger = LoggerFactory.getLogger(StandardJournalReader.class);
+    
+    private final File file;
+    
+    private ByteCountingInputStream compressedStream;
+    private ByteCountingInputStream decompressedStream;
+    
+    private Deserializer deserializer;
+    private int serializationVersion;
+    private boolean compressed;
+    
+    private long lastEventIdRead = -1L;
+    
+    
+    public StandardJournalReader(final File file) throws IOException {
+        this.file = file;
+        resetStreams();
+    }
+    
+    private void resetStreams() throws IOException {
+        final InputStream bufferedIn = new BufferedInputStream(new FileInputStream(file));
+        compressedStream = new ByteCountingInputStream(bufferedIn);
+        final DataInputStream dis = new DataInputStream(compressedStream);
+        final String codecName = dis.readUTF();
+        serializationVersion = dis.readInt();
+        compressed = dis.readBoolean();
+        deserializer = Deserializers.getDeserializer(codecName);
+        
+        resetDecompressedStream();
+    }
+    
+    private void resetDecompressedStream() throws IOException {
+        if ( compressed ) {
+            decompressedStream = new ByteCountingInputStream(new BufferedInputStream(new CompressionInputStream(compressedStream)), compressedStream.getBytesConsumed());
+        } else {
+            decompressedStream = compressedStream;
+        }
+    }
+    
+    @Override
+    public ProvenanceEventRecord nextEvent() throws IOException {
+        return nextEvent(true);
+    }
+    
+    @Override
+    public long getPosition() {
+        return decompressedStream.getBytesConsumed();
+    }
+    
+    private boolean isData(final InputStream in) throws IOException {
+        in.mark(1);
+        final int b = in.read();
+        if ( b < 0 ) {
+            return false;
+        }
+        in.reset();
+        
+        return true;
+    }
+    
+    ProvenanceEventRecord nextEvent(final boolean spanBlocks) throws IOException {
+        boolean isData = isData(decompressedStream);
+        if ( !isData ) {
+            if ( !spanBlocks ) {
+                return null;
+            }
+            
+            // we are allowed to span blocks. We're out of data but if we are compressed, it could
+            // just mean that the block has ended.
+            if ( !compressed ) {
+                return null;
+            }
+            
+            isData = isData(compressedStream);
+            if ( !isData ) {
+                return null;
+            }
+            
+            // There is no data in the compressed InputStream but there is in the underlying stream.
+            // This means we've hit the end of our block. We will create a new CompressionInputStream
+            // so that we can continue reading.
+            resetDecompressedStream();
+        }
+        
+        try {
+            final DataInputStream dis = new DataInputStream(decompressedStream);
+            final int eventLength = dis.readInt();
+            
+            final LimitingInputStream limitingInputStream = new LimitingInputStream(dis, eventLength);
+            final MinimumLengthInputStream minStream = new MinimumLengthInputStream(limitingInputStream, eventLength);
+            final ProvenanceEventRecord event = deserializer.deserialize(new DataInputStream(minStream), serializationVersion);
+            lastEventIdRead = event.getEventId();
+            return event;
+        } catch (final EOFException eof) {
+            logger.warn("{} Found unexpected End-of-File when reading from journal", this); 
+            return null;
+        }
+    }
+    
+    @Override
+    public ProvenanceEventRecord getEvent(final long blockOffset, final long eventId) throws IOException {
+        // If the requested event ID is less than the last event that we read, we need to reset to the beginning
+        // of the file. We do this because we know that the ID's are always increasing, so if we need an ID less
+        // than the previous ID, we have to go backward in the file. We can't do this with streams, so start the
+        // stream over.
+        if ( eventId < lastEventIdRead ) {
+            close();
+            resetStreams();
+        }
+        
+        final long bytesToSkip = blockOffset - compressedStream.getBytesConsumed();
+        if ( bytesToSkip > 0 ) {
+            StreamUtils.skip(compressedStream, bytesToSkip);
+            resetDecompressedStream();
+        }
+        
+        ProvenanceEventRecord event;
+        while ((event = nextEvent()) != null) {
+            if ( event.getEventId() == eventId ) {
+                return event;
+            }
+        }
+        
+        throw new IOException("Could not find event with ID " + eventId);
+    }
+
+    @Override
+    public void close() throws IOException {
+        decompressedStream.close();
+    }
+    
+    @Override
+    public String toString() {
+        return "StandardJournalReader[" + file + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
new file mode 100644
index 0000000..5a289fe
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.journals;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.Serializer;
+import org.apache.nifi.remote.io.CompressionOutputStream;
+import org.apache.nifi.stream.io.ByteCountingOutputStream;
+
+
+/**
+ * <p>
+ * Standard implementation of {@link JournalWriter}.
+ * </p>
+ * 
+ * <p>
+ * Writes out to a journal file using the format:
+ * 
+ * <pre>
+ * &lt;header&gt;
+ * &lt;begin block 1&gt;
+ * &lt;record 1&gt;
+ * &lt;record 2&gt;
+ * &lt;record 3&gt;
+ * &lt;end block 1&gt;
+ * &lt;begin block 2&gt;
+ * &lt;record 4&gt;
+ * &lt;record 5&gt;
+ * &lt;end block 2&gt;
+ * ...
+ * &lt;begin block N&gt;
+ * &lt;record N&gt;
+ * &lt;end block N&gt;
+ * </pre>
+ * 
+ * Where &lt;header&gt; is defined as:
+ * <pre>
+ *  String: serialization codec name (retrieved from serializer)
+ *      --> 2 bytes for length of string
+ *      --> N bytes for actual serialization codec name
+ *  int: serialization version
+ *  boolean: compressed: 1 -> compressed, 0 -> not compressed
+ * </pre>
+ * 
+ * And &lt;record&gt; is defined as:
+ * <pre>
+ * bytes 0-3: int: record length
+ * bytes 4-11: long: record id
+ * bytes 12-N: serialized event according to the applied {@link Serializer}
+ * </pre>
+ * </p>
+ * 
+ * <p>
+ * The structure of the &lt;begin block&gt; and &lt;end block&gt; element depend on whether or not
+ * compression is enabled. If the journal is not compressed, these elements are 0 bytes.
+ * If the journal is compressed, these are the compression header and compression footer, respectively.
+ * </p>
+ * 
+ */
+public class StandardJournalWriter implements JournalWriter {
+    private final long journalId;
+    private final File journalFile;
+    private final boolean compressed;
+    private final Serializer serializer;
+    private final long creationTime = System.nanoTime();
+    
+    private int eventCount;
+    private boolean blockStarted = false;
+    
+    private final FileOutputStream fos;
+    private ByteCountingOutputStream uncompressedStream;
+    private OutputStream compressedStream;
+    private ByteCountingOutputStream out;
+    
+    
+    public StandardJournalWriter(final long journalId, final File journalFile, final boolean compressed, final Serializer serializer) throws IOException {
+        this.journalId = journalId;
+        this.journalFile = journalFile;
+        this.compressed = compressed;
+        this.serializer = serializer;
+        this.fos = new FileOutputStream(journalFile);
+        
+        uncompressedStream = new ByteCountingOutputStream(fos);
+        writeHeader(uncompressedStream);
+        
+        if (compressed) {
+            compressedStream = new CompressionOutputStream(uncompressedStream);
+        } else {
+            compressedStream = fos;
+        }
+        
+        this.out = new ByteCountingOutputStream(compressedStream, uncompressedStream.getBytesWritten());
+    }
+
+    private void writeHeader(final OutputStream out) throws IOException {
+        final DataOutputStream dos = new DataOutputStream(out);
+        dos.writeUTF(serializer.getCodecName());
+        dos.writeInt(serializer.getVersion());
+        dos.writeBoolean(compressed);
+        dos.flush();
+    }
+    
+    @Override
+    public long getJournalId() {
+        return journalId;
+    }
+    
+    @Override
+    public void close() throws IOException {
+        finishBlock();
+        
+        if ( compressedStream != null ) {
+            compressedStream.flush();
+            compressedStream.close();
+        }
+    }
+
+    @Override
+    public void write(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        final DataOutputStream serializerDos = new DataOutputStream(baos);
+        
+        final BufferedOutputStream bos = new BufferedOutputStream(out);
+        final DataOutputStream outDos = new DataOutputStream(bos);
+        
+        try {
+            long id = firstEventId;
+            for ( final ProvenanceEventRecord event : events ) {
+                serializer.serialize(event, serializerDos);
+                serializerDos.flush();
+                
+                final int recordLength = 8 + baos.size();   // record length is length of ID (8 bytes) plus length of serialized record
+                outDos.writeInt(recordLength);
+                outDos.writeLong(id++);
+                baos.writeTo(outDos);
+                baos.reset();
+                
+                eventCount++;
+            }
+        } finally {
+            outDos.flush();
+        }
+    }
+    
+
+    @Override
+    public File getJournalFile() {
+        return journalFile;
+    }
+
+    @Override
+    public void sync() throws IOException {
+        fos.getFD().sync();
+    }
+
+    @Override
+    public long getSize() {
+        return out.getBytesWritten();
+    }
+
+    @Override
+    public int getEventCount() {
+        return eventCount;
+    }
+
+    @Override
+    public long getAge(final TimeUnit timeUnit) {
+        return timeUnit.convert(System.nanoTime() - creationTime, TimeUnit.NANOSECONDS);
+    }
+
+    @Override
+    public void finishBlock() throws IOException {
+        if ( !blockStarted ) {
+            return;
+        }
+        
+        blockStarted = false;
+        if ( !compressed ) {
+            return;
+        }
+
+        // Calling close() on CompressionOutputStream doesn't close the underlying stream -- it is designed
+        // such that calling close() will write out the Compression footer and become unusable but not
+        // close the underlying stream because the whole point of CompressionOutputStream as opposed to
+        // GZIPOutputStream is that with CompressionOutputStream we can concatenate many together on a single
+        // stream.
+        compressedStream.close();
+    }
+    
+    @Override
+    public void beginNewBlock() throws IOException {
+        if ( blockStarted ) {
+            throw new IllegalStateException("Block is already started");
+        }
+        blockStarted = true;
+        
+        if ( !compressed ) {
+            return;
+        }
+        if ( eventCount == 0 ) {
+            return;
+        }
+        
+        this.compressedStream = new CompressionOutputStream(uncompressedStream);
+        this.out = new ByteCountingOutputStream(compressedStream, uncompressedStream.getBytesWritten());
+    }
+    
+    @Override
+    public String toString() {
+        return "Journal Writer for " + journalFile;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
new file mode 100644
index 0000000..51f84a2
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
@@ -0,0 +1,424 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.partition;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.LuceneIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.LuceneIndexWriter;
+import org.apache.nifi.provenance.journaling.index.QueryUtils;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.JournalWriter;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalWriter;
+import org.apache.nifi.provenance.journaling.tasks.CompressionTask;
+import org.apache.nifi.provenance.journaling.toc.StandardTocWriter;
+import org.apache.nifi.provenance.journaling.toc.TocJournalReader;
+import org.apache.nifi.provenance.journaling.toc.TocWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JournalingPartition implements Partition {
+    private static final Logger logger = LoggerFactory.getLogger(JournalingPartition.class);
+    private static final String JOURNAL_FILE_EXTENSION = ".journal";
+    
+    private final String containerName;
+    private final String sectionName;
+    
+    private final File section;
+    private final File journalsDir;
+    private final JournalingRepositoryConfig config;
+    private final ExecutorService executor;
+    private final LuceneIndexWriter indexWriter;
+    
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+    
+    private JournalWriter journalWriter;
+    private TocWriter tocWriter;
+    private int numEventsAtEndOfLastBlock = 0;
+    private volatile long maxEventId = -1L;
+    private volatile Long earliestEventTime = null;
+    
+    public JournalingPartition(final String containerName, final String sectionName, final File sectionDir, final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
+        this.containerName = containerName;
+        this.sectionName = sectionName;
+        this.section = sectionDir;
+        this.journalsDir = new File(section, "journals");
+        this.config = config;
+        this.executor = executor;
+        
+        if (!journalsDir.exists() && !journalsDir.mkdirs()) {
+            throw new IOException("Could not create directory " + section);
+        }
+        
+        if ( journalsDir.exists() && journalsDir.isFile() ) {
+            throw new IOException("Could not create directory " + section + " because a file already exists with this name");
+        }
+        
+        if ( config.isReadOnly() ) {
+            indexWriter = null;
+        } else {
+            final File indexDir = new File(section, "index");
+            indexWriter = new LuceneIndexWriter(indexDir, config);
+        }
+    }
+    
+    
+    public EventIndexSearcher newIndexSearcher() throws IOException {
+        if (config.isReadOnly()) {
+            return new LuceneIndexSearcher(new File(section, "index"));
+        }
+        
+        return indexWriter.newIndexSearcher();
+    }
+    
+    protected JournalWriter getJournalWriter(final long firstEventId) throws IOException {
+        if ( config.isReadOnly() ) {
+            throw new IllegalStateException("Cannot update repository because it is read-only");
+        }
+        
+        if (isRolloverNecessary()) {
+            rollover(firstEventId);
+        }
+        
+        return journalWriter;
+    }
+    
+    @Override
+    public List<JournaledProvenanceEvent> registerEvents(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
+        writeLock.lock();
+        try {
+            final JournalWriter writer = getJournalWriter(firstEventId);
+    
+            if ( !events.isEmpty() ) {
+                final int eventsWritten = writer.getEventCount();
+                if ( eventsWritten - numEventsAtEndOfLastBlock > config.getBlockSize() ) {
+                    writer.finishBlock();
+                    tocWriter.addBlockOffset(writer.getSize());
+                    numEventsAtEndOfLastBlock = eventsWritten;
+                    writer.beginNewBlock();
+                }
+            }
+    
+            writer.write(events, firstEventId);
+            
+            final List<JournaledProvenanceEvent> storedEvents = new ArrayList<>(events.size());
+            long id = firstEventId;
+            for (final ProvenanceEventRecord event : events) {
+                final JournaledStorageLocation location = new JournaledStorageLocation(containerName, sectionName, 
+                        String.valueOf(writer.getJournalId()), tocWriter.getCurrentBlockIndex(), id++);
+                final JournaledProvenanceEvent storedEvent = new JournaledProvenanceEvent(event, location);
+                storedEvents.add(storedEvent);
+            }
+            
+            indexWriter.index(storedEvents);
+            
+            if ( config.isAlwaysSync() ) {
+                writer.sync();
+            }
+            
+            // update the maxEventId; we don't need a compareAndSet because the AtomicLong is modified
+            // only within a write lock. But we use AtomicLong so that we 
+            if ( id > maxEventId ) {
+                maxEventId = id;
+            }
+            
+            if ( earliestEventTime == null ) {
+                Long earliest = null;
+                for ( final ProvenanceEventRecord event : events ) {
+                    if ( earliest == null || event.getEventTime() < earliest ) {
+                        earliest = event.getEventTime();
+                    }
+                }
+                
+                earliestEventTime = earliest;
+            }
+            
+            return storedEvents;
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    // MUST be called with either the read lock or write lock held.
+    // determines whether or not we need to roll over the journal writer and toc writer.
+    private boolean isRolloverNecessary() {
+        if ( journalWriter == null ) {
+            return true;
+        }
+        
+        final long ageSeconds = journalWriter.getAge(TimeUnit.SECONDS);
+        final long rolloverSeconds = config.getJournalRolloverPeriod(TimeUnit.SECONDS);
+        if ( ageSeconds >= rolloverSeconds ) {
+            return true;
+        }
+        
+        if ( journalWriter.getSize() > config.getJournalCapacity() ) {
+            return true;
+        }
+        
+        return false;
+    }
+    
+    // MUST be called with write lock held.
+    private void rollover(final long firstEventId) throws IOException {
+        // if we have a writer already, close it and initiate rollover actions
+        if ( journalWriter != null ) {
+            journalWriter.finishBlock();
+            journalWriter.close();
+            tocWriter.close();
+            indexWriter.sync();
+        
+            if ( config.isCompressOnRollover() ) {
+                final File finishedFile = journalWriter.getJournalFile();
+                final File finishedTocFile = tocWriter.getFile();
+                executor.submit(new CompressionTask(finishedFile, journalWriter.getJournalId(), finishedTocFile));
+            }
+        }
+        
+        // create new writers and reset state.
+        final File journalFile = new File(journalsDir, firstEventId + JOURNAL_FILE_EXTENSION);
+        journalWriter = new StandardJournalWriter(firstEventId, journalFile, false, new StandardEventSerializer());
+        tocWriter = new StandardTocWriter(QueryUtils.getTocFile(journalFile), false);
+        numEventsAtEndOfLastBlock = 0;
+    }
+    
+
+    private Long getJournalId(final File file) {
+        long journalId;
+        final int dotIndex = file.getName().indexOf(".");
+        if ( dotIndex < 0 ) {
+            journalId = 0L;
+        } else {
+            try {
+                journalId = Long.parseLong(file.getName().substring(0, dotIndex));
+            } catch (final NumberFormatException nfe) {
+                return null;
+            }
+        }
+        
+        return journalId;
+    }
+    
+    @Override
+    public void restore() throws IOException {
+        // delete or rename files if stopped during rollover; compress any files that haven't been compressed
+        if ( !config.isReadOnly() ) {
+            final File[] children = journalsDir.listFiles();
+            if ( children != null ) {
+                // find the latest journal.
+                File latestJournal = null;
+                long latestJournalId = -1L;
+                
+                final List<File> journalFiles = new ArrayList<>();
+                
+                // find any journal files that either haven't been compressed or were partially compressed when
+                // we last shutdown and then restart compression.
+                for ( final File file : children ) {
+                    final String filename = file.getName();
+                    if ( !filename.contains(JOURNAL_FILE_EXTENSION) ) {
+                        continue;
+                    }
+                    
+                    final Long journalId = getJournalId(file);
+                    if ( journalId != null && journalId > latestJournalId ) {
+                        latestJournal = file;
+                        latestJournalId = journalId;
+                    }
+                    
+                    journalFiles.add(file);
+                    
+                    if ( !config.isCompressOnRollover() ) {
+                        continue;
+                    }
+                    
+                    if ( filename.endsWith(CompressionTask.FILE_EXTENSION) ) {
+                        final File uncompressedFile = new File(journalsDir, filename.replace(CompressionTask.FILE_EXTENSION, ""));
+                        if ( uncompressedFile.exists() ) {
+                            // both the compressed and uncompressed version of this journal exist. The Compression Task was
+                            // not complete when we shutdown. Delete the compressed journal and toc and re-start the Compression Task.
+                            final File tocFile = QueryUtils.getTocFile(uncompressedFile);
+                            executor.submit(new CompressionTask(uncompressedFile, getJournalId(uncompressedFile), tocFile));
+                        } else {
+                            // The compressed file exists but the uncompressed file does not. This means that we have finished
+                            // writing the compressed file and deleted the original journal file but then shutdown before
+                            // renaming the compressed file to the original filename. We can simply rename the compressed file
+                            // to the original file and then address the TOC file.
+                            final boolean rename = CompressionTask.rename(file, uncompressedFile);
+                            if ( !rename ) {
+                                logger.warn("{} During recovery, failed to rename {} to {}", this, file, uncompressedFile);
+                                continue;
+                            }
+                            
+                            // Check if the compressed TOC file exists. If not, we are finished.
+                            // If it does exist, then we know that it is complete, as described above, so we will go
+                            // ahead and replace the uncompressed version.
+                            final File tocFile = QueryUtils.getTocFile(uncompressedFile);
+                            final File compressedTocFile = new File(tocFile.getParentFile(), tocFile.getName() + CompressionTask.FILE_EXTENSION);
+                            if ( !compressedTocFile.exists() ) {
+                                continue;
+                            }
+                            
+                            tocFile.delete();
+                            
+                            final boolean renamedTocFile = CompressionTask.rename(compressedTocFile, tocFile);
+                            if ( !renamedTocFile ) {
+                                logger.warn("{} During recovery, failed to rename {} to {}", this, compressedTocFile, tocFile);
+                            }
+                        }
+                    }
+                }
+                
+                // Get the first event in the earliest journal file so that we know what the earliest time available is
+                Collections.sort(journalFiles, new Comparator<File>() {
+                    @Override
+                    public int compare(final File o1, final File o2) {
+                        return Long.compare(getJournalId(o1), getJournalId(o2));
+                    }
+                });
+                
+                for ( final File journal : journalFiles ) {
+                    try (final JournalReader reader = new StandardJournalReader(journal)) {
+                        final ProvenanceEventRecord record = reader.nextEvent();
+                        this.earliestEventTime = record.getEventTime();
+                        break;
+                    } catch (final IOException ioe) {
+                    }
+                }
+                
+                // Whatever was the last journal for this partition, we need to remove anything for that journal
+                // from the index and re-add them, and then sync the index. This allows us to avoid syncing
+                // the index each time (we sync only on rollover) but allows us to still ensure that we index
+                // all events.
+                if ( latestJournal != null ) {
+                    try {
+                        reindex(latestJournal);
+                    } catch (final EOFException eof) {
+                    }
+                }
+            }
+        }
+    }
+
+    
+    private void reindex(final File journalFile) throws IOException {
+        try (final TocJournalReader reader = new TocJournalReader(containerName, sectionName, String.valueOf(getJournalId(journalFile)), journalFile)) {
+            indexWriter.delete(containerName, sectionName, String.valueOf(getJournalId(journalFile)));
+            
+            long maxId = -1L;
+            final List<JournaledProvenanceEvent> storedEvents = new ArrayList<>(1000);
+            JournaledProvenanceEvent event;
+            while ((event = reader.nextJournaledEvent()) != null ) {
+                storedEvents.add(event);
+                maxId = event.getEventId();
+                
+                if ( storedEvents.size() == 1000 ) {
+                    indexWriter.index(storedEvents);
+                    storedEvents.clear();
+                }
+            }
+
+            if ( !storedEvents.isEmpty() ) {
+                indexWriter.index(storedEvents);
+            }
+            
+            indexWriter.sync();
+            this.maxEventId = maxId;
+        }
+    }
+
+    
+    @Override
+    public List<JournaledStorageLocation> getEvents(final long minEventId, final int maxRecords) throws IOException {
+        try (final EventIndexSearcher searcher = indexWriter.newIndexSearcher()) {
+            return searcher.getEvents(minEventId, maxRecords);
+        }
+    }
+    
+    @Override
+    public void shutdown() {
+        if ( journalWriter != null ) {
+            try {
+                journalWriter.finishBlock();
+            } catch (final IOException ioe) {
+                logger.warn("Failed to finish writing Block to {} due to {}", journalWriter, ioe);
+                if ( logger.isDebugEnabled() ) {
+                    logger.warn("", ioe);
+                }
+            }
+            
+            try {
+                journalWriter.close();
+            } catch (final IOException ioe) {
+                logger.warn("Failed to close {} due to {}", journalWriter, ioe);
+                if ( logger.isDebugEnabled() ) {
+                    logger.warn("", ioe);
+                }
+            }
+            
+            try {
+                tocWriter.close();
+            } catch (final IOException ioe) {
+                logger.warn("Failed to close {} due to {}", tocWriter, ioe);
+                if ( logger.isDebugEnabled() ) {
+                    logger.warn("", ioe);
+                }
+            }
+        }
+        
+        if ( indexWriter != null ) {
+            try {
+                indexWriter.close();
+            } catch (final IOException ioe) {
+                logger.warn("Failed to close {} due to {}", indexWriter, ioe);
+                if ( logger.isDebugEnabled() ) {
+                    logger.warn("", ioe);
+                }
+            }
+        }
+    }
+    
+    @Override
+    public long getMaxEventId() {
+        return maxEventId;
+    }
+
+    @Override
+    public Long getEarliestEventTime() throws IOException {
+        return earliestEventTime;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
new file mode 100644
index 0000000..e77c8d5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.partition;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+
+
+/**
+ * Represents a single Partition of the Journaling Provenance Repository. The repository is split into multiple
+ * partitions in order to provide higher throughput.
+ * 
+ * Implementations of this interface MUST be threadsafe.
+ */
+public interface Partition {
+
+    /**
+     * Returns a new EventIndexSearcher that can be used to search the events in this partition
+     * @return
+     * @throws IOException
+     */
+    EventIndexSearcher newIndexSearcher() throws IOException;
+    
+    /**
+     * Registers the given events with this partition. This includes persisting the events and indexing
+     * them so that they are searchable.
+     * @param events
+     * @return
+     */
+    List<JournaledProvenanceEvent> registerEvents(Collection<ProvenanceEventRecord> events, long firstEventId) throws IOException;
+    
+    /**
+     * Restore state after a restart of NiFi
+     */
+    void restore() throws IOException;
+    
+    /**
+     * Shuts down the Partition so that it can no longer be used
+     */
+    void shutdown();
+
+    /**
+     * Returns the largest event ID stored in this partition
+     * @return
+     */
+    long getMaxEventId();
+    
+    /**
+     * Returns the locations of events that have an id at least equal to minEventId, returning the events
+     * with the smallest ID's possible that are greater than minEventId
+     *
+     * @param minEventId
+     * @param maxRecords
+     * @return
+     */
+    List<JournaledStorageLocation> getEvents(long minEventId, int maxRecords) throws IOException;
+    
+    /**
+     * Returns the timestamp of the earliest event in this Partition, or <code>null</code> if the Partition
+     * contains no events
+     * @return
+     * @throws IOException
+     */
+    Long getEarliestEventTime() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionAction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionAction.java
new file mode 100644
index 0000000..8c680f5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionAction.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.partition;
+
+import java.io.IOException;
+
+public interface PartitionAction<T> {
+    T perform(Partition partition) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
new file mode 100644
index 0000000..edbf75b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.partition;
+
+import java.io.IOException;
+import java.util.Set;
+
+
+/**
+ * The PartitionManager is responsible for accessing and maintaining the Partitions so that they are
+ * are written to efficiently and in a thread-safe manner.
+ */
+public interface PartitionManager {
+
+    /**
+     * Performs the given action against one of the partitions
+     * 
+     * @param action the action to perform
+     * @param writeAction specifies whether or not the action writes to the repository
+     * @return
+     * @throws IOException
+     */
+    <T> T withPartition(PartitionAction<T> action, boolean writeAction) throws IOException;
+    
+    /**
+     * Performs the given action against one of the partitions
+     * 
+     * @param action the action to perform
+     * @param writeAction specifies whether or not the action writes to the repository
+     * @throws IOException
+     */
+    void withPartition(VoidPartitionAction action, boolean writeAction) throws IOException;
+    
+    /**
+     * Performs the given Action on each partition and returns the set of results.
+     * 
+     * @param action the action to perform
+     * @param writeAction specifies whether or not the action writes to the repository
+     * @return
+     */
+    <T> Set<T> withEachPartition(PartitionAction<T> action) throws IOException;
+    
+    /**
+     * Performs the given Action to each partition, optionally waiting for the action to complete
+     * @param action
+     * @param writeAction
+     * @param async if <code>true</code>, will perform the action asynchronously; if <code>false</code>, will
+     *  wait for the action to complete before returning
+     */
+    void withEachPartition(VoidPartitionAction action, boolean async);
+    
+    void shutdown();
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
new file mode 100644
index 0000000..4ac0fc6
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.partition;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.util.Tuple;
+
+public class QueuingPartitionManager implements PartitionManager {
+
+    private final JournalingRepositoryConfig config;
+    private final BlockingQueue<Partition> partitionQueue;
+    private final JournalingPartition[] partitionArray;
+    private final ExecutorService executor;
+    private volatile boolean shutdown = false;
+    
+    private final AtomicInteger blacklistedCount = new AtomicInteger(0);
+    
+    public QueuingPartitionManager(final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
+        this.config = config;
+        this.partitionQueue = new LinkedBlockingQueue<>(config.getPartitionCount());
+        this.partitionArray = new JournalingPartition[config.getPartitionCount()];
+        
+        final List<Tuple<String, File>> containerTuples = new ArrayList<>(config.getContainers().size());
+        for ( final Map.Entry<String, File> entry : config.getContainers().entrySet() ) {
+            containerTuples.add(new Tuple<>(entry.getKey(), entry.getValue()));
+        }
+        
+        for (int i=0; i < config.getPartitionCount(); i++) {
+            final Tuple<String, File> tuple = containerTuples.get(i % containerTuples.size());
+            final File section = new File(tuple.getValue(), String.valueOf(i));
+            
+            final JournalingPartition partition = new JournalingPartition(tuple.getKey(), String.valueOf(i), section, config, executor);
+            partitionQueue.offer(partition);
+            partitionArray[i] = partition;
+        }
+        
+        this.executor = executor;
+    }
+    
+    @Override
+    public void shutdown() {
+        this.shutdown = true;
+        
+        for ( final Partition partition : partitionArray ) {
+            partition.shutdown();
+        }
+    }
+    
+    private Partition nextPartition() {
+        Partition partition = null;
+        
+        while(partition == null) {
+            if (shutdown) {
+                throw new RuntimeException("Journaling Provenance Repository is shutting down");
+            }
+            
+            try {
+                partition = partitionQueue.poll(1, TimeUnit.SECONDS);
+            } catch (final InterruptedException ie) {
+            }
+            
+            if ( partition == null ) {
+                if ( blacklistedCount.get() >= config.getPartitionCount() ) {
+                    throw new RuntimeException("Cannot persist to the Journal Provenance Repository because all partitions have been blacklisted due to write failures");
+                }
+            }
+        }
+        
+        return partition;
+    }
+    
+    @Override
+    public <T> T withPartition(final PartitionAction<T> action, final boolean writeAction) throws IOException {
+        final Partition partition = nextPartition();
+
+        boolean ioe = false;
+        try {
+            return action.perform(partition);
+        } catch (final IOException e) {
+            ioe = true;
+            throw e;
+        } finally {
+            if ( ioe && writeAction ) {
+                // We failed to write to this Partition. This partition will no longer be usable until NiFi is restarted!
+                blacklistedCount.incrementAndGet();
+            } else {
+                partitionQueue.offer(partition);
+            }
+        }
+    }
+    
+    @Override
+    public void withPartition(final VoidPartitionAction action, final boolean writeAction) throws IOException {
+        final Partition partition = nextPartition();
+
+        boolean ioe = false;
+        try {
+            action.perform(partition);
+        } catch (final IOException e) {
+            ioe = true;
+            throw e;
+        } finally {
+            if ( ioe && writeAction ) {
+                // We failed to write to this Partition. This partition will no longer be usable until NiFi is restarted!
+                blacklistedCount.incrementAndGet();
+            } else {
+                partitionQueue.offer(partition);
+            }
+        }
+    }
+
+    
+    @Override
+    public <T> Set<T> withEachPartition(final PartitionAction<T> action) throws IOException {
+        final Set<T> results = new HashSet<>(partitionArray.length);
+        
+        // TODO: Do not use blacklisted partitions.
+        final Map<Partition, Future<T>> futures = new HashMap<>(partitionArray.length);
+        for ( final Partition partition : partitionArray ) {
+            final Callable<T> callable = new Callable<T>() {
+                @Override
+                public T call() throws Exception {
+                    return action.perform(partition);
+                }
+            };
+            
+            final Future<T> future = executor.submit(callable);
+            futures.put(partition, future);
+        }
+        
+        for ( final Map.Entry<Partition, Future<T>> entry : futures.entrySet() ) {
+            try {
+                final T result = entry.getValue().get();
+                results.add(result);
+            } catch (final ExecutionException ee) {
+                final Throwable cause = ee.getCause();
+                if ( cause instanceof IOException ) {
+                    throw (IOException) cause;
+                } else {
+                    throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        
+        return results;
+    }
+    
+    @Override
+    public void withEachPartition(final VoidPartitionAction action, final boolean async) {
+        
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/VoidPartitionAction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/VoidPartitionAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/VoidPartitionAction.java
new file mode 100644
index 0000000..beaa187
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/VoidPartitionAction.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.partition;
+
+import java.io.IOException;
+
+public interface VoidPartitionAction {
+    void perform(Partition partition) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
new file mode 100644
index 0000000..c23a405
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.tasks;
+
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.JournalWriter;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalWriter;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
+import org.apache.nifi.provenance.journaling.toc.StandardTocWriter;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
+import org.apache.nifi.provenance.journaling.toc.TocWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompressionTask implements Runnable {
+    public static final String FILE_EXTENSION = ".compress";
+    
+    private static final Logger logger = LoggerFactory.getLogger(CompressionTask.class);
+    
+    private final File journalFile;
+    private final long journalId;
+    private final File tocFile;
+    
+    public CompressionTask(final File journalFile, final long journalId, final File tocFile) {
+        this.journalFile = journalFile;
+        this.journalId = journalId;
+        this.tocFile = tocFile;
+    }
+
+    public void compress(final JournalReader reader, final JournalWriter writer, final TocReader tocReader, final TocWriter tocWriter) throws IOException {
+        ProvenanceEventRecord event;
+        
+        int blockIndex = 0;
+        long blockOffset = tocReader.getBlockOffset(blockIndex);
+        tocWriter.addBlockOffset(blockOffset);
+        long nextBlockOffset = tocReader.getBlockOffset(blockIndex + 1);
+        
+        try {
+            while ((event = reader.nextEvent()) != null) {
+                // Check if we've gone beyond the offset of the next block. If so, write
+                // out a new block in the TOC.
+                final long newPosition = reader.getPosition();
+                if ( newPosition > nextBlockOffset && nextBlockOffset > 0 ) {
+                    blockIndex++;
+                    blockOffset = tocReader.getBlockOffset(blockIndex);
+                    tocWriter.addBlockOffset(writer.getSize());
+                    
+                    nextBlockOffset = tocReader.getBlockOffset(blockIndex + 1);
+                }
+                
+                // Write the event to the compressed writer
+                writer.write(Collections.singleton(event), event.getEventId());
+            }
+        } catch (final EOFException eof) {
+            logger.warn("Found unexpected End-of-File when compressing {}", reader);
+        }
+    }
+
+    /**
+     * Attempts to delete the given file up to 10 times, waiting a bit in between each failed
+     * iteration, in case another process (for example, a virus scanner) has the file locked
+     * 
+     * @param file
+     * @return
+     */
+    private boolean delete(final File file) {
+        for (int i=0; i < 10; i++) {
+            if ( file.delete() || !file.exists() ) {
+                return true;
+            }
+            
+            try {
+                Thread.sleep(100L);
+            } catch (final InterruptedException ie) {
+            }
+        }
+        
+        return false;
+    }
+    
+    /**
+     * Attempts to rename the given original file to the renamed file up to 20 times, waiting a bit
+     * in between each failed iteration, in case another process (for example, a virus scanner) has 
+     * the file locked
+     * 
+     * @param original
+     * @param renamed
+     * @return
+     */
+    public static boolean rename(final File original, final File renamed) {
+        for (int i=0; i < 20; i++) {
+            if ( original.renameTo(renamed) ) {
+                return true;
+            }
+            
+            try {
+                Thread.sleep(100L);
+            } catch (final InterruptedException ie) {
+            }
+        }
+        
+        return false;
+    }
+    
+    @Override
+    public void run() {
+        try {
+            final File compressedFile = new File(journalFile.getParentFile(), journalFile.getName() + FILE_EXTENSION);
+            final File compressedTocFile = new File(tocFile.getParentFile(), tocFile.getName() + FILE_EXTENSION);
+
+            try (final JournalReader journalReader = new StandardJournalReader(journalFile);
+                final JournalWriter compressedWriter = new StandardJournalWriter(journalId, compressedFile, true, new StandardEventSerializer());
+                final TocReader tocReader = new StandardTocReader(tocFile);
+                final TocWriter compressedTocWriter = new StandardTocWriter(compressedTocFile, true)) {
+                
+                compress(journalReader, compressedWriter, tocReader, compressedTocWriter);
+                compressedWriter.sync();
+            }
+
+            final boolean deletedJournal = delete(journalFile);
+            if ( !deletedJournal ) {
+                delete(compressedFile);
+                delete(compressedTocFile);
+                logger.error("Failed to remove Journal file {}; considering compression task a failure", journalFile);
+                return;
+            }
+            
+            final boolean deletedToc = delete(tocFile);
+            if ( !deletedToc ) {
+                delete(compressedFile);
+                delete(compressedTocFile);
+                logger.error("Failed to remove TOC file for {}; considering compression task a failure", journalFile);
+                return;
+            }
+            
+            final boolean renamedJournal = rename(compressedFile, journalFile);
+            if ( !renamedJournal ) {
+                logger.error("Failed to rename {} to {}; this journal file may be inaccessible until it is renamed", compressedFile, journalFile);
+            }
+            
+            final boolean renamedToc = rename(compressedTocFile, tocFile);
+            if ( !renamedToc ) {
+                logger.error("Failed to rename {} to {}; this journal file may be inaccessible until it is renamed", compressedTocFile, tocFile);
+            }
+            
+            logger.info("Successfully compressed Journal File {}");
+        } catch (final IOException ioe) {
+            logger.error("Failed to compress Journal File {} due to {}", journalFile, ioe.toString());
+            if ( logger.isDebugEnabled() ) {
+                logger.error("", ioe);
+            }
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
new file mode 100644
index 0000000..995acf9
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.toc;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+/**
+ * Standard implementation of TocReader.
+ * 
+ * Expects .toc file to be in the following format;
+ * 
+ * byte 0: version
+ * byte 1: boolean: compressionFlag -> 0 = journal is NOT compressed, 1 = journal is compressed
+ * byte 2-9: long: offset of block 0
+ * byte 10-17: long: offset of block 1
+ * ...
+ * byte (N*8+2)-(N*8+9): long: offset of block N
+ */
+public class StandardTocReader implements TocReader {
+    private final boolean compressed;
+    private final long[] offsets;
+    
+    public StandardTocReader(final File file) throws IOException {
+        try (final FileInputStream fis = new FileInputStream(file);
+             final DataInputStream dis = new DataInputStream(fis)) {
+            
+            final int version = dis.read();
+            if ( version < 0 ) {
+                throw new EOFException();
+            }
+            
+            final int compressionFlag = dis.read();
+            if ( compressionFlag < 0 ) {
+                throw new EOFException();
+            }
+            
+            if ( compressionFlag == 0 ) {
+                compressed = false;
+            } else if ( compressionFlag == 1 ) {
+                compressed = true;
+            } else {
+                throw new IOException("Table of Contents appears to be corrupt: could not read 'compression flag' from header; expected value of 0 or 1 but got " + compressionFlag);
+            }
+            
+            final int numBlocks = (int) ((file.length() - 2) / 8);
+            offsets = new long[numBlocks];
+            
+            for (int i=0; i < numBlocks; i++) {
+                offsets[i] = dis.readLong();
+            }
+        }
+    }
+    
+    @Override
+    public boolean isCompressed() {
+        return compressed;
+    }
+    
+    @Override
+    public long getBlockOffset(final int blockIndex) {
+        if ( blockIndex >= offsets.length ) {
+            return -1L;
+        }
+        return offsets[blockIndex];
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
new file mode 100644
index 0000000..6058282
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.toc;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.FileAlreadyExistsException;
+
+/**
+ * Standard implementation of {@link TocWriter}.
+ * 
+ * Format of .toc file:
+ * byte 0: version
+ * byte 1: compressed: 0 -> not compressed, 1 -> compressed
+ * byte 2-9: long: offset of block 0
+ * byte 10-17: long: offset of block 1
+ * ...
+ * byte (N*8+2)-(N*8+9): long: offset of block N
+ */
+public class StandardTocWriter implements TocWriter {
+    public static final byte VERSION = 1;
+    
+    private final File file;
+    private final FileOutputStream fos;
+    private int index = 0;
+    
+    /**
+     * Creates a StandardTocWriter that writes to the given file.
+     * @param file the file to write to
+     * @param compressionFlag whether or not the journal is compressed
+     * @throws FileNotFoundException 
+     */
+    public StandardTocWriter(final File file, final boolean compressionFlag) throws IOException {
+        if ( file.exists() ) {
+            throw new FileAlreadyExistsException(file.getAbsolutePath());
+        }
+        
+        if ( !file.getParentFile().exists() && !file.getParentFile().mkdirs() ) {
+            throw new IOException("Could not create directory " + file.getParent());
+        }
+        
+        this.file = file;
+        fos = new FileOutputStream(file);
+        
+        fos.write(VERSION);
+        fos.write(compressionFlag ? 1 : 0);
+        fos.flush();
+        fos.getFD().sync();
+    }
+    
+    @Override
+    public void addBlockOffset(final long offset) throws IOException {
+        final BufferedOutputStream bos = new BufferedOutputStream(fos);
+        final DataOutputStream dos = new DataOutputStream(bos);
+        dos.writeLong(offset);
+        dos.flush();
+        
+        fos.getFD().sync();
+    }
+    
+    @Override
+    public int getCurrentBlockIndex() {
+        return index;
+    }
+
+    @Override
+    public void close() throws IOException {
+        fos.close();
+    }
+    
+    @Override
+    public File getFile() {
+        return file;
+    }
+    
+    @Override
+    public String toString() {
+        return "TOC Writer for " + file;
+    }
+}


[03/12] incubator-nifi git commit: implemented ability to persist and recover records

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
new file mode 100644
index 0000000..2130e73
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.provenance.ProvenanceEventBuilder;
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.SearchableFieldParser;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.QueryUtils;
+import org.apache.nifi.provenance.journaling.journals.JournalReader;
+import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
+import org.apache.nifi.provenance.journaling.partition.Partition;
+import org.apache.nifi.provenance.journaling.partition.PartitionAction;
+import org.apache.nifi.provenance.journaling.partition.PartitionManager;
+import org.apache.nifi.provenance.journaling.partition.QueuingPartitionManager;
+import org.apache.nifi.provenance.journaling.partition.VoidPartitionAction;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
+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.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JournalingProvenanceRepository implements ProvenanceEventRepository {
+    private static final Logger logger = LoggerFactory.getLogger(JournalingProvenanceRepository.class);
+    
+    private final JournalingRepositoryConfig config;
+    private final PartitionManager partitionManager;
+    private final AtomicLong idGenerator = new AtomicLong(0L);
+    
+    private EventReporter eventReporter;    // effectively final
+    private final ExecutorService executor;
+    
+    
+    public JournalingProvenanceRepository() throws IOException {
+        this(createConfig());
+    }
+    
+    public JournalingProvenanceRepository(final JournalingRepositoryConfig config) throws IOException {
+        this.config = config;
+        this.executor = Executors.newFixedThreadPool(config.getThreadPoolSize());
+        this.partitionManager = new QueuingPartitionManager(config, executor);
+    }
+    
+    
+    private static JournalingRepositoryConfig createConfig()  {
+        final NiFiProperties properties = NiFiProperties.getInstance();
+        final Map<String, Path> storageDirectories = properties.getProvenanceRepositoryPaths();
+        if (storageDirectories.isEmpty()) {
+            storageDirectories.put("provenance_repository", Paths.get("provenance_repository"));
+        }
+        final String storageTime = properties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_TIME, "24 hours");
+        final String storageSize = properties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_SIZE, "1 GB");
+        final String rolloverTime = properties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_TIME, "5 mins");
+        final String rolloverSize = properties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_SIZE, "100 MB");
+        final String shardSize = properties.getProperty(NiFiProperties.PROVENANCE_INDEX_SHARD_SIZE, "500 MB");
+        final int queryThreads = properties.getIntegerProperty(NiFiProperties.PROVENANCE_QUERY_THREAD_POOL_SIZE, 2);
+        final int journalCount = properties.getIntegerProperty(NiFiProperties.PROVENANCE_JOURNAL_COUNT, 16);
+
+        final long storageMillis = FormatUtils.getTimeDuration(storageTime, TimeUnit.MILLISECONDS);
+        final long maxStorageBytes = DataUnit.parseDataSize(storageSize, DataUnit.B).longValue();
+        final long rolloverMillis = FormatUtils.getTimeDuration(rolloverTime, TimeUnit.MILLISECONDS);
+        final long rolloverBytes = DataUnit.parseDataSize(rolloverSize, DataUnit.B).longValue();
+
+        final boolean compressOnRollover = Boolean.parseBoolean(properties.getProperty(NiFiProperties.PROVENANCE_COMPRESS_ON_ROLLOVER));
+        final String indexedFieldString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_FIELDS);
+        final String indexedAttrString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_ATTRIBUTES);
+
+        final Boolean alwaysSync = Boolean.parseBoolean(properties.getProperty("nifi.provenance.repository.always.sync", "false"));
+
+        final List<SearchableField> searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true);
+        final List<SearchableField> searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false);
+
+        // We always want to index the Event Time.
+        if (!searchableFields.contains(SearchableFields.EventTime)) {
+            searchableFields.add(SearchableFields.EventTime);
+        }
+
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        
+        final Map<String, File> containers = new HashMap<>(storageDirectories.size());
+        for ( final Map.Entry<String, Path> entry : storageDirectories.entrySet() ) {
+            containers.put(entry.getKey(), entry.getValue().toFile());
+        }
+        config.setContainers(containers);
+        config.setCompressOnRollover(compressOnRollover);
+        config.setSearchableFields(searchableFields);
+        config.setSearchableAttributes(searchableAttributes);
+        config.setJournalCapacity(rolloverBytes);
+        config.setJournalRolloverPeriod(rolloverMillis, TimeUnit.MILLISECONDS);
+        config.setEventExpiration(storageMillis, TimeUnit.MILLISECONDS);
+        config.setMaxStorageCapacity(maxStorageBytes);
+        config.setThreadPoolSize(queryThreads);
+        config.setPartitionCount(journalCount);
+
+        if (shardSize != null) {
+            config.setDesiredIndexSize(DataUnit.parseDataSize(shardSize, DataUnit.B).longValue());
+        }
+
+        config.setAlwaysSync(alwaysSync);
+
+        return config;
+    }
+    
+    @Override
+    public synchronized void initialize(final EventReporter eventReporter) throws IOException {
+        this.eventReporter = eventReporter;
+    }
+
+    @Override
+    public ProvenanceEventBuilder eventBuilder() {
+        return new StandardProvenanceEventRecord.Builder();
+    }
+
+    @Override
+    public void registerEvent(final ProvenanceEventRecord event) throws IOException {
+        registerEvents(Collections.singleton(event));
+    }
+
+    @Override
+    public void registerEvents(final Collection<ProvenanceEventRecord> events) throws IOException {
+        partitionManager.withPartition(new VoidPartitionAction() {
+            @Override
+            public void perform(final Partition partition) throws IOException {
+                partition.registerEvents(events, idGenerator.getAndAdd(events.size()));
+            }
+        }, true);
+    }
+
+    @Override
+    public StoredProvenanceEvent getEvent(final long id) throws IOException {
+        final List<StoredProvenanceEvent> events = getEvents(id, 1);
+        if ( events.isEmpty() ) {
+            return null;
+        }
+
+        // We have to check the id of the event returned, because we are requesting up to 1 record
+        // starting with the given id. However, if that ID doesn't exist, we could get a record
+        // with a larger id.
+        final StoredProvenanceEvent event = events.get(0);
+        if ( event.getEventId() == id ) {
+            return event;
+        }
+        
+        return null;
+    }
+    
+    @Override
+    public List<StoredProvenanceEvent> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
+        // Must generate query to determine the appropriate StorageLocation objects and then call
+        // getEvent(List<StorageLocation>)
+        final Set<List<JournaledStorageLocation>> resultSet = partitionManager.withEachPartition(
+            new PartitionAction<List<JournaledStorageLocation>>() {
+                @Override
+                public List<JournaledStorageLocation> perform(final Partition partition) throws IOException {
+                    return partition.getEvents(firstRecordId, maxRecords);
+                }
+        });
+        
+        final ArrayList<JournaledStorageLocation> locations = new ArrayList<>(maxRecords);
+        for ( final List<JournaledStorageLocation> list : resultSet ) {
+            for ( final JournaledStorageLocation location : list ) {
+                locations.add(location);
+            }
+        }
+        
+        Collections.sort(locations, new Comparator<JournaledStorageLocation>() {
+            @Override
+            public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+                return Long.compare(o1.getEventId(), o2.getEventId());
+            }
+        });
+        
+        locations.trimToSize();
+        
+        @SuppressWarnings({ "rawtypes", "unchecked" })
+        final List<StorageLocation> storageLocations = (List<StorageLocation>) ((List) locations);
+        return getEvents(storageLocations);
+    }
+    
+    @Override
+    public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+        final List<StoredProvenanceEvent> storedEvents = getEvents(Collections.singletonList(location));
+        return (storedEvents == null || storedEvents.isEmpty()) ? null : storedEvents.get(0);
+    }
+    
+    
+    
+    @Override
+    public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> locations) throws IOException {
+        // Group the locations by journal files because we want a single thread, at most, per journal file.
+        final Map<File, List<JournaledStorageLocation>> orderedLocations = QueryUtils.orderLocations(locations, config);
+        
+        // Go through each journal file and create a callable that can lookup the records for that journal file.
+        final List<Future<List<StoredProvenanceEvent>>> futures = new ArrayList<>();
+        for ( final Map.Entry<File, List<JournaledStorageLocation>> entry : orderedLocations.entrySet() ) {
+            final File journalFile = entry.getKey();
+            final List<JournaledStorageLocation> locationsForFile = entry.getValue();
+            
+            final Callable<List<StoredProvenanceEvent>> callable = new Callable<List<StoredProvenanceEvent>>() {
+                @Override
+                public List<StoredProvenanceEvent> call() throws Exception {
+                    try(final TocReader tocReader = new StandardTocReader(new File(journalFile.getParentFile(), journalFile.getName() + ".toc"));
+                        final JournalReader reader = new StandardJournalReader(journalFile)) 
+                    {
+                        final List<StoredProvenanceEvent> storedEvents = new ArrayList<>(locationsForFile.size());
+                        
+                        for ( final JournaledStorageLocation location : locationsForFile ) {
+                            final long blockOffset = tocReader.getBlockOffset(location.getBlockIndex());
+                            final ProvenanceEventRecord event = reader.getEvent(blockOffset, location.getEventId());
+                            
+                            storedEvents.add(new JournaledProvenanceEvent(event, location));
+                        }
+                        
+                        return storedEvents;
+                    }
+                }
+            };
+            
+            final Future<List<StoredProvenanceEvent>> future = executor.submit(callable);
+            futures.add(future);
+        }
+        
+        // Get all of the events from the futures, waiting for them to finish.
+        final Map<StorageLocation, StoredProvenanceEvent> locationToEventMap = new HashMap<>(locations.size());
+        for ( final Future<List<StoredProvenanceEvent>> future : futures ) {
+            try {
+                final List<StoredProvenanceEvent> events = future.get();
+                
+                // Map the location to the event, so that we can then re-order the events in the same order
+                // that the locations were passed to us.
+                for ( final StoredProvenanceEvent event : events ) {
+                    locationToEventMap.put(event.getStorageLocation(), event);
+                }
+            } catch (final ExecutionException ee) {
+                final Throwable cause = ee.getCause();
+                if ( cause instanceof IOException ) {
+                    throw (IOException) cause;
+                } else {
+                    throw new RuntimeException(cause);
+                }
+            } catch (final InterruptedException ie) {
+                throw new RuntimeException(ie);
+            }
+        }
+        
+        // Sort Events by the order of the provided locations.
+        final List<StoredProvenanceEvent> sortedEvents = new ArrayList<>(locations.size());
+        for ( final StorageLocation location : locations ) {
+            final StoredProvenanceEvent event = locationToEventMap.get(location);
+            if ( event != null ) {
+                sortedEvents.add(event);
+            }
+        }
+        
+        return sortedEvents;
+    }
+    
+
+    @Override
+    public Long getMaxEventId() throws IOException {
+        final Set<Long> maxIds = partitionManager.withEachPartition(new PartitionAction<Long>() {
+            @Override
+            public Long perform(final Partition partition) throws IOException {
+                return partition.getMaxEventId();
+            }
+        });
+        
+        Long maxId = null;
+        for ( final Long id : maxIds ) {
+            if ( id == null ) {
+                continue;
+            }
+            
+            if ( maxId == null || id > maxId ) {
+                maxId = id;
+            }
+        }
+        
+        return maxId;
+    }
+
+    
+    @Override
+    public QuerySubmission submitQuery(final Query query) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ComputeLineageSubmission retrieveLineageSubmission(final String lineageIdentifier) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ComputeLineageSubmission submitExpandParents(final long eventId) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public ComputeLineageSubmission submitExpandChildren(final long eventId) {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public void close() throws IOException {
+        partitionManager.shutdown();
+        executor.shutdown();
+    }
+
+    @Override
+    public List<SearchableField> getSearchableFields() {
+        return config.getSearchableFields();
+    }
+
+    @Override
+    public List<SearchableField> getSearchableAttributes() {
+        return config.getSearchableAttributes();
+    }
+
+    @Override
+    public Long getEarliestEventTime() throws IOException {
+        // Get the earliest event timestamp for each partition
+        final Set<Long> earliestTimes = partitionManager.withEachPartition(new PartitionAction<Long>() {
+            @Override
+            public Long perform(final Partition partition) throws IOException {
+                return partition.getEarliestEventTime();
+            }
+        });
+        
+        // Find the latest timestamp for each of the "earliest" timestamps.
+        // This is a bit odd, but we're doing it for a good reason:
+        //      The UI is going to show the earliest time available. Because we have a partitioned write-ahead
+        //      log, if we just return the timestamp of the earliest event available, we could end up returning
+        //      a time for an event that exists but the next event in its lineage does not exist because it was
+        //      already aged off of a different journal. To avoid this, we return the "latest of the earliest"
+        //      timestamps. This way, we know that no event with a larger ID has been aged off from any of the
+        //      partitions.
+        Long latest = null;
+        for ( final Long earliestTime : earliestTimes ) {
+            if ( earliestTime == null ) {
+                continue;
+            }
+            
+            if ( latest == null || earliestTime > latest ) {
+                latest = earliestTime;
+            }
+        }
+        
+        return latest;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
new file mode 100644
index 0000000..6dd7be9
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.config;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.nifi.provenance.search.SearchableField;
+
+public class JournalingRepositoryConfig {
+    private Map<String, File> containers = new HashMap<>();
+    private long expirationMillis = TimeUnit.MILLISECONDS.convert(24, TimeUnit.HOURS);
+    private long storageCapacity = 1024L * 1024L * 1024L;   // 1 GB
+    private long rolloverMillis = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES);
+    private long journalCapacity = 1024L * 1024L * 5L;   // 5 MB
+    private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB
+    private int partitionCount = 16;
+    private int blockSize = 100;
+
+    private List<SearchableField> searchableFields = new ArrayList<>();
+    private List<SearchableField> searchableAttributes = new ArrayList<>();
+    private boolean compress = true;
+    private boolean alwaysSync = false;
+    private int threadPoolSize = 4;
+    private boolean readOnly = false;
+
+    public void setReadOnly(final boolean readOnly) {
+        this.readOnly = readOnly;
+    }
+
+    public boolean isReadOnly() {
+        return readOnly;
+    }
+
+    /**
+     * Specifies where the repository will store data
+     *
+     * @return
+     */
+    public Map<String, File> getContainers() {
+        return Collections.unmodifiableMap(containers);
+    }
+
+    /**
+     * Specifies where the repository should store data
+     *
+     * @param storageDirectory
+     */
+    public void setContainers(final Map<String, File> containers) {
+        this.containers = new HashMap<>(containers);
+    }
+
+    /**
+     * Returns the maximum amount of time that a given record will stay in the
+     * repository
+     *
+     * @param timeUnit
+     * @return
+     */
+    public long getEventExpiration(final TimeUnit timeUnit) {
+        return timeUnit.convert(expirationMillis, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * Specifies how long a record should stay in the repository
+     *
+     * @param expiration
+     * @param timeUnit
+     */
+    public void setEventExpiration(final long expiration, final TimeUnit timeUnit) {
+        this.expirationMillis = TimeUnit.MILLISECONDS.convert(expiration, timeUnit);
+    }
+
+    /**
+     * Returns the maximum amount of data to store in the repository (in bytes)
+     *
+     * @return
+     */
+    public long getMaxStorageCapacity() {
+        return storageCapacity;
+    }
+
+    /**
+     * Sets the maximum amount of data to store in the repository (in bytes)
+     * @param maxStorageCapacity
+     */
+    public void setMaxStorageCapacity(final long maxStorageCapacity) {
+        this.storageCapacity = maxStorageCapacity;
+    }
+
+    /**
+     * Returns the maximum amount of time to write to a single event file
+     *
+     * @param timeUnit
+     * @return
+     */
+    public long getJournalRolloverPeriod(final TimeUnit timeUnit) {
+        return timeUnit.convert(rolloverMillis, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * Sets the maximum amount of time to write to a single event file
+     *
+     * @param rolloverPeriod
+     * @param timeUnit
+     */
+    public void setJournalRolloverPeriod(final long rolloverPeriod, final TimeUnit timeUnit) {
+        this.rolloverMillis = TimeUnit.MILLISECONDS.convert(rolloverPeriod, timeUnit);
+    }
+
+    /**
+     * Returns the number of bytes (pre-compression) that will be
+     * written to a single journal file before the file is rolled over
+     *
+     * @return
+     */
+    public long getJournalCapacity() {
+        return journalCapacity;
+    }
+
+    /**
+     * Sets the number of bytes (pre-compression) that will be written
+     * to a single journal file before the file is rolled over
+     *
+     * @param journalCapacity
+     */
+    public void setJournalCapacity(final long journalCapacity) {
+        this.journalCapacity = journalCapacity;
+    }
+
+    /**
+     * Returns the fields that can be indexed
+     *
+     * @return
+     */
+    public List<SearchableField> getSearchableFields() {
+        return Collections.unmodifiableList(searchableFields);
+    }
+
+    /**
+     * Sets the fields to index
+     *
+     * @param searchableFields
+     */
+    public void setSearchableFields(final List<SearchableField> searchableFields) {
+        this.searchableFields = new ArrayList<>(searchableFields);
+    }
+
+    /**
+     * Returns the FlowFile attributes that can be indexed
+     *
+     * @return
+     */
+    public List<SearchableField> getSearchableAttributes() {
+        return Collections.unmodifiableList(searchableAttributes);
+    }
+
+    /**
+     * Sets the FlowFile attributes to index
+     *
+     * @param searchableAttributes
+     */
+    public void setSearchableAttributes(final List<SearchableField> searchableAttributes) {
+        this.searchableAttributes = new ArrayList<>(searchableAttributes);
+    }
+
+    /**
+     * Indicates whether or not event files will be compressed when they are
+     * rolled over
+     *
+     * @return
+     */
+    public boolean isCompressOnRollover() {
+        return compress;
+    }
+
+    /**
+     * Specifies whether or not to compress event files on rollover
+     *
+     * @param compress
+     */
+    public void setCompressOnRollover(final boolean compress) {
+        this.compress = compress;
+    }
+
+    public int getThreadPoolSize() {
+        return threadPoolSize;
+    }
+
+    public void setThreadPoolSize(final int queryThreadPoolSize) {
+        if (queryThreadPoolSize < 1) {
+            throw new IllegalArgumentException();
+        }
+        this.threadPoolSize = queryThreadPoolSize;
+    }
+
+    /**
+     * <p>
+     * Specifies the desired size of each Provenance Event index shard, in
+     * bytes. We shard the index for a few reasons:
+     * </p>
+     *
+     * <ol>
+     * <li>
+     * A very large index requires a significant amount of Java heap space to
+     * search. As the size of the shard increases, the required Java heap space
+     * also increases.
+     * </li>
+     * <li>
+     * By having multiple shards, we have the ability to use multiple concurrent
+     * threads to search the individual shards, resulting in far less latency
+     * when performing a search across millions or billions of records.
+     * </li>
+     * <li>
+     * We keep track of which time ranges each index shard spans. As a result,
+     * we are able to determine which shards need to be searched if a search
+     * provides a date range. This can greatly increase the speed of a search
+     * and reduce resource utilization.
+     * </li>
+     * </ol>
+     *
+     * @param bytes
+     */
+    public void setDesiredIndexSize(final long bytes) {
+        this.desiredIndexBytes = bytes;
+    }
+
+    /**
+     * Returns the desired size of each index shard. See the
+     * {@Link #setDesiredIndexSize} method for an explanation of why we choose
+     * to shard the index.
+     *
+     * @return
+     */
+    public long getDesiredIndexSize() {
+        return desiredIndexBytes;
+    }
+
+    /**
+     * Sets the number of Journal files to use when persisting records.
+     *
+     * @param numJournals
+     */
+    public void setPartitionCount(final int numJournals) {
+        if (numJournals < 1) {
+            throw new IllegalArgumentException();
+        }
+
+        this.partitionCount = numJournals;
+    }
+
+    /**
+     * Returns the number of Journal files that will be used when persisting
+     * records.
+     *
+     * @return
+     */
+    public int getPartitionCount() {
+        return partitionCount;
+    }
+
+    /**
+     * Specifies whether or not the Repository should sync all updates to disk.
+     *
+     * @return
+     */
+    public boolean isAlwaysSync() {
+        return alwaysSync;
+    }
+
+    /**
+     * Configures whether or not the Repository should sync all updates to disk.
+     * Setting this value to true means that updates are guaranteed to be
+     * persisted across restarted, even if there is a power failure or a sudden
+     * Operating System crash, but it can be very expensive.
+     *
+     * @param alwaysSync
+     */
+    public void setAlwaysSync(boolean alwaysSync) {
+        this.alwaysSync = alwaysSync;
+    }
+
+    /**
+     * Returns the minimum number of Provenance Events that should be written to a single Block.
+     * Events are written out in blocks, which are later optionally compressed. A larger block size
+     * will potentially result in better compression. However, a smaller block size will result
+     * in better performance when reading the data. The default value is 100 events per block.
+     * 
+     * @return
+     */
+    public int getBlockSize() {
+        return blockSize;
+    }
+    
+    /**
+     * Sets the minimum number of Provenance Events that should be written to a single Block.
+     * Events are written out in blocks, which are later optionally compressed. A larger block size
+     * will potentially result in better compression. However, a smaller block size will result
+     * in better performance when reading the data. The default value is 100 events per block.
+     * 
+     * @return
+     */
+    public void setBlockSize(final int blockSize) {
+        if ( blockSize < 1 ) {
+            throw new IllegalArgumentException("Cannot set block size to " + blockSize + "; must be a positive integer");
+        }
+        this.blockSize = blockSize;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
new file mode 100644
index 0000000..b669c53
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.search.Query;
+
+public interface EventIndexSearcher extends Closeable {
+    /**
+     * Searches the repository for any events that match the provided query and returns the locations
+     * where those events are stored
+     * @param query
+     * @return
+     */
+    SearchResult search(Query query) throws IOException;
+    
+    List<JournaledStorageLocation> getEvents(long minEventId, int maxResults) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
new file mode 100644
index 0000000..1f231e9
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+
+public interface EventIndexWriter extends Closeable {
+
+    /**
+     * Adds all of the events to the index.
+     * @param events
+     * @throws IOException
+     */
+    void index(final Collection<JournaledProvenanceEvent> events) throws IOException;
+    
+    /**
+     * Forces all updates to the index to be pushed to disk.
+     * @throws IOException
+     */
+    void sync() throws IOException;
+    
+    /**
+     * Deletes any records that belong to the given container/section/journal
+     * @param containerName
+     * @param section
+     * @param journalId
+     * @throws IOException
+     */
+    void delete(String containerName, String section, String journalId) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexedFieldNames.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexedFieldNames.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexedFieldNames.java
new file mode 100644
index 0000000..977df9f
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexedFieldNames.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+public class IndexedFieldNames {
+
+    public static final String CONTAINER_NAME = "containerName";
+    public static final String SECTION_NAME = "sectionName";
+    public static final String JOURNAL_ID = "journalId";
+    public static final String BLOCK_INDEX = "blockIndex";
+    public static final String EVENT_ID = "eventId";
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/JournalingSearchableFields.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/JournalingSearchableFields.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/JournalingSearchableFields.java
new file mode 100644
index 0000000..9ec9f5d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/JournalingSearchableFields.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import org.apache.nifi.provenance.NamedSearchableField;
+import org.apache.nifi.provenance.search.SearchableField;
+
+public class JournalingSearchableFields {
+    public static SearchableField CONTAINER_NAME = new NamedSearchableField(IndexedFieldNames.CONTAINER_NAME, IndexedFieldNames.CONTAINER_NAME, "Container Name", false);
+    public static SearchableField SECTION_NAME = new NamedSearchableField(IndexedFieldNames.SECTION_NAME, IndexedFieldNames.SECTION_NAME, "Section Name", false);
+    public static SearchableField JOURNAL_ID = new NamedSearchableField(IndexedFieldNames.JOURNAL_ID, IndexedFieldNames.JOURNAL_ID, "Journal ID", false);
+    public static SearchableField BLOCK_INDEX = new NamedSearchableField(IndexedFieldNames.BLOCK_INDEX, IndexedFieldNames.BLOCK_INDEX, "Block Index", false);
+    public static SearchableField EVENT_ID = new NamedSearchableField(IndexedFieldNames.EVENT_ID, IndexedFieldNames.EVENT_ID, "Event ID", false);
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
new file mode 100644
index 0000000..32dc7c3
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortField.Type;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.search.Query;
+
+public class LuceneIndexSearcher implements EventIndexSearcher {
+    private final DirectoryReader reader;
+    private final IndexSearcher searcher;
+    private final FSDirectory fsDirectory;
+    
+    public LuceneIndexSearcher(final File indexDirectory) throws IOException {
+        this.fsDirectory = FSDirectory.open(indexDirectory);
+        this.reader = DirectoryReader.open(fsDirectory);
+        this.searcher = new IndexSearcher(reader);
+    }
+    
+    public LuceneIndexSearcher(final DirectoryReader reader) {
+        this.reader = reader;
+        this.searcher = new IndexSearcher(reader);
+        this.fsDirectory = null;
+    }
+    
+    @Override
+    public void close() throws IOException {
+        IOException suppressed = null;
+        try {
+            reader.close();
+        } catch (final IOException ioe) {
+            suppressed = ioe;
+        }
+        
+        if ( fsDirectory != null ) {
+            fsDirectory.close();
+        }
+        
+        if ( suppressed != null ) {
+            throw suppressed;
+        }
+    }
+    
+    private JournaledStorageLocation createLocation(final Document document) {
+        final String containerName = document.get(IndexedFieldNames.CONTAINER_NAME);
+        final String sectionName = document.get(IndexedFieldNames.SECTION_NAME);
+        final String journalId = document.get(IndexedFieldNames.JOURNAL_ID);
+        final int blockIndex = document.getField(IndexedFieldNames.BLOCK_INDEX).numericValue().intValue();
+        final long eventId = document.getField(IndexedFieldNames.EVENT_ID).numericValue().longValue();
+        
+        return new JournaledStorageLocation(containerName, sectionName, journalId, blockIndex, eventId);
+    }
+    
+    private List<JournaledStorageLocation> getLocations(final TopDocs topDocs) throws IOException {
+        final ScoreDoc[] scoreDocs = topDocs.scoreDocs;
+        final List<JournaledStorageLocation> locations = new ArrayList<>(scoreDocs.length);
+        for ( final ScoreDoc scoreDoc : scoreDocs ) {
+            final Document document = reader.document(scoreDoc.doc);
+            locations.add(createLocation(document));
+        }
+        
+        return locations;
+    }
+    
+    @Override
+    public SearchResult search(final Query provenanceQuery) throws IOException {
+        final org.apache.lucene.search.Query luceneQuery = QueryUtils.convertQueryToLucene(provenanceQuery);
+        final TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults());
+        final List<JournaledStorageLocation> locations = getLocations(topDocs);
+        
+        return new SearchResult(locations, topDocs.totalHits);
+    }
+
+    @Override
+    public List<JournaledStorageLocation> getEvents(final long minEventId, final int maxResults) throws IOException {
+        final BooleanQuery query = new BooleanQuery();
+        query.add(NumericRangeQuery.newLongRange(IndexedFieldNames.EVENT_ID, minEventId, null, true, true), Occur.MUST);
+        
+        final TopDocs topDocs = searcher.search(query, maxResults, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG)));
+        return getLocations(topDocs);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
new file mode 100644
index 0000000..e955ae5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.Version;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.search.SearchableField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LuceneIndexWriter implements EventIndexWriter {
+    private static final Logger logger = LoggerFactory.getLogger(LuceneIndexWriter.class);
+    
+    @SuppressWarnings("unused")
+    private final JournalingRepositoryConfig config;
+    private final Set<SearchableField> nonAttributeSearchableFields;
+    private final Set<SearchableField> attributeSearchableFields;
+    
+    private final Directory directory;
+    private final Analyzer analyzer;
+    private final IndexWriter indexWriter;
+    private final AtomicLong indexMaxId = new AtomicLong(-1L);
+    
+    public LuceneIndexWriter(final File indexDir, final JournalingRepositoryConfig config) throws IOException {
+        this.config = config;
+        
+        attributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(config.getSearchableAttributes()));
+        nonAttributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(config.getSearchableFields()));
+        
+        directory = FSDirectory.open(indexDir);
+        analyzer = new StandardAnalyzer();
+        final IndexWriterConfig writerConfig = new IndexWriterConfig(Version.LATEST, analyzer);
+        indexWriter = new IndexWriter(directory, writerConfig);
+    }
+    
+    public EventIndexSearcher newIndexSearcher() throws IOException {
+        final DirectoryReader reader = DirectoryReader.open(indexWriter, false);
+        return new LuceneIndexSearcher(reader);
+    }
+
+    @Override
+    public void close() throws IOException {
+        IOException suppressed = null;
+        try {
+            indexWriter.close();
+        } catch (final IOException ioe) {
+            suppressed = ioe;
+        }
+        
+        analyzer.close();
+        
+        try {
+            directory.close();
+        } catch (final IOException ioe) {
+            if ( suppressed != null ) {
+                ioe.addSuppressed(suppressed);
+            }
+            
+            throw ioe;
+        }
+    }
+    
+    
+    private void addField(final Document doc, final SearchableField field, final String value, final Store store) {
+        if (value == null || (!nonAttributeSearchableFields.contains(field) && !field.isAttribute())) {
+            return;
+        }
+
+        doc.add(new StringField(field.getSearchableFieldName(), value.toLowerCase(), store));
+    }
+    
+    @Override
+    public void index(final Collection<JournaledProvenanceEvent> events) throws IOException {
+        long maxId = this.indexMaxId.get();
+        
+        for ( final JournaledProvenanceEvent event : events ) {
+            maxId = event.getEventId();
+
+            final Map<String, String> attributes = event.getAttributes();
+
+            final Document doc = new Document();
+            addField(doc, SearchableFields.FlowFileUUID, event.getFlowFileUuid(), Store.NO);
+            addField(doc, SearchableFields.Filename, attributes.get(CoreAttributes.FILENAME.key()), Store.NO);
+            addField(doc, SearchableFields.ComponentID, event.getComponentId(), Store.NO);
+            addField(doc, SearchableFields.AlternateIdentifierURI, event.getAlternateIdentifierUri(), Store.NO);
+            addField(doc, SearchableFields.EventType, event.getEventType().name(), Store.NO);
+            addField(doc, SearchableFields.Relationship, event.getRelationship(), Store.NO);
+            addField(doc, SearchableFields.Details, event.getDetails(), Store.NO);
+            addField(doc, SearchableFields.ContentClaimSection, event.getContentClaimSection(), Store.NO);
+            addField(doc, SearchableFields.ContentClaimContainer, event.getContentClaimContainer(), Store.NO);
+            addField(doc, SearchableFields.ContentClaimIdentifier, event.getContentClaimIdentifier(), Store.NO);
+            addField(doc, SearchableFields.SourceQueueIdentifier, event.getSourceQueueIdentifier(), Store.NO);
+
+            if (nonAttributeSearchableFields.contains(SearchableFields.TransitURI)) {
+                addField(doc, SearchableFields.TransitURI, event.getTransitUri(), Store.NO);
+            }
+
+            for (final SearchableField searchableField : attributeSearchableFields) {
+                addField(doc, searchableField, attributes.get(searchableField.getSearchableFieldName()), Store.NO);
+            }
+
+            // Index the fields that we always index (unless there's nothing else to index at all)
+            doc.add(new LongField(SearchableFields.LineageStartDate.getSearchableFieldName(), event.getLineageStartDate(), Store.NO));
+            doc.add(new LongField(SearchableFields.EventTime.getSearchableFieldName(), event.getEventTime(), Store.NO));
+            doc.add(new LongField(SearchableFields.FileSize.getSearchableFieldName(), event.getFileSize(), Store.NO));
+
+            final JournaledStorageLocation location = event.getStorageLocation();
+            doc.add(new StringField(IndexedFieldNames.CONTAINER_NAME, location.getContainerName(), Store.YES));
+            doc.add(new StringField(IndexedFieldNames.SECTION_NAME, location.getSectionName(), Store.YES));
+            doc.add(new StringField(IndexedFieldNames.JOURNAL_ID, location.getJournalId(), Store.YES));
+            doc.add(new LongField(IndexedFieldNames.BLOCK_INDEX, location.getBlockIndex(), Store.YES));
+            doc.add(new LongField(IndexedFieldNames.EVENT_ID, location.getEventId(), Store.YES));
+
+            for (final String lineageIdentifier : event.getLineageIdentifiers()) {
+                addField(doc, SearchableFields.LineageIdentifier, lineageIdentifier, Store.NO);
+            }
+
+            // If it's event is a FORK, or JOIN, add the FlowFileUUID for all child/parent UUIDs.
+            if (event.getEventType() == ProvenanceEventType.FORK || event.getEventType() == ProvenanceEventType.CLONE || event.getEventType() == ProvenanceEventType.REPLAY) {
+                for (final String uuid : event.getChildUuids()) {
+                    if (!uuid.equals(event.getFlowFileUuid())) {
+                        addField(doc, SearchableFields.FlowFileUUID, uuid, Store.NO);
+                    }
+                }
+            } else if (event.getEventType() == ProvenanceEventType.JOIN) {
+                for (final String uuid : event.getParentUuids()) {
+                    if (!uuid.equals(event.getFlowFileUuid())) {
+                        addField(doc, SearchableFields.FlowFileUUID, uuid, Store.NO);
+                    }
+                }
+            } else if (event.getEventType() == ProvenanceEventType.RECEIVE && event.getSourceSystemFlowFileIdentifier() != null) {
+                // If we get a receive with a Source System FlowFile Identifier, we add another Document that shows the UUID
+                // that the Source System uses to refer to the data.
+                final String sourceIdentifier = event.getSourceSystemFlowFileIdentifier();
+                final String sourceFlowFileUUID;
+                final int lastColon = sourceIdentifier.lastIndexOf(":");
+                if (lastColon > -1 && lastColon < sourceIdentifier.length() - 2) {
+                    sourceFlowFileUUID = sourceIdentifier.substring(lastColon + 1);
+                } else {
+                    sourceFlowFileUUID = null;
+                }
+
+                if (sourceFlowFileUUID != null) {
+                    addField(doc, SearchableFields.FlowFileUUID, sourceFlowFileUUID, Store.NO);
+                }
+            }
+
+            indexWriter.addDocument(doc);
+        }
+
+        // Update the index's max id
+        boolean updated = false;
+        do {
+            long curMax = indexMaxId.get();
+            if ( maxId > curMax ) {
+                updated = indexMaxId.compareAndSet(curMax, maxId);
+            } else {
+                updated = true;
+            }
+        } while (!updated);
+    }
+    
+    
+    @Override
+    public void delete(final String containerName, final String section, final String journalId) throws IOException {
+        final BooleanQuery query = new BooleanQuery();
+        query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.CONTAINER_NAME, containerName)), Occur.MUST));
+        query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.SECTION_NAME, section)), Occur.MUST));
+        query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.JOURNAL_ID, journalId)), Occur.MUST));
+        
+        indexWriter.deleteDocuments(query);
+    }
+    
+    
+    @Override
+    public void sync() throws IOException {
+        indexWriter.commit();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
new file mode 100644
index 0000000..4ae4b16
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.WildcardQuery;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StorageLocation;
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.search.SearchTerm;
+
+public class QueryUtils {
+    public static org.apache.lucene.search.Query convertQueryToLucene(final org.apache.nifi.provenance.search.Query query) {
+        if (query.getStartDate() == null && query.getEndDate() == null && query.getSearchTerms().isEmpty()) {
+            return new MatchAllDocsQuery();
+        }
+
+        final BooleanQuery luceneQuery = new BooleanQuery();
+        for (final SearchTerm searchTerm : query.getSearchTerms()) {
+            final String searchValue = searchTerm.getValue();
+            if (searchValue == null) {
+                throw new IllegalArgumentException("Empty search value not allowed (for term '" + searchTerm.getSearchableField().getFriendlyName() + "')");
+            }
+
+            if (searchValue.contains("*") || searchValue.contains("?")) {
+                luceneQuery.add(new BooleanClause(new WildcardQuery(new Term(searchTerm.getSearchableField().getSearchableFieldName(), searchTerm.getValue().toLowerCase())), Occur.MUST));
+            } else {
+                luceneQuery.add(new BooleanClause(new TermQuery(new Term(searchTerm.getSearchableField().getSearchableFieldName(), searchTerm.getValue().toLowerCase())), Occur.MUST));
+            }
+        }
+
+        final Long minBytes = query.getMinFileSize() == null ? null : DataUnit.parseDataSize(query.getMinFileSize(), DataUnit.B).longValue();
+        final Long maxBytes = query.getMaxFileSize() == null ? null : DataUnit.parseDataSize(query.getMaxFileSize(), DataUnit.B).longValue();
+        if (minBytes != null || maxBytes != null) {
+            luceneQuery.add(NumericRangeQuery.newLongRange(SearchableFields.FileSize.getSearchableFieldName(), minBytes, maxBytes, true, true), Occur.MUST);
+        }
+
+        final Long minDateTime = query.getStartDate() == null ? null : query.getStartDate().getTime();
+        final Long maxDateTime = query.getEndDate() == null ? null : query.getEndDate().getTime();
+        if (maxDateTime != null || minDateTime != null) {
+            luceneQuery.add(NumericRangeQuery.newLongRange(SearchableFields.EventTime.getSearchableFieldName(), minDateTime, maxDateTime, true, true), Occur.MUST);
+        }
+
+        return luceneQuery;
+    }
+    
+    
+    private static File getJournalFile(final JournaledStorageLocation location, final JournalingRepositoryConfig config) throws FileNotFoundException {
+        final File containerDir = config.getContainers().get(location.getContainerName());
+        if ( containerDir == null ) {
+            throw new FileNotFoundException("Could not find Container with name " + location.getContainerName());
+        }
+        
+        final String sectionName = location.getSectionName();
+        final File sectionFile = new File(containerDir, sectionName);
+        final File journalDir = new File(sectionFile, "journals");
+        final File journalFile = new File(journalDir, location.getJournalId() + ".journal");
+        
+        return journalFile;
+    }
+    
+    
+    public static Map<File, List<JournaledStorageLocation>> orderLocations(final List<StorageLocation> locations, final JournalingRepositoryConfig config) throws FileNotFoundException, IOException {
+        final Map<File, List<JournaledStorageLocation>> map = new HashMap<>();
+        
+        for ( final StorageLocation location : locations ) {
+            if ( !(location instanceof JournaledStorageLocation) ) {
+                throw new IllegalArgumentException(location + " is not a valid StorageLocation for this repository");
+            }
+            
+            final JournaledStorageLocation journaledLocation = (JournaledStorageLocation) location;
+            final File journalFile = getJournalFile(journaledLocation, config);
+            List<JournaledStorageLocation> list = map.get(journalFile);
+            if ( list == null ) {
+                list = new ArrayList<>();
+                map.put(journalFile, list);
+            }
+            
+            list.add(journaledLocation);
+        }
+        
+        for ( final List<JournaledStorageLocation> list : map.values() ) {
+            Collections.sort(list);
+        }
+        
+        return map;
+    }
+    
+    public static File getTocFile(final File journalFile) {
+        return new File(journalFile.getParentFile(), journalFile.getName() + ".toc");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/SearchResult.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/SearchResult.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/SearchResult.java
new file mode 100644
index 0000000..ac82438
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/SearchResult.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.util.List;
+
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+
+public class SearchResult {
+    private final int totalCount;
+    private final List<JournaledStorageLocation> locations;
+    
+    public SearchResult(final List<JournaledStorageLocation> locations, final int totalCount) {
+        this.totalCount = totalCount;
+        this.locations = locations;
+    }
+
+    public int getTotalCount() {
+        return totalCount;
+    }
+
+    public List<JournaledStorageLocation> getLocations() {
+        return locations;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializer.java
new file mode 100644
index 0000000..67b1cb6
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializer.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.io;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface Deserializer {
+
+    String getCodecName();
+    
+    ProvenanceEventRecord deserialize(DataInputStream in, int serializationVersion) throws IOException;
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializers.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializers.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializers.java
new file mode 100644
index 0000000..4be87e3
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Deserializers.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.io;
+
+public class Deserializers {
+    
+    public static Deserializer getDeserializer(final String codecName) {
+        switch (codecName) {
+            case StandardEventDeserializer.CODEC_NAME:
+                return new StandardEventDeserializer();
+            default:
+                throw new IllegalArgumentException("Unknown Provenance Serialization Codec: " + codecName);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Serializer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Serializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Serializer.java
new file mode 100644
index 0000000..8219b4c
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/Serializer.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.io;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+
+public interface Serializer {
+
+    /**
+     * Returns the serialization version that is used to serialize records
+     * @return
+     */
+    int getVersion();
+    
+    /**
+     * Returns the name of the codec used to serialize the records
+     * @return
+     */
+    String getCodecName();
+    
+    /**
+     * Serializes the given even to the given DataOutputStream.
+     * This method should NOT serialize the ID, as the ID is not yet known. The ID will instead by
+     * serialized to the stream appropriately by the JournalWriter.
+     * 
+     * @param event
+     * @param out
+     * @throws IOException
+     */
+    void serialize(ProvenanceEventRecord event, DataOutputStream out) throws IOException;
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventDeserializer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventDeserializer.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventDeserializer.java
new file mode 100644
index 0000000..fb537ee
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/io/StandardEventDeserializer.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.io;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.journaling.IdEnrichedProvenanceEvent;
+import org.apache.nifi.stream.io.StreamUtils;
+
+public class StandardEventDeserializer implements Deserializer {
+    public static final String CODEC_NAME = StandardEventSerializer.CODEC_NAME;
+    
+    @Override
+    public String getCodecName() {
+        return CODEC_NAME;
+    }
+    
+    @Override
+    public ProvenanceEventRecord deserialize(final DataInputStream in, final int serializationVersion) throws IOException {
+        final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder();
+
+        final long eventId = in.readLong();
+        final String eventTypeName = in.readUTF();
+        final ProvenanceEventType eventType = ProvenanceEventType.valueOf(eventTypeName);
+        builder.setEventType(eventType);
+        builder.setEventTime(in.readLong());
+
+        final Long flowFileEntryDate = in.readLong();
+        builder.setEventDuration(in.readLong());
+
+        final Set<String> lineageIdentifiers = new HashSet<>();
+        final int numLineageIdentifiers = in.readInt();
+        for (int i = 0; i < numLineageIdentifiers; i++) {
+            lineageIdentifiers.add(readUUID(in));
+        }
+
+        final long lineageStartDate = in.readLong();
+
+        builder.setComponentId(readNullableString(in));
+        builder.setComponentType(readNullableString(in));
+
+        final String uuid = readUUID(in);
+        builder.setFlowFileUUID(uuid);
+        builder.setDetails(readNullableString(in));
+
+        // Read in the FlowFile Attributes
+        final Map<String, String> previousAttrs = readAttributes(in, false);
+        final Map<String, String> attrUpdates = readAttributes(in, true);
+        builder.setAttributes(previousAttrs, attrUpdates);
+
+        final boolean hasContentClaim = in.readBoolean();
+        if (hasContentClaim) {
+            builder.setCurrentContentClaim(in.readUTF(), in.readUTF(), in.readUTF(), in.readLong(), in.readLong());
+        } else {
+            builder.setCurrentContentClaim(null, null, null, null, 0L);
+        }
+
+        final boolean hasPreviousClaim = in.readBoolean();
+        if (hasPreviousClaim) {
+            builder.setPreviousContentClaim(in.readUTF(), in.readUTF(), in.readUTF(), in.readLong(), in.readLong());
+        }
+
+        builder.setSourceQueueIdentifier(readNullableString(in));
+
+        // Read Event-Type specific fields.
+        if (eventType == ProvenanceEventType.FORK || eventType == ProvenanceEventType.JOIN || eventType == ProvenanceEventType.CLONE || eventType == ProvenanceEventType.REPLAY) {
+            final int numParents = in.readInt();
+            for (int i = 0; i < numParents; i++) {
+                builder.addParentUuid(readUUID(in));
+            }
+
+            final int numChildren = in.readInt();
+            for (int i = 0; i < numChildren; i++) {
+                builder.addChildUuid(readUUID(in));
+            }
+        } else if (eventType == ProvenanceEventType.RECEIVE) {
+            builder.setTransitUri(readNullableString(in));
+            builder.setSourceSystemFlowFileIdentifier(readNullableString(in));
+        } else if (eventType == ProvenanceEventType.SEND) {
+            builder.setTransitUri(readNullableString(in));
+        } else if (eventType == ProvenanceEventType.ADDINFO) {
+            builder.setAlternateIdentifierUri(readNullableString(in));
+        } else if (eventType == ProvenanceEventType.ROUTE) {
+            builder.setRelationship(readNullableString(in));
+        }
+
+        builder.setFlowFileEntryDate(flowFileEntryDate);
+        builder.setLineageIdentifiers(lineageIdentifiers);
+        builder.setLineageStartDate(lineageStartDate);
+        final ProvenanceEventRecord event = builder.build();
+        
+        return new IdEnrichedProvenanceEvent(event, eventId);
+    }
+
+    
+    private static Map<String, String> readAttributes(final DataInputStream dis, final boolean valueNullable) throws IOException {
+        final int numAttributes = dis.readInt();
+        final Map<String, String> attrs = new HashMap<>();
+        for (int i = 0; i < numAttributes; i++) {
+            final String key = readLongString(dis);
+            final String value = valueNullable ? readLongNullableString(dis) : readLongString(dis);
+            attrs.put(key, value);
+        }
+
+        return attrs;
+    }
+
+    private static String readUUID(final DataInputStream in) throws IOException {
+        final long msb = in.readLong();
+        final long lsb = in.readLong();
+        return new UUID(msb, lsb).toString();
+    }
+
+    private static String readNullableString(final DataInputStream in) throws IOException {
+        final boolean valueExists = in.readBoolean();
+        if (valueExists) {
+            return in.readUTF();
+        } else {
+            return null;
+        }
+    }
+
+    private static String readLongNullableString(final DataInputStream in) throws IOException {
+        final boolean valueExists = in.readBoolean();
+        if (valueExists) {
+            return readLongString(in);
+        } else {
+            return null;
+        }
+    }
+
+    private static String readLongString(final DataInputStream in) throws IOException {
+        final int length = in.readInt();
+        final byte[] strBytes = new byte[length];
+        StreamUtils.fillBuffer(in, strBytes);
+        return new String(strBytes, StandardCharsets.UTF_8);
+    }
+}


[04/12] incubator-nifi git commit: implemented ability to persist and recover records

Posted by ma...@apache.org.
implemented ability to persist and recover records


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a68bef62
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a68bef62
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a68bef62

Branch: refs/heads/journaling-prov-repo
Commit: a68bef62865d0dbdc2d4e65f1fc677f05dbca33c
Parents: ed53b46
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Feb 12 09:28:07 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Feb 12 09:28:07 2015 -0500

----------------------------------------------------------------------
 .../provenance/ProvenanceEventRepository.java   |  49 +-
 .../apache/nifi/provenance/StorageLocation.java |  25 +
 .../nifi/provenance/StoredProvenanceEvent.java  |  31 ++
 .../nifi/provenance/search/QueryResult.java     |   4 +-
 .../nifi/provenance/NamedSearchableField.java   |   4 +-
 .../nifi/provenance/StandardQueryResult.java    |   8 +-
 .../nifi/remote/io/CompressionInputStream.java  |   2 +-
 .../nifi/stream/io/ByteCountingInputStream.java |   5 +
 .../stream/io/ByteCountingOutputStream.java     |   5 +
 .../nifi/stream/io/LimitingInputStream.java     | 111 ++++
 .../stream/io/MinimumLengthInputStream.java     |  93 ++++
 .../util/RemoteProcessGroupUtils.class          | Bin 0 -> 9526 bytes
 .../remote/AbstractCommunicationsSession.class  | Bin 0 -> 2308 bytes
 .../nifi/remote/RemoteResourceFactory.class     | Bin 0 -> 8707 bytes
 .../nifi/remote/RemoteResourceManager.class     | Bin 0 -> 6898 bytes
 .../apache/nifi/remote/RemoteSiteListener.class | Bin 0 -> 841 bytes
 .../nifi/remote/SocketRemoteSiteListener.class  | Bin 0 -> 8448 bytes
 ...emoteGroupPort$EndpointConnectionState.class | Bin 0 -> 5427 bytes
 .../nifi/remote/StandardRemoteGroupPort.class   | Bin 0 -> 10677 bytes
 .../StandardRootGroupPort$FlowFileRequest.class | Bin 0 -> 5836 bytes
 ...StandardRootGroupPort$ProcessingResult.class | Bin 0 -> 5032 bytes
 ...upPort$StandardPortAuthorizationResult.class | Bin 0 -> 5159 bytes
 .../nifi/remote/StandardRootGroupPort.class     | Bin 0 -> 9700 bytes
 .../remote/codec/StandardFlowFileCodec.class    | Bin 0 -> 8538 bytes
 .../exception/UnsupportedCodecException.class   | Bin 0 -> 1057 bytes
 .../SocketChannelCommunicationsSession.class    | Bin 0 -> 3735 bytes
 .../remote/io/socket/SocketChannelInput.class   | Bin 0 -> 4008 bytes
 .../remote/io/socket/SocketChannelOutput.class  | Bin 0 -> 3741 bytes
 .../SSLSocketChannelCommunicationsSession.class | Bin 0 -> 4611 bytes
 .../io/socket/ssl/SSLSocketChannelInput.class   | Bin 0 -> 3127 bytes
 .../io/socket/ssl/SSLSocketChannelOutput.class  | Bin 0 -> 2587 bytes
 .../socket/ClusterManagerServerProtocol.class   | Bin 0 -> 10540 bytes
 .../protocol/socket/HandshakeProperty.class     | Bin 0 -> 917 bytes
 .../nifi/remote/protocol/socket/Response.class  | Bin 0 -> 2674 bytes
 .../remote/protocol/socket/ResponseCode.class   | Bin 0 -> 6889 bytes
 .../protocol/socket/SocketClientProtocol.class  | Bin 0 -> 8965 bytes
 .../socket/SocketFlowFileServerProtocol.class   | Bin 0 -> 8806 bytes
 .../remote/TestStandardRemoteGroupPort.class    | Bin 0 -> 5974 bytes
 .../pom.xml                                     |  40 ++
 .../journaling/IdEnrichedProvenanceEvent.java   | 165 ++++++
 .../journaling/JournaledProvenanceEvent.java    | 180 +++++++
 .../journaling/JournaledStorageLocation.java    |  87 ++++
 .../JournalingProvenanceRepository.java         | 413 +++++++++++++++
 .../config/JournalingRepositoryConfig.java      | 328 ++++++++++++
 .../journaling/index/EventIndexSearcher.java    |  36 ++
 .../journaling/index/EventIndexWriter.java      |  48 ++
 .../journaling/index/IndexedFieldNames.java     |  27 +
 .../index/JournalingSearchableFields.java       |  29 ++
 .../journaling/index/LuceneIndexSearcher.java   | 113 ++++
 .../journaling/index/LuceneIndexWriter.java     | 223 ++++++++
 .../provenance/journaling/index/QueryUtils.java | 124 +++++
 .../journaling/index/SearchResult.java          |  40 ++
 .../provenance/journaling/io/Deserializer.java  |  30 ++
 .../provenance/journaling/io/Deserializers.java |  30 ++
 .../provenance/journaling/io/Serializer.java    |  49 ++
 .../io/StandardEventDeserializer.java           | 162 ++++++
 .../journaling/io/StandardEventSerializer.java  | 170 ++++++
 .../journaling/journals/JournalReader.java      |  47 ++
 .../journaling/journals/JournalWriter.java      |  96 ++++
 .../journals/StandardJournalReader.java         | 178 +++++++
 .../journals/StandardJournalWriter.java         | 236 +++++++++
 .../partition/JournalingPartition.java          | 424 +++++++++++++++
 .../journaling/partition/Partition.java         |  85 +++
 .../journaling/partition/PartitionAction.java   |  23 +
 .../journaling/partition/PartitionManager.java  |  67 +++
 .../partition/QueuingPartitionManager.java      | 185 +++++++
 .../partition/VoidPartitionAction.java          |  23 +
 .../journaling/tasks/CompressionTask.java       | 177 +++++++
 .../journaling/toc/StandardTocReader.java       |  89 ++++
 .../journaling/toc/StandardTocWriter.java       |  98 ++++
 .../journaling/toc/TocJournalReader.java        |  98 ++++
 .../provenance/journaling/toc/TocReader.java    |  46 ++
 .../provenance/journaling/toc/TocWriter.java    |  46 ++
 .../nifi/provenance/journaling/TestUtil.java    |  55 ++
 .../journaling/index/TestEventIndexWriter.java  |  85 +++
 .../journals/TestJournalReadWrite.java          |  82 +++
 .../journals/TestStandardJournalReader.java     | 516 +++++++++++++++++++
 .../journals/TestStandardJournalWriter.java     | 130 +++++
 .../journaling/toc/TestStandardTocReader.java   |  91 ++++
 .../VolatileProvenanceRepository.java           | 109 +++-
 .../TestVolatileProvenanceRepository.java       |   2 +-
 81 files changed, 5569 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
index 39c829e..8c7a044 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRepository.java
@@ -16,7 +16,9 @@
  */
 package org.apache.nifi.provenance;
 
+import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.nifi.events.EventReporter;
@@ -31,7 +33,7 @@ import org.apache.nifi.provenance.search.SearchableField;
  * has stored, and providing query capabilities against the events.
  *
  */
-public interface ProvenanceEventRepository {
+public interface ProvenanceEventRepository extends Closeable {
 
     /**
      * Performs any initialization needed. This should be called only by the
@@ -56,7 +58,7 @@ public interface ProvenanceEventRepository {
      *
      * @param event
      */
-    void registerEvent(ProvenanceEventRecord event);
+    void registerEvent(ProvenanceEventRecord event) throws IOException;
 
     /**
      * Adds the given events to the repository.
@@ -68,7 +70,7 @@ public interface ProvenanceEventRepository {
      *
      * @param events
      */
-    void registerEvents(Iterable<ProvenanceEventRecord> events);
+    void registerEvents(Collection<ProvenanceEventRecord> events) throws IOException;
 
     /**
      * Returns a List of all <code>ProvenanceEventRecord</code>s in the
@@ -80,15 +82,26 @@ public interface ProvenanceEventRepository {
      * @return
      * @throws java.io.IOException
      */
-    List<ProvenanceEventRecord> getEvents(long firstRecordId, final int maxRecords) throws IOException;
+    List<StoredProvenanceEvent> getEvents(long firstRecordId, final int maxRecords) throws IOException;
 
     /**
+     * Returns a List of all <code>ProvenanceEventRecord</code>s in the repository whose locations
+     * match those specified. If any event cannot be found, it will be skipped.
+     * 
+     * @param storageLocations
+     * @return
+     * @throws IOException
+     */
+    List<StoredProvenanceEvent> getEvents(List<StorageLocation> storageLocations) throws IOException;
+    
+    
+    /**
      * Returns the largest ID of any event that is queryable in the repository.
      * If no queryable events exists, returns null
      *
      * @return
      */
-    Long getMaxEventId();
+    Long getMaxEventId() throws IOException;
 
     /**
      * Submits an asynchronous request to process the given query, returning an
@@ -139,9 +152,19 @@ public interface ProvenanceEventRepository {
      * @return
      * @throws IOException
      */
-    ProvenanceEventRecord getEvent(long id) throws IOException;
+    StoredProvenanceEvent getEvent(long id) throws IOException;
 
     /**
+     * Returns the Provenance Event Record with the given location, if it exists, or
+     * <code>null</code> otherwise
+     * 
+     * @param location
+     * @return
+     * @throws IOException
+     */
+    StoredProvenanceEvent getEvent(StorageLocation location) throws IOException;
+    
+    /**
      * Submits a request to expand the parents of the event with the given id
      *
      * @param eventId the one-up id of the Event to expand
@@ -166,13 +189,6 @@ public interface ProvenanceEventRepository {
     ComputeLineageSubmission submitExpandChildren(long eventId);
 
     /**
-     * Closes the repository, freeing any resources
-     *
-     * @throws IOException
-     */
-    void close() throws IOException;
-
-    /**
      * Returns a list of all fields that can be searched via the
      * {@link #submitQuery(nifi.provenance.search.Query)} method
      *
@@ -187,4 +203,11 @@ public interface ProvenanceEventRepository {
      * @return
      */
     List<SearchableField> getSearchableAttributes();
+    
+    /**
+     * Returns the timestamp of the earliest event that is available in the repository, or <code>null</code> if no
+     * events exist in the repository
+     * @return
+     */
+    Long getEarliestEventTime() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StorageLocation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StorageLocation.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StorageLocation.java
new file mode 100644
index 0000000..ad45065
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StorageLocation.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance;
+
+/**
+ * Marker interface that a {@link ProvenanceEventRepository} provides when it stores a provenance event.
+ * This storage location can then be used to retrieve the record at a later date.
+ */
+public interface StorageLocation {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StoredProvenanceEvent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StoredProvenanceEvent.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StoredProvenanceEvent.java
new file mode 100644
index 0000000..67f9d9f
--- /dev/null
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/StoredProvenanceEvent.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance;
+
+
+/**
+ * Represents a Provenance event that has been stored in a Provenance Event Repository.
+ */
+public interface StoredProvenanceEvent extends ProvenanceEventRecord {
+
+    /**
+     * Returns the location at which this Provenance Event has been stored.
+     * @return
+     */
+    StorageLocation getStorageLocation();
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
index 3dd0b71..5561014 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java
@@ -19,7 +19,7 @@ package org.apache.nifi.provenance.search;
 import java.util.Date;
 import java.util.List;
 
-import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
 
 public interface QueryResult {
 
@@ -29,7 +29,7 @@ public interface QueryResult {
      *
      * @return
      */
-    List<ProvenanceEventRecord> getMatchingEvents();
+    List<StoredProvenanceEvent> getMatchingEvents();
 
     /**
      * Returns the total number of Provenance Events that hit

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
index dc2903f..38d65b5 100644
--- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java
@@ -32,11 +32,11 @@ public class NamedSearchableField implements SearchableField {
     private final String friendlyName;
     private final boolean attribute;
 
-    NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute) {
+    public NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute) {
         this(identifier, searchableName, friendlyName, attribute, SearchableFieldType.STRING);
     }
 
-    NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute, final SearchableFieldType fieldType) {
+    public NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute, final SearchableFieldType fieldType) {
         this.identifier = requireNonNull(identifier);
         this.searchableName = requireNonNull(searchableName);
         this.friendlyName = requireNonNull(friendlyName);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
index 9a9a27d..7e75357 100644
--- a/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
+++ b/nifi/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java
@@ -40,7 +40,7 @@ public class StandardQueryResult implements QueryResult {
 
     private final Lock writeLock = rwLock.writeLock();
     // guarded by writeLock
-    private final List<ProvenanceEventRecord> matchingRecords = new ArrayList<>();
+    private final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>();
     private long totalHitCount;
     private int numCompletedSteps = 0;
     private Date expirationDate;
@@ -58,14 +58,14 @@ public class StandardQueryResult implements QueryResult {
     }
 
     @Override
-    public List<ProvenanceEventRecord> getMatchingEvents() {
+    public List<StoredProvenanceEvent> getMatchingEvents() {
         readLock.lock();
         try {
             if (matchingRecords.size() <= query.getMaxResults()) {
                 return new ArrayList<>(matchingRecords);
             }
 
-            final List<ProvenanceEventRecord> copy = new ArrayList<>(query.getMaxResults());
+            final List<StoredProvenanceEvent> copy = new ArrayList<>(query.getMaxResults());
             for (int i = 0; i < query.getMaxResults(); i++) {
                 copy.add(matchingRecords.get(i));
             }
@@ -141,7 +141,7 @@ public class StandardQueryResult implements QueryResult {
         }
     }
 
-    public void update(final Collection<ProvenanceEventRecord> matchingRecords, final long totalHits) {
+    public void update(final Collection<StoredProvenanceEvent> matchingRecords, final long totalHits) {
         writeLock.lock();
         try {
             this.matchingRecords.addAll(matchingRecords);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
index 71cf894..d953185 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java
@@ -142,7 +142,7 @@ public class CompressionInputStream extends InputStream {
             return -1;
         }
 
-        return buffer[bufferIndex++];
+        return buffer[bufferIndex++] & 0xFF;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
index 8294af3..3977a3d 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingInputStream.java
@@ -28,7 +28,12 @@ public class ByteCountingInputStream extends InputStream {
     private long bytesSinceMark = 0L;
 
     public ByteCountingInputStream(final InputStream in) {
+        this(in, 0);
+    }
+    
+    public ByteCountingInputStream(final InputStream in, final long initialCount) {
         this.in = in;
+        this.bytesRead = initialCount;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
index d8e1a42..dc8d197 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/ByteCountingOutputStream.java
@@ -25,7 +25,12 @@ public class ByteCountingOutputStream extends OutputStream {
     private long bytesWritten = 0L;
 
     public ByteCountingOutputStream(final OutputStream out) {
+        this(out, 0L);
+    }
+    
+    public ByteCountingOutputStream(final OutputStream out, final long initialCount) {
         this.out = out;
+        this.bytesWritten = initialCount;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
new file mode 100644
index 0000000..421d579
--- /dev/null
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
@@ -0,0 +1,111 @@
+/*
+ * 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.stream.io;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LimitingInputStream extends InputStream {
+
+    private final InputStream in;
+    private final long limit;
+    private long bytesRead = 0;
+
+    public LimitingInputStream(final InputStream in, final long limit) {
+        this.in = in;
+        this.limit = limit;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int val = in.read();
+        if (val > -1) {
+            bytesRead++;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(b.length, limit - bytesRead);
+
+        final int val = in.read(b, 0, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(len, limit - bytesRead);
+
+        final int val = in.read(b, off, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public long skip(final long n) throws IOException {
+        final long skipped = in.skip(Math.min(n, limit - bytesRead));
+        bytesRead += skipped;
+        return skipped;
+    }
+
+    @Override
+    public int available() throws IOException {
+        return in.available();
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public void mark(int readlimit) {
+        in.mark(readlimit);
+    }
+
+    @Override
+    public boolean markSupported() {
+        return in.markSupported();
+    }
+
+    @Override
+    public void reset() throws IOException {
+        in.reset();
+    }
+
+    public long getLimit() {
+    	return limit;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java
new file mode 100644
index 0000000..2e93599
--- /dev/null
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/MinimumLengthInputStream.java
@@ -0,0 +1,93 @@
+/*
+ * 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.stream.io;
+
+import java.io.EOFException;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * An InputStream that will throw EOFException if the underlying InputStream runs out of data before reaching the
+ * configured minimum amount of data
+ */
+public class MinimumLengthInputStream extends FilterInputStream {
+
+	private final long minLength;
+	private long consumedCount = 0L;
+	
+	public MinimumLengthInputStream(final InputStream in, final long minLength) {
+		super(in);
+		this.minLength = minLength;
+	}
+
+	
+	@Override
+	public int read() throws IOException {
+		final int b = super.read();
+		if ( b < 0 && consumedCount < minLength ) {
+			throw new EOFException();
+		}
+		
+		if ( b >= 0 ) {
+			consumedCount++;
+		}
+		
+		return b;
+	}
+	
+	@Override
+	public int read(byte[] b) throws IOException {
+		return read(b, 0, b.length);
+	}
+	
+	public int read(byte[] b, int off, int len) throws IOException {
+		final int num = super.read(b, off, len);
+		
+		if ( num < 0 && consumedCount < minLength ) {
+			throw new EOFException();
+		}
+		
+		if ( num >= 0 ) {
+			consumedCount += num;
+		}
+
+		return num;
+	}
+	
+	@Override
+	public long skip(final long n) throws IOException {
+		long skipped = super.skip(n);
+		if ( skipped < 1 ) {
+			final int b = super.read();
+			if ( b >= 0 ) {
+				skipped = 1;
+			}
+		}
+		
+		if ( skipped < 0 && consumedCount < minLength ) {
+			throw new EOFException();
+		}
+		
+		if ( skipped >= 0 ) {
+			consumedCount += skipped;
+		}
+		
+		return skipped;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class
new file mode 100644
index 0000000..a6951d4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/controller/util/RemoteProcessGroupUtils.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class
new file mode 100644
index 0000000..2e868ea
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/AbstractCommunicationsSession.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class
new file mode 100644
index 0000000..9c6e821
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceFactory.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class
new file mode 100644
index 0000000..bb3fc77
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteResourceManager.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class
new file mode 100644
index 0000000..a56b5ba
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/RemoteSiteListener.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class
new file mode 100644
index 0000000..9780f75
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class
new file mode 100644
index 0000000..f184d64
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort$EndpointConnectionState.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class
new file mode 100644
index 0000000..0740bb4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class
new file mode 100644
index 0000000..d517458
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$FlowFileRequest.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class
new file mode 100644
index 0000000..1cf5ceb
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$ProcessingResult.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class
new file mode 100644
index 0000000..3ad7542
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort$StandardPortAuthorizationResult.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class
new file mode 100644
index 0000000..4db4735
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class
new file mode 100644
index 0000000..e49ffe8
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/codec/StandardFlowFileCodec.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class
new file mode 100644
index 0000000..39dd49a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/exception/UnsupportedCodecException.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class
new file mode 100644
index 0000000..b415421
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelCommunicationsSession.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class
new file mode 100644
index 0000000..551097e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class
new file mode 100644
index 0000000..6913767
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class
new file mode 100644
index 0000000..f6e9f20
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelCommunicationsSession.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class
new file mode 100644
index 0000000..41fe366
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class
new file mode 100644
index 0000000..9b9cdc0
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutput.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class
new file mode 100644
index 0000000..2ef1c39
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ClusterManagerServerProtocol.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class
new file mode 100644
index 0000000..fad8245
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/HandshakeProperty.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class
new file mode 100644
index 0000000..27ec8d3
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/Response.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class
new file mode 100644
index 0000000..4673aec
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/ResponseCode.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class
new file mode 100644
index 0000000..ac7e1b9
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class
new file mode 100644
index 0000000..933ef4d
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class
new file mode 100644
index 0000000..0e77276
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/bin/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
new file mode 100644
index 0000000..5997281
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/pom.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0"?>
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<groupId>org.apache.nifi</groupId>
+		<artifactId>nifi-provenance-repository-bundle</artifactId>
+		<version>0.0.2-incubating-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>nifi-journaling-provenance-repository</artifactId>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-api</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-data-provenance-utils</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.nifi</groupId>
+			<artifactId>nifi-properties</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-core</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-analyzers-common</artifactId>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.lucene</groupId>
+			<artifactId>lucene-queryparser</artifactId>
+		</dependency>
+	</dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/IdEnrichedProvenanceEvent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/IdEnrichedProvenanceEvent.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/IdEnrichedProvenanceEvent.java
new file mode 100644
index 0000000..f9bc76a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/IdEnrichedProvenanceEvent.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+
+
+/**
+ * Wraps a Provenance Event Record but provides a different Event ID for the record. This is done because we
+ * do not want to allow the ProvenanceEventBuilder to expose an identifier, because the identifier should be
+ * assigned only when the event has been persisted to the repository.
+ */
+public class IdEnrichedProvenanceEvent implements ProvenanceEventRecord {
+    private final ProvenanceEventRecord event;
+    private final long id;
+    
+    public IdEnrichedProvenanceEvent(final ProvenanceEventRecord event, final long id) {
+        this.event = event;
+        this.id = id;
+    }
+
+    @Override
+    public long getEventId() {
+        return id;
+    }
+    
+    public long getEventTime() {
+        return event.getEventTime();
+    }
+
+    public long getFlowFileEntryDate() {
+        return event.getFlowFileEntryDate();
+    }
+
+    public long getLineageStartDate() {
+        return event.getLineageStartDate();
+    }
+
+    public Set<String> getLineageIdentifiers() {
+        return event.getLineageIdentifiers();
+    }
+
+    public long getFileSize() {
+        return event.getFileSize();
+    }
+
+    public Long getPreviousFileSize() {
+        return event.getPreviousFileSize();
+    }
+
+    public long getEventDuration() {
+        return event.getEventDuration();
+    }
+
+    public ProvenanceEventType getEventType() {
+        return event.getEventType();
+    }
+
+    public Map<String, String> getAttributes() {
+        return event.getAttributes();
+    }
+
+    public Map<String, String> getPreviousAttributes() {
+        return event.getPreviousAttributes();
+    }
+
+    public Map<String, String> getUpdatedAttributes() {
+        return event.getUpdatedAttributes();
+    }
+
+    public String getComponentId() {
+        return event.getComponentId();
+    }
+
+    public String getComponentType() {
+        return event.getComponentType();
+    }
+
+    public String getTransitUri() {
+        return event.getTransitUri();
+    }
+
+    public String getSourceSystemFlowFileIdentifier() {
+        return event.getSourceSystemFlowFileIdentifier();
+    }
+
+    public String getFlowFileUuid() {
+        return event.getFlowFileUuid();
+    }
+
+    public List<String> getParentUuids() {
+        return event.getParentUuids();
+    }
+
+    public List<String> getChildUuids() {
+        return event.getChildUuids();
+    }
+
+    public String getAlternateIdentifierUri() {
+        return event.getAlternateIdentifierUri();
+    }
+
+    public String getDetails() {
+        return event.getDetails();
+    }
+
+    public String getRelationship() {
+        return event.getRelationship();
+    }
+
+    public String getSourceQueueIdentifier() {
+        return event.getSourceQueueIdentifier();
+    }
+
+    public String getContentClaimSection() {
+        return event.getContentClaimSection();
+    }
+
+    public String getPreviousContentClaimSection() {
+        return event.getPreviousContentClaimSection();
+    }
+
+    public String getContentClaimContainer() {
+        return event.getContentClaimContainer();
+    }
+
+    public String getPreviousContentClaimContainer() {
+        return event.getPreviousContentClaimContainer();
+    }
+
+    public String getContentClaimIdentifier() {
+        return event.getContentClaimIdentifier();
+    }
+
+    public String getPreviousContentClaimIdentifier() {
+        return event.getPreviousContentClaimIdentifier();
+    }
+
+    public Long getContentClaimOffset() {
+        return event.getContentClaimOffset();
+    }
+
+    public Long getPreviousContentClaimOffset() {
+        return event.getPreviousContentClaimOffset();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledProvenanceEvent.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledProvenanceEvent.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledProvenanceEvent.java
new file mode 100644
index 0000000..ac68495
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledProvenanceEvent.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+
+public class JournaledProvenanceEvent implements StoredProvenanceEvent {
+
+    private final ProvenanceEventRecord event;
+    private final JournaledStorageLocation location;
+    
+    public JournaledProvenanceEvent(final ProvenanceEventRecord event, final JournaledStorageLocation location) {
+        this.event = event;
+        this.location = location;
+    }
+
+    @Override
+    public JournaledStorageLocation getStorageLocation() {
+        return location;
+    }
+    
+    public long getEventId() {
+        return event.getEventId();
+    }
+
+    public long getEventTime() {
+        return event.getEventTime();
+    }
+
+    public long getFlowFileEntryDate() {
+        return event.getFlowFileEntryDate();
+    }
+
+    public long getLineageStartDate() {
+        return event.getLineageStartDate();
+    }
+
+    public Set<String> getLineageIdentifiers() {
+        return event.getLineageIdentifiers();
+    }
+
+    public long getFileSize() {
+        return event.getFileSize();
+    }
+
+    public Long getPreviousFileSize() {
+        return event.getPreviousFileSize();
+    }
+
+    public long getEventDuration() {
+        return event.getEventDuration();
+    }
+
+    public ProvenanceEventType getEventType() {
+        return event.getEventType();
+    }
+
+    public Map<String, String> getAttributes() {
+        return event.getAttributes();
+    }
+
+    public Map<String, String> getPreviousAttributes() {
+        return event.getPreviousAttributes();
+    }
+
+    public Map<String, String> getUpdatedAttributes() {
+        return event.getUpdatedAttributes();
+    }
+
+    public String getComponentId() {
+        return event.getComponentId();
+    }
+
+    public String getComponentType() {
+        return event.getComponentType();
+    }
+
+    public String getTransitUri() {
+        return event.getTransitUri();
+    }
+
+    public String getSourceSystemFlowFileIdentifier() {
+        return event.getSourceSystemFlowFileIdentifier();
+    }
+
+    public String getFlowFileUuid() {
+        return event.getFlowFileUuid();
+    }
+
+    public List<String> getParentUuids() {
+        return event.getParentUuids();
+    }
+
+    public List<String> getChildUuids() {
+        return event.getChildUuids();
+    }
+
+    public String getAlternateIdentifierUri() {
+        return event.getAlternateIdentifierUri();
+    }
+
+    public String getDetails() {
+        return event.getDetails();
+    }
+
+    public String getRelationship() {
+        return event.getRelationship();
+    }
+
+    public String getSourceQueueIdentifier() {
+        return event.getSourceQueueIdentifier();
+    }
+
+    public String getContentClaimSection() {
+        return event.getContentClaimSection();
+    }
+
+    public String getPreviousContentClaimSection() {
+        return event.getPreviousContentClaimSection();
+    }
+
+    public String getContentClaimContainer() {
+        return event.getContentClaimContainer();
+    }
+
+    public String getPreviousContentClaimContainer() {
+        return event.getPreviousContentClaimContainer();
+    }
+
+    public String getContentClaimIdentifier() {
+        return event.getContentClaimIdentifier();
+    }
+
+    public String getPreviousContentClaimIdentifier() {
+        return event.getPreviousContentClaimIdentifier();
+    }
+
+    public Long getContentClaimOffset() {
+        return event.getContentClaimOffset();
+    }
+
+    public Long getPreviousContentClaimOffset() {
+        return event.getPreviousContentClaimOffset();
+    }
+
+    public boolean equals(Object obj) {
+        return location.equals(obj);
+    }
+
+    public int hashCode() {
+        return location.hashCode();
+    }
+
+    public String toString() {
+        return location.toString();
+    }
+    
+    
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a68bef62/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
new file mode 100644
index 0000000..cdb98c8
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling;
+
+import java.util.Comparator;
+
+import org.apache.nifi.provenance.StorageLocation;
+
+public class JournaledStorageLocation implements StorageLocation, Comparable<JournaledStorageLocation>, Comparator<JournaledStorageLocation> {
+    private final String containerName;
+    private final String sectionName;
+    private final String journalId;
+    private final int blockIndex;
+    private final long eventId;
+    
+    public JournaledStorageLocation(final String containerName, final String sectionName, final String journalId, final int blockIndex, final long eventId) {
+        this.containerName = containerName;
+        this.sectionName = sectionName;
+        this.journalId = journalId;
+        this.blockIndex = blockIndex;
+        this.eventId = eventId;
+    }
+    
+    public String getContainerName() {
+        return containerName;
+    }
+    
+    public String getSectionName() {
+        return sectionName;
+    }
+    
+    public String getJournalId() {
+        return journalId;
+    }
+    
+    public int getBlockIndex() {
+        return blockIndex;
+    }
+    
+    public long getEventId() {
+        return eventId;
+    }
+
+    @Override
+    public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+        final int containerVal = o1.getContainerName().compareTo(o2.getContainerName());
+        if ( containerVal != 0 ) {
+            return containerVal;
+        }
+        
+        final int sectionVal = o1.getSectionName().compareTo(o2.getSectionName());
+        if ( sectionVal != 0 ) {
+            return sectionVal;
+        }
+        
+        final int journalVal = o1.getJournalId().compareTo(o2.getJournalId());
+        if ( journalVal != 0 ) {
+            return journalVal;
+        }
+        
+        final int blockVal = Integer.compare(o1.getBlockIndex(), o2.getBlockIndex());
+        if ( blockVal != 0 ) {
+            return blockVal;
+        }
+        
+        return Long.compare(o1.getEventId(), o2.getEventId());
+    }
+
+    @Override
+    public int compareTo(final JournaledStorageLocation o) {
+        return compare(this, o);
+    }
+}


[08/12] incubator-nifi git commit: NIFI-388: Initial implementation of prov repo; not yet finished but pushing so that the code is not lost

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/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 4cce231..0753e9e 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
@@ -19,28 +19,39 @@ package org.apache.nifi.provenance.journaling.query;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.nifi.provenance.AsyncLineageSubmission;
 import org.apache.nifi.provenance.AsyncQuerySubmission;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
+import org.apache.nifi.provenance.ProvenanceEventRepository;
 import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
 import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
 import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.IndexAction;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
 import org.apache.nifi.provenance.journaling.index.QueryUtils;
 import org.apache.nifi.provenance.journaling.index.SearchResult;
+import org.apache.nifi.provenance.journaling.index.VoidIndexAction;
 import org.apache.nifi.provenance.journaling.journals.JournalReader;
 import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
-import org.apache.nifi.provenance.journaling.partition.Partition;
-import org.apache.nifi.provenance.journaling.partition.PartitionManager;
-import org.apache.nifi.provenance.journaling.partition.VoidPartitionAction;
 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.search.Query;
 import org.apache.nifi.provenance.search.QuerySubmission;
 import org.slf4j.Logger;
@@ -50,14 +61,17 @@ public class StandardQueryManager implements QueryManager {
     private static final Logger logger = LoggerFactory.getLogger(StandardQueryManager.class);
     
     private final int maxConcurrentQueries;
+    private final IndexManager indexManager;
+    private final ExecutorService executor;
     private final JournalingRepositoryConfig config;
-    private final PartitionManager partitionManager;
     private final ConcurrentMap<String, AsyncQuerySubmission> querySubmissionMap = new ConcurrentHashMap<>();
+    private final ConcurrentMap<String, AsyncLineageSubmission> lineageSubmissionMap = new ConcurrentHashMap<>();
     
-    public StandardQueryManager(final PartitionManager partitionManager, final JournalingRepositoryConfig config, final int maxConcurrentQueries) {
+    public StandardQueryManager(final IndexManager indexManager, final ExecutorService executor, final JournalingRepositoryConfig config, final int maxConcurrentQueries) {
         this.config = config;
         this.maxConcurrentQueries = maxConcurrentQueries;
-        this.partitionManager = partitionManager;
+        this.indexManager = indexManager;
+        this.executor = executor;
     }
     
     @Override
@@ -74,12 +88,66 @@ public class StandardQueryManager implements QueryManager {
         if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) {
             final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
 
+            // empty query. Just get the latest events.
+            final Runnable runnable = new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        logger.debug("Fetching latest events from Provenance repo");
+                        final long indexStartNanos = System.nanoTime();
+                        
+                        // Query each index for the latest events.
+                        final Set<List<JournaledStorageLocation>> locationSet = indexManager.withEachIndex(new IndexAction<List<JournaledStorageLocation>>() {
+                            @Override
+                            public List<JournaledStorageLocation> perform(final EventIndexSearcher searcher) throws IOException {
+                                return searcher.getLatestEvents(query.getMaxResults());
+                            }
+                        });
+                        final long indexMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - indexStartNanos);
+                        final long retrievalStartNanos = System.nanoTime();
+                        
+                        final List<JournaledStorageLocation> orderedLocations = new ArrayList<>();
+                        for ( final List<JournaledStorageLocation> locations : locationSet ) {
+                            orderedLocations.addAll(locations);
+                        }
+                        
+                        Collections.sort(orderedLocations, new Comparator<JournaledStorageLocation>() {
+                            @Override
+                            public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+                                return Long.compare(o1.getEventId(), o2.getEventId());
+                            }
+                        });
+
+                        final List<JournaledStorageLocation> locationsToKeep;
+                        if ( orderedLocations.size() > query.getMaxResults() ) {
+                            locationsToKeep = orderedLocations.subList(0, query.getMaxResults());
+                        } else {
+                            locationsToKeep = orderedLocations;
+                        }
+                        
+                        final List<StoredProvenanceEvent> matchingRecords = getEvents(locationsToKeep, new AtomicInteger(locationsToKeep.size()));
+                        
+                        final long totalNumEvents = indexManager.getNumberOfEvents();
+                        final long retrievalMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - retrievalStartNanos);
+                        logger.debug("Updated query result with {} matching records; total number of events = {}; index search took {} millis, event retrieval took {} millis", matchingRecords.size(), totalNumEvents, indexMillis, retrievalMillis);
+                        result.getResult().update(matchingRecords, totalNumEvents);
+                    } catch (final Exception e) {
+                        result.getResult().setError("Failed to obtain latest events in repository due to " + e);
+                        logger.error("Failed to obtain latest events in repository due to {}", e.toString());
+                        if ( logger.isDebugEnabled() ) {
+                            logger.error("", e);
+                        }
+                    }
+                }
+            };
+            
+            executor.submit(runnable);
             querySubmissionMap.put(query.getIdentifier(), result);
             return result;
         }
 
         final AtomicInteger retrievalCount = new AtomicInteger(query.getMaxResults());
-        final AsyncQuerySubmission submission = new AsyncQuerySubmission(query, config.getPartitionCount()) {
+        final AsyncQuerySubmission submission = new AsyncQuerySubmission(query, indexManager.getNumberOfIndices()) {
             @Override
             public void cancel() {
                 super.cancel();
@@ -89,56 +157,252 @@ public class StandardQueryManager implements QueryManager {
         
         querySubmissionMap.put(query.getIdentifier(), submission);
 
-        partitionManager.withEachPartition(new VoidPartitionAction() {
-            @SuppressWarnings({ "rawtypes", "unchecked" })
-            @Override
-            public void perform(final Partition partition) throws IOException {
-                logger.debug("Running {} against {}", query, partition);
-                
-                try (final EventIndexSearcher searcher = partition.newIndexSearcher()) {
-                    final SearchResult searchResult = searcher.search(query);
-                    logger.debug("{} has {} hits against {} over {} files", query, searchResult.getTotalCount(), partition, searchResult.getLocations().size());
-                    
-                    final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>();
-                    final Map<File, List<JournaledStorageLocation>> locationMap = QueryUtils.orderLocations((List) searchResult.getLocations(), config);
-                    for ( final Map.Entry<File, List<JournaledStorageLocation>> entry : locationMap.entrySet() ) {
-                        final File journalFile = entry.getKey();
-                        final List<JournaledStorageLocation> locations = entry.getValue();
+        try {
+            indexManager.withEachIndex(new VoidIndexAction() {
+                @Override
+                public void perform(final EventIndexSearcher searcher) throws IOException {
+                    try {
+                        logger.debug("Running {} against {}", query, searcher);
                         
-                        if ( retrievalCount.get() <= 0 ) {
-                            break;
-                        }
+                        final long indexStart = System.nanoTime();
+                        final SearchResult searchResult = searcher.search(query);
+                        final long indexMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - indexStart);
+                        logger.debug("{} has {} hits against {} over {} files", query, searchResult.getTotalCount(), searcher, searchResult.getLocations().size());
                         
-                        try (final JournalReader reader = new StandardJournalReader(journalFile);
-                             final TocReader tocReader = new StandardTocReader(QueryUtils.getTocFile(journalFile))) {
-                            
-                            for ( final JournaledStorageLocation location : locations ) {
-                                final long blockOffset = tocReader.getBlockOffset(location.getBlockIndex());
-                                final ProvenanceEventRecord event = reader.getEvent(blockOffset, location.getEventId());
-                                matchingRecords.add(new JournaledProvenanceEvent(event, location));
-                                
-                                final int recordsLeft = retrievalCount.decrementAndGet();
-                                if ( recordsLeft <= 0 ) {
-                                    break;
-                                }
-                            }
-                        }
+                        final long retrievalStart = System.nanoTime();
+                        final List<StoredProvenanceEvent> matchingRecords = getEvents(searchResult.getLocations(), retrievalCount);
+                        final long retrievalMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - retrievalStart);
+                        
+                        logger.debug("Finished executing {} against {}; found {} total matches, retrieved {} of them; index search took {} millis, record retrieval took {} millis", 
+                                query, searcher, searchResult.getTotalCount(), matchingRecords.size(), indexMillis, retrievalMillis);
+                        submission.getResult().update(matchingRecords, searchResult.getTotalCount());
+                    } catch (final Throwable t) {
+                        submission.getResult().setError("Failed to execute query " + query + " against " + searcher + " due to " + t);
+                        throw t;
                     }
-                    
-                    logger.debug("Finished executing {} against {}", query, partition);
-                    submission.getResult().update(matchingRecords, searchResult.getTotalCount());
-                } catch (final Exception e) {
-                    submission.getResult().setError("Failed to query " + partition + " due to " + e.toString());
-                    throw e;
                 }
+            }, true);
+        } catch (final IOException ioe) {
+            // only set the error here if it's not already set because we have the least amount of information here
+            if ( submission.getResult().getError() == null ) {
+                submission.getResult().setError("Failed to execute query " + query + " due to " + ioe);
             }
-        }, true);
+        }
         
         return submission;
     }
 
+    
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    private List<StoredProvenanceEvent> getEvents(final List<JournaledStorageLocation> allLocations, final AtomicInteger retrievalCount) throws IOException {
+        final List<StoredProvenanceEvent> matchingRecords = new ArrayList<>();
+        final Map<File, List<JournaledStorageLocation>> locationMap = QueryUtils.orderLocations((List) allLocations, config);
+        for ( final Map.Entry<File, List<JournaledStorageLocation>> entry : locationMap.entrySet() ) {
+            final File journalFile = entry.getKey();
+            final List<JournaledStorageLocation> locations = entry.getValue();
+            
+            if ( retrievalCount.get() <= 0 ) {
+                break;
+            }
+            
+            try (final JournalReader reader = new StandardJournalReader(journalFile);
+                 final TocReader tocReader = new StandardTocReader(QueryUtils.getTocFile(journalFile))) {
+                
+                for ( final JournaledStorageLocation location : locations ) {
+                    final long blockOffset = tocReader.getBlockOffset(location.getBlockIndex());
+                    final ProvenanceEventRecord event = reader.getEvent(blockOffset, location.getEventId());
+                    matchingRecords.add(new JournaledProvenanceEvent(event, location));
+                    
+                    final int recordsLeft = retrievalCount.decrementAndGet();
+                    if ( recordsLeft <= 0 ) {
+                        break;
+                    }
+                }
+            }
+        }
+        
+        return matchingRecords;
+    }
+    
     @Override
     public QuerySubmission retrieveQuerySubmission(final String queryIdentifier) {
         return querySubmissionMap.get(queryIdentifier);
     }
+    
+    @Override
+    public ComputeLineageSubmission retrieveLineageSubmission(final String lineageIdentifier) {
+        return lineageSubmissionMap.get(lineageIdentifier);
+    }
+    
+    @Override
+    public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) {
+        return submitLineageComputation(Collections.singleton(flowFileUuid), LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE);
+    }
+    
+    private AsyncLineageSubmission submitLineageComputation(final Collection<String> flowFileUuids, final LineageComputationType computationType, final Long eventId, final long startTimestamp, final long endTimestamp) {
+        final AsyncLineageSubmission lineageSubmission = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, indexManager.getNumberOfIndices());
+
+        final AtomicInteger retrievalCount = new AtomicInteger(2000);
+        final Runnable runnable = new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    indexManager.withEachIndex(new VoidIndexAction() {
+                        @Override
+                        public void perform(EventIndexSearcher searcher) throws IOException {
+                            logger.debug("Obtaining lineage events for FlowFile UUIDs {} for {}", flowFileUuids, searcher);
+                            final long startNanos = System.nanoTime();
+                            
+                            final List<JournaledStorageLocation> locations = searcher.getEventsForFlowFiles(flowFileUuids, startTimestamp, endTimestamp);
+                            final List<StoredProvenanceEvent> matchingRecords = getEvents(locations, retrievalCount);
+                            lineageSubmission.getResult().update(matchingRecords);
+                            
+                            final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+                            logger.debug("Finished querying for lineage events; found {} events in {} millis", matchingRecords.size(), millis);
+                        }
+                    });
+                } catch (final IOException ioe) {
+                    lineageSubmission.getResult().setError("Failed to calculate FlowFile Lineage due to " + ioe);
+                    logger.error("Failed to calculate FlowFile Lineage due to {}", ioe.toString());
+                    if ( logger.isDebugEnabled() ) {
+                        logger.error("", ioe);
+                    }
+                }
+            }
+        };
+        
+        executor.submit(runnable);
+        lineageSubmissionMap.putIfAbsent(lineageSubmission.getLineageIdentifier(), lineageSubmission);
+        return lineageSubmission;
+    }
+
+    
+    @Override
+    public ComputeLineageSubmission submitExpandChildren(final ProvenanceEventRepository eventRepo, final long eventId) {
+        final Set<String> flowFileUuids = Collections.synchronizedSet(new HashSet<String>());
+        final AsyncLineageSubmission lineageSubmission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, flowFileUuids, indexManager.getNumberOfIndices());
+
+        final Runnable runnable = new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    logger.debug("Obtaining event with id {} in order to expand children", eventId);
+                    final StoredProvenanceEvent event = eventRepo.getEvent(eventId);
+                    if ( event == null ) {
+                        lineageSubmission.getResult().setError("Cannot expand children of event with ID " + eventId + " because that event cannot be found");
+                        logger.warn("Cannot expand children of event with ID {} because that event cannot be found", eventId);
+                        return;
+                    }
+                    
+                    logger.debug("Found event with id {}; searching for children", eventId);
+                    
+                    switch (event.getEventType()) {
+                        case CLONE:
+                        case FORK:
+                        case JOIN:
+                        case REPLAY:
+                            break;
+                        default:
+                            logger.warn("Cannot expand children of event with ID {} because event type is {}", eventId, event.getEventType());
+                            lineageSubmission.getResult().setError("Cannot expand children of event with ID " + eventId + 
+                                    " because that event is of type " + event.getEventType() + 
+                                    ", and that type does not support expansion of children");
+                            return;
+                    }
+                    
+                    final List<String> childUuids = event.getChildUuids();
+                    flowFileUuids.addAll(childUuids);
+                    
+                    final AtomicInteger retrievalCount = new AtomicInteger(100);
+                    indexManager.withEachIndex(new VoidIndexAction() {
+                        @Override
+                        public void perform(EventIndexSearcher searcher) throws IOException {
+                            final long startNanos = System.nanoTime();
+                            logger.debug("Finding children of event with id {} using {}", eventId, searcher);
+                            
+                            final List<JournaledStorageLocation> locations = searcher.getEventsForFlowFiles(flowFileUuids, event.getEventTime(), Long.MAX_VALUE);
+                            final List<StoredProvenanceEvent> matchingRecords = getEvents(locations, retrievalCount);
+                            lineageSubmission.getResult().update(matchingRecords);
+                            
+                            final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+                            logger.debug("Found {} children of event {} in {} millis", matchingRecords.size(), eventId, millis);
+                        }
+                    });
+                } catch (final IOException ioe) {
+                    
+                }
+            }
+        };
+        
+        executor.submit(runnable);
+        lineageSubmissionMap.putIfAbsent(lineageSubmission.getLineageIdentifier(), lineageSubmission);
+        return lineageSubmission;
+    }
+    
+    @Override
+    public ComputeLineageSubmission submitExpandParents(final ProvenanceEventRepository eventRepo, final long eventId) {
+        final Set<String> flowFileUuids = Collections.synchronizedSet(new HashSet<String>());
+        final AsyncLineageSubmission lineageSubmission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, flowFileUuids, indexManager.getNumberOfIndices());
+
+        final Runnable runnable = new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    logger.debug("Obtaining event with id {} in order to expand children", eventId);
+                    final StoredProvenanceEvent event = eventRepo.getEvent(eventId);
+                    if ( event == null ) {
+                        logger.warn("Cannot expand children of event with ID {} because that event cannot be found", eventId);
+                        lineageSubmission.getResult().setError("Cannot expand children of event with ID " + eventId + " because that event cannot be found");
+                        return;
+                    }
+                    
+                    logger.debug("Found event with id {}; searching for children", eventId);
+                    
+                    switch (event.getEventType()) {
+                        case CLONE:
+                        case FORK:
+                        case JOIN:
+                        case REPLAY:
+                            break;
+                        default:
+                            logger.warn("Cannot expand parents of event with ID {} because event type is {}", eventId, event.getEventType());
+                            lineageSubmission.getResult().setError("Cannot expand parents of event with ID " + eventId + 
+                                    " because that event is of type " + event.getEventType() + 
+                                    ", and that type does not support expansion of children");
+                            return;
+                    }
+                    
+                    final List<String> parentUuids = event.getParentUuids();
+                    flowFileUuids.addAll(parentUuids);
+                    
+                    final AtomicInteger retrievalCount = new AtomicInteger(100);
+                    indexManager.withEachIndex(new VoidIndexAction() {
+                        @Override
+                        public void perform(EventIndexSearcher searcher) throws IOException {
+                            final long startNanos = System.nanoTime();
+                            logger.debug("Finding parents of event with id {} using {}", eventId, searcher);
+                            
+                            final List<JournaledStorageLocation> locations = searcher.getEventsForFlowFiles(flowFileUuids, event.getLineageStartDate(), event.getEventTime());
+                            final List<StoredProvenanceEvent> matchingRecords = getEvents(locations, retrievalCount);
+                            lineageSubmission.getResult().update(matchingRecords);
+                            
+                            final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+                            logger.debug("Found {} parents of event {} in {} millis", matchingRecords.size(), eventId, millis);
+                        }
+                    });
+                } catch (final IOException ioe) {
+                    
+                }
+            }
+        };
+        
+        executor.submit(runnable);
+        lineageSubmissionMap.putIfAbsent(lineageSubmission.getLineageIdentifier(), lineageSubmission);
+        return lineageSubmission;
+    }
+    
+    
+    @Override
+    public void close() throws IOException {
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
index a6a487b..fc9fb46 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
@@ -18,8 +18,11 @@ package org.apache.nifi.provenance.journaling.tasks;
 
 import java.io.EOFException;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collections;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
@@ -34,7 +37,10 @@ import org.apache.nifi.provenance.journaling.toc.TocWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class CompressionTask implements Runnable {
+/**
+ * Compresses a journal file and returns the new size of the journal
+ */
+public class CompressionTask implements Callable<Long> {
     public static final String FILE_EXTENSION = ".compress";
     
     private static final Logger logger = LoggerFactory.getLogger(CompressionTask.class);
@@ -56,7 +62,10 @@ public class CompressionTask implements Runnable {
         long blockOffset = tocReader.getBlockOffset(blockIndex);
         tocWriter.addBlockOffset(blockOffset);
         long nextBlockOffset = tocReader.getBlockOffset(blockIndex + 1);
-        
+
+        // we write the events one at a time here so that we can ensure that when the block
+        // changes we are able to insert a new block into the TOC, as the blocks have to contain
+        // the same number of events, since the index just knows about the block index.
         try {
             while ((event = reader.nextEvent()) != null) {
                 // Check if we've gone beyond the offset of the next block. If so, write
@@ -97,7 +106,7 @@ public class CompressionTask implements Runnable {
             }
         }
         
-        return false;
+        return !file.exists();
     }
     
     /**
@@ -125,11 +134,24 @@ public class CompressionTask implements Runnable {
     }
     
     @Override
-    public void run() {
+    public Long call() {
+        final long startNanos = System.nanoTime();
+        final long preCompressionSize = journalFile.length();
+        
         try {
             final File compressedFile = new File(journalFile.getParentFile(), journalFile.getName() + FILE_EXTENSION);
             final File compressedTocFile = new File(tocFile.getParentFile(), tocFile.getName() + FILE_EXTENSION);
 
+            if ( compressedFile.exists() && !compressedFile.delete() ) {
+                logger.error("Compressed file {} already exists and could not remove it; compression task failed", compressedFile);
+                return preCompressionSize;
+            }
+            
+            if ( compressedTocFile.exists() && !compressedTocFile.delete() ) {
+                logger.error("Compressed TOC file {} already exists and could not remove it; compression task failed", compressedTocFile);
+                return preCompressionSize;
+            }
+            
             try (final JournalReader journalReader = new StandardJournalReader(journalFile);
                 final JournalWriter compressedWriter = new StandardJournalWriter(journalId, compressedFile, true, new StandardEventSerializer());
                 final TocReader tocReader = new StandardTocReader(tocFile);
@@ -137,14 +159,19 @@ public class CompressionTask implements Runnable {
                 
                 compress(journalReader, compressedWriter, tocReader, compressedTocWriter);
                 compressedWriter.sync();
+            } catch (final FileNotFoundException fnfe) {
+                logger.info("Failed to compress Journal File {} because it has already been removed", journalFile);
+                return 0L;
             }
-
+            
+            final long postCompressionSize = compressedFile.length();
+            
             final boolean deletedJournal = delete(journalFile);
             if ( !deletedJournal ) {
                 delete(compressedFile);
                 delete(compressedTocFile);
                 logger.error("Failed to remove Journal file {}; considering compression task a failure", journalFile);
-                return;
+                return preCompressionSize;
             }
             
             final boolean deletedToc = delete(tocFile);
@@ -152,7 +179,7 @@ public class CompressionTask implements Runnable {
                 delete(compressedFile);
                 delete(compressedTocFile);
                 logger.error("Failed to remove TOC file for {}; considering compression task a failure", journalFile);
-                return;
+                return preCompressionSize;
             }
             
             final boolean renamedJournal = rename(compressedFile, journalFile);
@@ -165,12 +192,18 @@ public class CompressionTask implements Runnable {
                 logger.error("Failed to rename {} to {}; this journal file may be inaccessible until it is renamed", compressedTocFile, tocFile);
             }
             
-            logger.info("Successfully compressed Journal File {}");
+            final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+            final double percent = (postCompressionSize / preCompressionSize) * 100D;
+            final String pct = String.format("%.2f", percent);
+            logger.info("Successfully compressed Journal File {} in {} millis; size changed from {} bytes to {} bytes ({}% of original size)", journalFile, millis, preCompressionSize, postCompressionSize, pct);
+            return postCompressionSize;
         } catch (final IOException ioe) {
             logger.error("Failed to compress Journal File {} due to {}", journalFile, ioe.toString());
             if ( logger.isDebugEnabled() ) {
                 logger.error("", ioe);
             }
+            
+            return preCompressionSize;
         }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
index 995acf9..ae4635f 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocReader.java
@@ -83,6 +83,14 @@ public class StandardTocReader implements TocReader {
     }
 
     @Override
+    public long getLastBlockOffset() {
+        if ( offsets.length == 0 ) {
+            return 0L;
+        }
+        return offsets[offsets.length - 1];
+    }
+    
+    @Override
     public void close() throws IOException {
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
index fea6057..9ee07e0 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
@@ -16,12 +16,17 @@
  */
 package org.apache.nifi.provenance.journaling.toc;
 
+import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.EOFException;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.nio.file.FileAlreadyExistsException;
 
 /**
@@ -51,7 +56,24 @@ public class StandardTocWriter implements TocWriter {
      */
     public StandardTocWriter(final File file, final boolean compressionFlag, final boolean alwaysSync) throws IOException {
         if ( file.exists() ) {
-            throw new FileAlreadyExistsException(file.getAbsolutePath());
+            // Check if the header actually exists. If so, throw FileAlreadyExistsException
+            // If no data is in the file, we will just overwrite it.
+            try (final InputStream fis = new FileInputStream(file);
+                 final InputStream bis = new BufferedInputStream(fis);
+                 final DataInputStream dis = new DataInputStream(bis)) {
+                dis.read();
+                dis.read();
+
+                // we always add the first offset when the writer is created so we allow this to exist.
+                dis.readLong();
+                final int nextByte = dis.read();
+                
+                if ( nextByte > -1 ) {
+                    throw new FileAlreadyExistsException(file.getAbsolutePath());
+                }
+            } catch (final EOFException eof) {
+                // no real data. overwrite file.
+            }
         }
         
         if ( !file.getParentFile().exists() && !file.getParentFile().mkdirs() ) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
index eca664e..c3f9df5 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocJournalReader.java
@@ -33,13 +33,13 @@ public class TocJournalReader implements Closeable {
     
     private final String containerName;
     private final String sectionName;
-    private final String journalId;
+    private final Long journalId;
     
     private int blockIndex;
     private long nextBlockOffset;
     
     
-    public TocJournalReader(final String containerName, final String sectionName, final String journalId, final File journalFile) throws IOException {
+    public TocJournalReader(final String containerName, final String sectionName, final Long journalId, final File journalFile) throws IOException {
         this.containerName = containerName;
         this.sectionName = sectionName;
         this.journalId = journalId;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
index 9f6a264..18d7189 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/TocReader.java
@@ -43,4 +43,9 @@ public interface TocReader extends Closeable {
      */
     long getBlockOffset(int blockIndex);
     
+    /**
+     * Returns the byte offset into the Journal File of the last Block in the given index
+     * @return
+     */
+    long getLastBlockOffset();
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/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 a547a8a..30e100a 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
@@ -23,14 +23,23 @@ import static org.junit.Assert.assertNull;
 import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
 import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.lineage.ComputeLineageResult;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
+import org.apache.nifi.provenance.lineage.LineageNode;
+import org.apache.nifi.provenance.lineage.LineageNodeType;
+import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
 import org.apache.nifi.provenance.search.Query;
 import org.apache.nifi.provenance.search.QueryResult;
 import org.apache.nifi.provenance.search.QuerySubmission;
@@ -93,6 +102,190 @@ public class TestJournalingProvenanceRepository {
     }
     
     
+    @Test
+    public void testStoreRestartAndRetrieve() throws IOException {
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        config.setPartitionCount(3);
+        
+        try {
+            try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+                repo.initialize(null);
+                final Map<String, String> attributes = new HashMap<>();
+                
+                for (int i=0; i < 10; i++) {
+                    attributes.put("i", String.valueOf(i));
+                    repo.registerEvent(TestUtil.generateEvent(i, attributes));
+                }
+                
+                assertEquals(10L, repo.getMaxEventId().longValue());
+            }
+    
+            try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+                repo.initialize(null);
+
+                assertEquals(10L, repo.getMaxEventId().longValue());
+
+                // retrieve records one at a time.
+                for (int i=0; i < 10; i++) {
+                    final StoredProvenanceEvent event = repo.getEvent(i);
+                    assertNotNull(event);
+                    assertEquals((long) i, event.getEventId());
+                    assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+                }
+                
+                final List<StoredProvenanceEvent> events = repo.getEvents(0, 1000);
+                assertNotNull(events);
+                assertEquals(10, events.size());
+                for (int i=0; i < 10; i++) {
+                    final StoredProvenanceEvent event = events.get(i);
+                    assertNotNull(event);
+                    assertEquals((long) i, event.getEventId());
+                    assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+                }
+            }
+        } finally {
+            for ( final File file : containers.values() ) {
+                if ( file.exists() ) {
+                    FileUtils.deleteFile(file, true);
+                }
+            }
+        }
+    }
+    
+    
+    @Test
+    public void testStoreRestartRetrieveAndExpireOnTime() throws IOException, InterruptedException {
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        config.setPartitionCount(3);
+        
+        try {
+            try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+                repo.initialize(null);
+                final Map<String, String> attributes = new HashMap<>();
+                
+                for (int i=0; i < 10; i++) {
+                    attributes.put("i", String.valueOf(i));
+                    repo.registerEvent(TestUtil.generateEvent(i, attributes));
+                }
+                
+                assertEquals(10L, repo.getMaxEventId().longValue());
+            }
+    
+            config.setExpirationFrequency(1, TimeUnit.SECONDS);
+            try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+                repo.initialize(null);
+
+                assertEquals(10L, repo.getMaxEventId().longValue());
+
+                // retrieve records one at a time.
+                for (int i=0; i < 10; i++) {
+                    final StoredProvenanceEvent event = repo.getEvent(i);
+                    assertNotNull(event);
+                    assertEquals((long) i, event.getEventId());
+                    assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+                }
+                
+                final List<StoredProvenanceEvent> events = repo.getEvents(0, 1000);
+                assertNotNull(events);
+                assertEquals(10, events.size());
+                for (int i=0; i < 10; i++) {
+                    final StoredProvenanceEvent event = events.get(i);
+                    assertNotNull(event);
+                    assertEquals((long) i, event.getEventId());
+                    assertEquals("00000000-0000-0000-0000-00000000000" + i, event.getFlowFileUuid());
+                }
+                
+                // wait a bit for the events to be expired
+                TimeUnit.SECONDS.sleep(2L);
+                
+                // retrieve records one at a time.
+                for (int i=0; i < 10; i++) {
+                    final StoredProvenanceEvent event = repo.getEvent(i);
+                    assertNull("Event " + i + " still exists", event);
+                }
+                
+                final List<StoredProvenanceEvent> allEvents = repo.getEvents(0, 1000);
+                assertNotNull(allEvents);
+                assertEquals(0, allEvents.size());
+            }
+        } finally {
+            for ( final File file : containers.values() ) {
+                if ( file.exists() ) {
+                    FileUtils.deleteFile(file, true);
+                }
+            }
+        }
+    }
+    
+    
+    @Test
+    public void testExpireOnSize() throws IOException, InterruptedException {
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        config.setPartitionCount(3);
+        config.setMaxStorageCapacity(1024L * 50);
+        config.setEventExpiration(2, TimeUnit.SECONDS);
+        config.setExpirationFrequency(1, TimeUnit.SECONDS);
+        config.setJournalRolloverPeriod(1, TimeUnit.SECONDS);
+        config.setCompressOnRollover(false);
+        
+        try {
+            try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+                repo.initialize(null);
+                final Map<String, String> attributes = new HashMap<>();
+                
+                final int numEventsToInsert = 1000;
+                for (int i=0; i < numEventsToInsert; i++) {
+                    attributes.put("i", String.valueOf(i));
+                    repo.registerEvent(TestUtil.generateEvent(i, attributes));
+                }
+
+                final List<StoredProvenanceEvent> eventsBeforeExpire = repo.getEvents(0, numEventsToInsert * 2);
+                assertNotNull(eventsBeforeExpire);
+                assertEquals(numEventsToInsert, eventsBeforeExpire.size());
+                
+                // wait a bit for expiration to occur
+                TimeUnit.SECONDS.sleep(3L);
+                
+                // generate an event for each partition to force a rollover of the journals
+                for (int i=0; i < config.getPartitionCount(); i++) {
+                    repo.registerEvent(TestUtil.generateEvent(100000L));
+                }
+
+                TimeUnit.SECONDS.sleep(1L);
+  
+                // retrieve records one at a time.
+                for (int i=0; i < numEventsToInsert; i++) {
+                    final StoredProvenanceEvent event = repo.getEvent(i);
+                    assertNull("Event " + i + " still exists", event);
+                }
+                
+                final List<StoredProvenanceEvent> eventsAfterExpire = repo.getEvents(0, numEventsToInsert * 2);
+                assertNotNull(eventsAfterExpire);
+                assertEquals(3, eventsAfterExpire.size());
+            }
+        } finally {
+            for ( final File file : containers.values() ) {
+                if ( file.exists() ) {
+                    FileUtils.deleteFile(file, true);
+                }
+            }
+        }
+    }
+    
+    
+    
     @Test(timeout=10000000)
     public void testSearchByUUID() throws IOException, InterruptedException {
         final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
@@ -141,4 +334,98 @@ public class TestJournalingProvenanceRepository {
         }
     }
     
+    
+    @Test(timeout=10000)
+    public void testReceiveDropLineage() throws IOException, InterruptedException {
+        final JournalingRepositoryConfig config = new JournalingRepositoryConfig();
+        final Map<String, File> containers = new HashMap<>();
+        containers.put("container1", new File("target/" + UUID.randomUUID().toString()));
+        containers.put("container2", new File("target/" + UUID.randomUUID().toString()));
+        config.setContainers(containers);
+        
+        config.setPartitionCount(3);
+        config.setSearchableFields(Arrays.asList(new SearchableField[] {
+                SearchableFields.FlowFileUUID
+        }));
+        
+        try (final JournalingProvenanceRepository repo = new JournalingProvenanceRepository(config)) {
+            repo.initialize(null);
+        
+            final String uuid = "00000000-0000-0000-0000-000000000001";
+            final Map<String, String> attributes = new HashMap<>();
+            attributes.put("abc", "xyz");
+            attributes.put("uuid", uuid);
+            attributes.put("filename", "file-" + uuid);
+    
+            final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder()
+                .setEventType(ProvenanceEventType.RECEIVE)
+                .setFlowFileUUID(uuid)
+                .setComponentType("Unit Test")
+                .setComponentId(UUID.randomUUID().toString())
+                .setEventTime(System.currentTimeMillis())
+                .setFlowFileEntryDate(System.currentTimeMillis() - 1000L)
+                .setLineageStartDate(System.currentTimeMillis() - 2000L)
+                .setCurrentContentClaim(null, null, null, null, 0L)
+                .setAttributes(null, attributes == null ? Collections.<String, String>emptyMap() : attributes);
+
+            builder.setTransitUri("nifi://unit-test");
+            attributes.put("uuid", uuid);
+            builder.setComponentId("1234");
+            builder.setComponentType("dummy processor");
+    
+            // Add RECEIVE Event
+            repo.registerEvent(builder.build());
+    
+            builder.setEventTime(System.currentTimeMillis() + 1);
+            builder.setEventType(ProvenanceEventType.DROP);
+            builder.setTransitUri(null);
+            
+            // Add DROP event
+            repo.registerEvent(builder.build());
+            
+            // register unrelated even to make sure we don't get this one.
+            builder.setFlowFileUUID("00000000-0000-0000-0000-000000000002");
+            repo.registerEvent(builder.build());
+            
+            final ComputeLineageSubmission submission = repo.submitLineageComputation(uuid);
+            assertNotNull(submission);
+            
+            final ComputeLineageResult result = submission.getResult();
+            while ( !result.isFinished() ) {
+                Thread.sleep(50L);
+            }
+            
+            assertNull(result.getError());
+            
+            final List<LineageNode> nodes = result.getNodes();
+            assertEquals(3, nodes.size());  // RECEIVE, FlowFile node, DROP
+            
+            int receiveCount = 0;
+            int dropCount = 0;
+            int flowFileNodeCount = 0;
+            for ( final LineageNode node : nodes ) {
+                assertEquals(uuid, node.getFlowFileUuid());
+                
+                if ( LineageNodeType.PROVENANCE_EVENT_NODE.equals(node.getNodeType()) ) {
+                    final ProvenanceEventLineageNode eventNode = (ProvenanceEventLineageNode) node;
+                    if ( eventNode.getEventType() == ProvenanceEventType.RECEIVE ) {
+                        receiveCount++;
+                    } else if ( eventNode.getEventType() == ProvenanceEventType.DROP ) {
+                        dropCount++;
+                    }
+                } else {
+                    flowFileNodeCount++;
+                }
+            }
+            
+            assertEquals(1, receiveCount);
+            assertEquals(1, dropCount);
+            assertEquals(1, flowFileNodeCount);
+        } finally {
+            for ( final File file : containers.values() ) {
+                FileUtils.deleteFile(file, true);
+            }
+        }
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/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 dfaeb1a..e611aaa 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
@@ -35,6 +35,7 @@ import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
 import org.apache.nifi.provenance.search.Query;
 import org.apache.nifi.provenance.search.SearchTerms;
 import org.apache.nifi.provenance.search.SearchableField;
+import org.apache.nifi.util.file.FileUtils;
 import org.junit.Test;
 
 public class TestEventIndexWriter {
@@ -51,11 +52,9 @@ public class TestEventIndexWriter {
         
         final File indexDir = new File("target/" + UUID.randomUUID().toString());
         
-        final File journalFile = new File("target/" + UUID.randomUUID().toString());
         try (final LuceneIndexWriter indexWriter = new LuceneIndexWriter(indexDir, config)) {
-            
             final ProvenanceEventRecord event = TestUtil.generateEvent(23L);
-            final JournaledStorageLocation location = new JournaledStorageLocation("container", "section", "journalId", 2, 23L);
+            final JournaledStorageLocation location = new JournaledStorageLocation("container", "section", 1L, 2, 23L);
             final JournaledProvenanceEvent storedEvent = new JournaledProvenanceEvent(event, location);
             indexWriter.index(Collections.singleton(storedEvent));
             
@@ -72,12 +71,12 @@ public class TestEventIndexWriter {
                 assertNotNull(found);
                 assertEquals("container", found.getContainerName());
                 assertEquals("section", found.getSectionName());
-                assertEquals("journalId", found.getJournalId());
+                assertEquals(1L, found.getJournalId().longValue());
                 assertEquals(2, found.getBlockIndex());
                 assertEquals(23L, found.getEventId());
             }
         } finally {
-            journalFile.delete();
+            FileUtils.deleteFile(indexDir, true);
         }
         
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
index d5eab8e..43efa7e 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/journals/TestStandardJournalWriter.java
@@ -17,11 +17,13 @@
 package org.apache.nifi.provenance.journaling.journals;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.FileAlreadyExistsException;
 import java.nio.file.Files;
 import java.util.Collections;
 import java.util.UUID;
@@ -32,11 +34,47 @@ import org.apache.nifi.provenance.journaling.TestUtil;
 import org.apache.nifi.provenance.journaling.io.StandardEventDeserializer;
 import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
 import org.apache.nifi.remote.io.CompressionInputStream;
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestStandardJournalWriter {
 
     @Test
+    public void testOverwriteEmptyFile() throws IOException {
+        final File journalFile = new File("target/" + UUID.randomUUID().toString());
+        try {
+            assertTrue( journalFile.createNewFile() );
+            
+            try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, true, new StandardEventSerializer())) {
+                
+            }
+        } finally {
+            FileUtils.deleteFile(journalFile, false);
+        }
+    }
+    
+    @Test
+    public void testDoNotOverwriteNonEmptyFile() throws IOException {
+        final File journalFile = new File("target/" + UUID.randomUUID().toString());
+        try {
+            assertTrue( journalFile.createNewFile() );
+            
+            try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, true, new StandardEventSerializer())) {
+                writer.write(Collections.singleton(TestUtil.generateEvent(1L)), 1L);
+            }
+            
+            try (final StandardJournalWriter writer = new StandardJournalWriter(1L, journalFile, true, new StandardEventSerializer())) {
+                Assert.fail("StandardJournalWriter attempted to overwrite existing file");
+            } catch (final FileAlreadyExistsException faee) {
+                // expected
+            }
+        } finally {
+            FileUtils.deleteFile(journalFile, false);
+        }
+    }
+    
+    @Test
     public void testOneBlockOneRecordWriteCompressed() throws IOException {
         final File journalFile = new File("target/" + UUID.randomUUID().toString());
         

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocWriter.java
new file mode 100644
index 0000000..3cddb5e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/test/java/org/apache/nifi/provenance/journaling/toc/TestStandardTocWriter.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.toc;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileAlreadyExistsException;
+import java.util.UUID;
+
+import org.apache.nifi.util.file.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestStandardTocWriter {
+    @Test
+    public void testOverwriteEmptyFile() throws IOException {
+        final File tocFile = new File("target/" + UUID.randomUUID().toString() + ".toc");
+        try {
+            assertTrue( tocFile.createNewFile() );
+            
+            try (final StandardTocWriter writer = new StandardTocWriter(tocFile, false, false)) {
+            }
+        } finally {
+            FileUtils.deleteFile(tocFile, false);
+        }
+    }
+    
+    @Test
+    public void testDoNotOverwriteNonEmptyFile() throws IOException {
+        final File tocFile = new File("target/" + UUID.randomUUID().toString() + ".toc");
+        try {
+            assertTrue( tocFile.createNewFile() );
+            
+            try (final StandardTocWriter writer = new StandardTocWriter(tocFile, false, false)) {
+                writer.addBlockOffset(0L);
+                writer.addBlockOffset(34L);
+            }
+            
+            try (final StandardTocWriter writer = new StandardTocWriter(tocFile, false, false)) {
+                Assert.fail("StandardTocWriter attempted to overwrite existing file");
+            } catch (final FileAlreadyExistsException faee) {
+                // expected
+            }
+        } finally {
+            FileUtils.deleteFile(tocFile, false);
+        }
+    }
+}


[09/12] incubator-nifi git commit: NIFI-388: Initial implementation of prov repo; not yet finished but pushing so that the code is not lost

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
index 1ace37f..bba6899 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
@@ -23,12 +23,16 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
@@ -37,9 +41,6 @@ import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
 import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
 import org.apache.nifi.provenance.journaling.index.EventIndexWriter;
 import org.apache.nifi.provenance.journaling.index.IndexManager;
-import org.apache.nifi.provenance.journaling.index.LuceneIndexSearcher;
-import org.apache.nifi.provenance.journaling.index.LuceneIndexWriter;
-import org.apache.nifi.provenance.journaling.index.MultiIndexSearcher;
 import org.apache.nifi.provenance.journaling.index.QueryUtils;
 import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
 import org.apache.nifi.provenance.journaling.journals.JournalReader;
@@ -47,8 +48,9 @@ import org.apache.nifi.provenance.journaling.journals.JournalWriter;
 import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
 import org.apache.nifi.provenance.journaling.journals.StandardJournalWriter;
 import org.apache.nifi.provenance.journaling.tasks.CompressionTask;
+import org.apache.nifi.provenance.journaling.toc.StandardTocReader;
 import org.apache.nifi.provenance.journaling.toc.StandardTocWriter;
-import org.apache.nifi.provenance.journaling.toc.TocJournalReader;
+import org.apache.nifi.provenance.journaling.toc.TocReader;
 import org.apache.nifi.provenance.journaling.toc.TocWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,29 +64,32 @@ public class JournalingPartition implements Partition {
     
     private final File section;
     private final File journalsDir;
-    private final JournalingRepositoryConfig config;
+    private final IndexManager indexManager;
+    private final AtomicLong containerSize;
     private final ExecutorService executor;
-    
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-    private final Lock readLock = rwLock.readLock();
-    private final Lock writeLock = rwLock.writeLock();
+    private final JournalingRepositoryConfig config;
     
     private JournalWriter journalWriter;
     private TocWriter tocWriter;
     private int numEventsAtEndOfLastBlock = 0;
     private volatile long maxEventId = -1L;
     private volatile Long earliestEventTime = null;
+
+    private final Lock lock = new ReentrantLock();
+    private boolean writable = true;    // guarded by lock
+    private final List<File> timeOrderedJournalFiles = Collections.synchronizedList(new ArrayList<File>());
+    private final AtomicLong partitionSize = new AtomicLong(0L);
     
-    private final IndexManager indexManager;
-    
-    public JournalingPartition(final IndexManager indexManager, final String containerName, final int sectionIndex, final File sectionDir, final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
+    public JournalingPartition(final IndexManager indexManager, final String containerName, final int sectionIndex, final File sectionDir, 
+            final JournalingRepositoryConfig config, final AtomicLong containerSize, final ExecutorService compressionExecutor) throws IOException {
         this.indexManager = indexManager;
+        this.containerSize = containerSize;
         this.containerName = containerName;
         this.sectionIndex = sectionIndex;
         this.section = sectionDir;
         this.journalsDir = new File(section, "journals");
         this.config = config;
-        this.executor = executor;
+        this.executor = compressionExecutor;
         
         if (!journalsDir.exists() && !journalsDir.mkdirs()) {
             throw new IOException("Could not create directory " + section);
@@ -119,18 +124,24 @@ public class JournalingPartition implements Partition {
     
     @Override
     public List<JournaledProvenanceEvent> registerEvents(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
-        writeLock.lock();
+        if ( events.isEmpty() ) {
+            return Collections.emptyList();
+        }
+        
+        lock.lock();
         try {
+            if ( !writable ) {
+                throw new IOException("Cannot write to partition " + this + " because there was previously a write failure. The partition will fix itself in time if I/O problems are resolved");
+            }
+            
             final JournalWriter writer = getJournalWriter(firstEventId);
     
-            if ( !events.isEmpty() ) {
-                final int eventsWritten = writer.getEventCount();
-                if ( eventsWritten - numEventsAtEndOfLastBlock > config.getBlockSize() ) {
-                    writer.finishBlock();
-                    tocWriter.addBlockOffset(writer.getSize());
-                    numEventsAtEndOfLastBlock = eventsWritten;
-                    writer.beginNewBlock();
-                }
+            final int eventsWritten = writer.getEventCount();
+            if ( eventsWritten - numEventsAtEndOfLastBlock > config.getBlockSize() ) {
+                writer.finishBlock();
+                tocWriter.addBlockOffset(writer.getSize());
+                numEventsAtEndOfLastBlock = eventsWritten;
+                writer.beginNewBlock();
             }
     
             writer.write(events, firstEventId);
@@ -139,7 +150,7 @@ public class JournalingPartition implements Partition {
             long id = firstEventId;
             for (final ProvenanceEventRecord event : events) {
                 final JournaledStorageLocation location = new JournaledStorageLocation(containerName, String.valueOf(sectionIndex), 
-                        String.valueOf(writer.getJournalId()), tocWriter.getCurrentBlockIndex(), id++);
+                        writer.getJournalId(), tocWriter.getCurrentBlockIndex(), id++);
                 final JournaledProvenanceEvent storedEvent = new JournaledProvenanceEvent(event, location);
                 storedEvents.add(storedEvent);
             }
@@ -169,8 +180,11 @@ public class JournalingPartition implements Partition {
             }
             
             return storedEvents;
+        } catch (final IOException ioe) {
+            writable = false;
+            throw ioe;
         } finally {
-            writeLock.unlock();
+            lock.unlock();
         }
     }
 
@@ -194,43 +208,66 @@ public class JournalingPartition implements Partition {
         return false;
     }
     
+    private void updateSize(final long delta) {
+        partitionSize.addAndGet(delta);
+        containerSize.addAndGet(delta);
+    }
+    
     // MUST be called with write lock held.
+    /**
+     * Rolls over the current journal (if any) and begins writing top a new journal.
+     * 
+     * <p>
+     * <b>NOTE:</b> This method MUST be called with the write lock held!!
+     * </p>
+     * 
+     * @param firstEventId the ID of the first event to add to this journal
+     * @throws IOException
+     */
     private void rollover(final long firstEventId) throws IOException {
-        // TODO: Rework how rollover works because we now have index manager!!
-        
         // if we have a writer already, close it and initiate rollover actions
+        final File finishedFile = journalWriter == null ? null : journalWriter.getJournalFile();
         if ( journalWriter != null ) {
             journalWriter.finishBlock();
             journalWriter.close();
             tocWriter.close();
 
-            final EventIndexWriter curWriter = getIndexWriter();
+            final File finishedTocFile = tocWriter.getFile();
+            updateSize(finishedFile.length());
+            
             executor.submit(new Runnable() {
                 @Override
                 public void run() {
-                    try {
-                        curWriter.sync();
-                    } catch (final IOException e) {
-                        
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
+                    if ( config.isCompressOnRollover() ) {
+                        final long originalSize = finishedFile.length();
+                        final long compressedFileSize = new CompressionTask(finishedFile, journalWriter.getJournalId(), finishedTocFile).call();
+                        final long sizeAdded = compressedFileSize - originalSize;
+                        updateSize(sizeAdded);
                     }
                 }
             });
             
-            if ( config.isCompressOnRollover() ) {
-                final File finishedFile = journalWriter.getJournalFile();
-                final File finishedTocFile = tocWriter.getFile();
-                executor.submit(new CompressionTask(finishedFile, journalWriter.getJournalId(), finishedTocFile));
-            }
+            timeOrderedJournalFiles.add(finishedFile);
         }
         
         // create new writers and reset state.
         final File journalFile = new File(journalsDir, firstEventId + JOURNAL_FILE_EXTENSION);
         journalWriter = new StandardJournalWriter(firstEventId, journalFile, false, new StandardEventSerializer());
-        tocWriter = new StandardTocWriter(QueryUtils.getTocFile(journalFile), false, config.isAlwaysSync());
-        tocWriter.addBlockOffset(journalWriter.getSize());
-        numEventsAtEndOfLastBlock = 0;
+        try {
+            tocWriter = new StandardTocWriter(QueryUtils.getTocFile(journalFile), false, config.isAlwaysSync());
+            tocWriter.addBlockOffset(journalWriter.getSize());
+            numEventsAtEndOfLastBlock = 0;
+        } catch (final Exception e) {
+            try {
+                journalWriter.close();
+            } catch (final IOException ioe) {}
+            
+            journalWriter = null;
+            
+            throw e;
+        }
+        
+        logger.debug("Rolling over {} from {} to {}", this, finishedFile, journalFile);
     }
     
 
@@ -252,33 +289,24 @@ public class JournalingPartition implements Partition {
     
     @Override
     public void restore() throws IOException {
-        writeLock.lock();
+        lock.lock();
         try {
             // delete or rename files if stopped during rollover; compress any files that haven't been compressed
             if ( !config.isReadOnly() ) {
                 final File[] children = journalsDir.listFiles();
                 if ( children != null ) {
-                    // find the latest journal.
-                    File latestJournal = null;
-                    long latestJournalId = -1L;
-                    
                     final List<File> journalFiles = new ArrayList<>();
                     
                     // find any journal files that either haven't been compressed or were partially compressed when
                     // we last shutdown and then restart compression.
                     for ( final File file : children ) {
                         final String filename = file.getName();
-                        if ( !filename.contains(JOURNAL_FILE_EXTENSION) ) {
+                        if ( !filename.endsWith(JOURNAL_FILE_EXTENSION) ) {
                             continue;
                         }
                         
-                        final Long journalId = getJournalId(file);
-                        if ( journalId != null && journalId > latestJournalId ) {
-                            latestJournal = file;
-                            latestJournalId = journalId;
-                        }
-                        
                         journalFiles.add(file);
+                        updateSize(file.length());
                         
                         if ( !config.isCompressOnRollover() ) {
                             continue;
@@ -290,7 +318,15 @@ public class JournalingPartition implements Partition {
                                 // both the compressed and uncompressed version of this journal exist. The Compression Task was
                                 // not complete when we shutdown. Delete the compressed journal and toc and re-start the Compression Task.
                                 final File tocFile = QueryUtils.getTocFile(uncompressedFile);
-                                executor.submit(new CompressionTask(uncompressedFile, getJournalId(uncompressedFile), tocFile));
+                                executor.submit(new Runnable() {
+                                    @Override
+                                    public void run() {
+                                        final long originalSize = uncompressedFile.length();
+                                        final long compressedSize = new CompressionTask(uncompressedFile, getJournalId(uncompressedFile), tocFile).call();
+                                        final long sizeAdded = compressedSize - originalSize;
+                                        updateSize(sizeAdded);
+                                    }
+                                });
                             } else {
                                 // The compressed file exists but the uncompressed file does not. This means that we have finished
                                 // writing the compressed file and deleted the original journal file but then shutdown before
@@ -321,6 +357,22 @@ public class JournalingPartition implements Partition {
                         }
                     }
                     
+                    // we want to sort the list of all journal files.
+                    // we need to create a map of file to last mod time, rather than comparing
+                    // by using File.lastModified() because the File.lastModified() value could potentially
+                    // change while running the comparator, which violates the comparator's contract.
+                    timeOrderedJournalFiles.addAll(journalFiles);
+                    final Map<File, Long> lastModTimes = new HashMap<>();
+                    for ( final File journalFile : journalFiles ) {
+                        lastModTimes.put(journalFile, journalFile.lastModified());
+                    }
+                    Collections.sort(timeOrderedJournalFiles, new Comparator<File>() {
+                        @Override
+                        public int compare(final File o1, final File o2) {
+                            return lastModTimes.get(o1).compareTo(lastModTimes.get(o2));
+                        }
+                    });
+                    
                     // Get the first event in the earliest journal file so that we know what the earliest time available is
                     Collections.sort(journalFiles, new Comparator<File>() {
                         @Override
@@ -332,61 +384,83 @@ public class JournalingPartition implements Partition {
                     for ( final File journal : journalFiles ) {
                         try (final JournalReader reader = new StandardJournalReader(journal)) {
                             final ProvenanceEventRecord record = reader.nextEvent();
-                            this.earliestEventTime = record.getEventTime();
-                            break;
+                            if ( record != null ) {
+                                this.earliestEventTime = record.getEventTime();
+                                break;
+                            }
                         } catch (final IOException ioe) {
                         }
                     }
-                    
-                    // Whatever was the last journal for this partition, we need to remove anything for that journal
-                    // from the index and re-add them, and then sync the index. This allows us to avoid syncing
-                    // the index each time (we sync only on rollover) but allows us to still ensure that we index
-                    // all events.
-                    if ( latestJournal != null ) {
+
+                    // order such that latest journal file is first.
+                    Collections.reverse(journalFiles);
+                    for ( final File journal : journalFiles ) {
+                        try (final JournalReader reader = new StandardJournalReader(journal);
+                             final TocReader tocReader = new StandardTocReader(QueryUtils.getTocFile(journal))) {
+                            
+                            final long lastBlockOffset = tocReader.getLastBlockOffset();
+                            final ProvenanceEventRecord lastEvent = reader.getLastEvent(lastBlockOffset);
+                            if ( lastEvent != null ) {
+                                maxEventId = lastEvent.getEventId() + 1;
+                                break;
+                            }
+                        } catch (final EOFException eof) {}
+                    }
+
+                    // We need to re-index all of the journal files that have not been indexed. We can do this by determining
+                    // what is the largest event id that has been indexed for this container and section, and then re-indexing
+                    // any file that has an event with an id larger than that.
+                    // In order to do that, we iterate over the journal files in the order of newest (largest id) to oldest
+                    // (smallest id). If the first event id in a file is greater than the max indexed, we re-index the file.
+                    // Beyond that, we need to re-index one additional journal file because it's possible that if the first id
+                    // is 10 and the max index id is 15, the file containing 10 could also go up to 20. So we re-index one
+                    // file that has a min id less than what has been indexed; then we are done.
+                    final Long maxIndexedId = indexManager.getMaxEventId(containerName, String.valueOf(sectionIndex));
+                    final List<File> reindexJournals = new ArrayList<>();
+                    for ( final File journalFile : journalFiles ) {
+                        final Long firstEventId;
                         try {
-                            reindex(latestJournal);
-                        } catch (final EOFException eof) {
+                            firstEventId = getJournalId(journalFile);
+                        } catch (final NumberFormatException nfe) {
+                            // not a journal; skip this file
+                            continue;
                         }
+                        
+                        if ( maxIndexedId == null || firstEventId > maxIndexedId ) {
+                            reindexJournals.add(journalFile);
+                        } else {
+                            reindexJournals.add(journalFile);
+                            break;
+                        }
+                    }
+                    
+                    // Make sure that the indexes are not pointing to events that no longer exist.
+                    if ( journalFiles.isEmpty() ) {
+                        indexManager.deleteEventsBefore(containerName, sectionIndex, Long.MAX_VALUE);
+                    } else {
+                        final File firstJournalFile = journalFiles.get(0);
+                        indexManager.deleteEventsBefore(containerName, sectionIndex, getJournalId(firstJournalFile));
+                    }
+                    
+                    // The reindexJournals list is currently in order of newest to oldest. We need to re-index
+                    // in order of oldest to newest, so reverse the list.
+                    Collections.reverse(reindexJournals);
+                    
+                    logger.info("Reindexing {} journal files that were not found in index for container {} and section {}", reindexJournals.size(), containerName, sectionIndex);
+                    final long reindexStart = System.nanoTime();
+                    for ( final File journalFile : reindexJournals ) {
+                        indexManager.reindex(containerName, sectionIndex, getJournalId(journalFile), journalFile);
                     }
+                    final long reindexMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - reindexStart);
+                    logger.info("Finished reindexing {} journal files for container {} and section {}; reindex took {} millis", 
+                            reindexJournals.size(), containerName, sectionIndex, reindexMillis);
                 }
             }
         } finally {
-            writeLock.unlock();
+            lock.unlock();
         }
     }
 
-    
-    private void reindex(final File journalFile) throws IOException {
-        // TODO: Rework how recovery works because we now have index manager!!
-        try (final TocJournalReader reader = new TocJournalReader(containerName, String.valueOf(sectionIndex), String.valueOf(getJournalId(journalFile)), journalFile)) {
-            // We don't know which index contains the data for this journal, so remove the journal
-            // from both.
-            for (final LuceneIndexWriter indexWriter : indexWriters ) {
-                indexWriter.delete(containerName, String.valueOf(sectionIndex), String.valueOf(getJournalId(journalFile)));
-            }
-            
-            long maxId = -1L;
-            final List<JournaledProvenanceEvent> storedEvents = new ArrayList<>(1000);
-            JournaledProvenanceEvent event;
-            final LuceneIndexWriter indexWriter = indexWriters[0];
-            while ((event = reader.nextJournaledEvent()) != null ) {
-                storedEvents.add(event);
-                maxId = event.getEventId();
-                
-                if ( storedEvents.size() == 1000 ) {
-                    indexWriter.index(storedEvents);
-                    storedEvents.clear();
-                }
-            }
-
-            if ( !storedEvents.isEmpty() ) {
-                indexWriter.index(storedEvents);
-            }
-            
-            indexWriter.sync();
-            this.maxEventId = maxId;
-        }
-    }
 
     
     @Override
@@ -443,4 +517,129 @@ public class JournalingPartition implements Partition {
     public String toString() {
         return "Partition[section=" + sectionIndex + "]";
     }
+    
+    @Override
+    public void verifyWritable(final long nextId) throws IOException {
+        final long freeSpace = section.getFreeSpace();
+        final long freeMegs = freeSpace / 1024 / 1024;
+        if (freeMegs < 10) {
+            // if not at least 10 MB, don't even try to write
+            throw new IOException("Not Enough Disk Space: partition housing " + section + " has only " + freeMegs + " MB of storage available");
+        }
+        
+        rollover(nextId);
+        writable = true;
+    }
+    
+    private boolean delete(final File journalFile) {
+        for (int i=0; i < 10; i++) {
+            if ( journalFile.delete() || !journalFile.exists() ) {
+                return true;
+            } else {
+                try {
+                    Thread.sleep(100L);
+                } catch (final InterruptedException ie) {}
+            }
+        }
+        
+        return false;
+    }
+    
+    @Override
+    public void deleteOldEvents(final long earliestEventTimeToDelete) throws IOException {
+        final Set<File> removeFromTimeOrdered = new HashSet<>();
+        
+        final long start = System.nanoTime();
+        try {
+            for ( final File journalFile : timeOrderedJournalFiles ) {
+                // since these are time-ordered, if we find one that we don't want to delete, we're done.
+                if ( journalFile.lastModified() < earliestEventTimeToDelete ) {
+                    return;
+                }
+                
+                final long journalSize;
+                if ( journalFile.exists() ) {
+                    journalSize = journalFile.length();
+                } else {
+                    continue;
+                }
+                
+                if ( delete(journalFile) ) {
+                    removeFromTimeOrdered.add(journalFile);
+                } else {
+                    logger.warn("Failed to remove expired journal file {}; will attempt to delete again later", journalFile);
+                }
+                
+                updateSize(-journalSize);
+                final File tocFile = QueryUtils.getTocFile(journalFile);
+                if ( !delete(tocFile) ) {
+                    logger.warn("Failed to remove TOC file for expired journal file {}; will attempt to delete again later", journalFile);
+                }
+            }
+        } finally {
+            timeOrderedJournalFiles.removeAll(removeFromTimeOrdered);
+        }
+        
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+        logger.info("Removed {} expired journal files from container {}, section {}; total time for deletion was {} millis", 
+                removeFromTimeOrdered.size(), containerName, sectionIndex, millis);
+    }
+    
+    
+    @Override
+    public void deleteOldest() throws IOException {
+        File removeFromTimeOrdered = null;
+        
+        final long start = System.nanoTime();
+        try {
+            for ( final File journalFile : timeOrderedJournalFiles ) {
+                final long journalSize;
+                if ( journalFile.exists() ) {
+                    journalSize = journalFile.length();
+                } else {
+                    continue;
+                }
+                
+                if ( delete(journalFile) ) {
+                    removeFromTimeOrdered = journalFile;
+                } else {
+                    throw new IOException("Cannot delete oldest event file " + journalFile);
+                }
+                
+                final File tocFile = QueryUtils.getTocFile(journalFile);
+                if ( !delete(tocFile) ) {
+                    logger.warn("Failed to remove TOC file for expired journal file {}; will attempt to delete again later", journalFile);
+                }
+                
+                updateSize(-journalSize);
+                indexManager.deleteEvents(containerName, sectionIndex, getJournalId(journalFile));
+            }
+        } finally {
+            if ( removeFromTimeOrdered != null ) {
+                timeOrderedJournalFiles.remove(removeFromTimeOrdered);
+                
+                final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+                logger.info("Removed oldest event file {} from container {}, section {}; total time for deletion was {} millis", 
+                        removeFromTimeOrdered, containerName, sectionIndex, millis);
+            } else {
+                logger.debug("No journals to remove for {}", this);
+            }
+        }
+    }
+    
+    
+    @Override
+    public long getPartitionSize() {
+        return partitionSize.get();
+    }
+    
+    @Override
+    public long getContainerSize() {
+        return containerSize.get();
+    }
+    
+    @Override
+    public String getContainerName() {
+        return containerName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
index e77c8d5..9efae04 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/Partition.java
@@ -66,6 +66,12 @@ public interface Partition {
     long getMaxEventId();
     
     /**
+     * Returns the name of the container that this Partition operates on
+     * @return
+     */
+    String getContainerName();
+    
+    /**
      * Returns the locations of events that have an id at least equal to minEventId, returning the events
      * with the smallest ID's possible that are greater than minEventId
      *
@@ -82,4 +88,39 @@ public interface Partition {
      * @throws IOException
      */
     Long getEarliestEventTime() throws IOException;
+    
+    /**
+     * Verifies that the partition is able to be written to. A Partition may need to create a new journal
+     * in order to verify. In this case, the nextId is provided so that the Partition knows the minimum event id
+     * that will be written to the partition
+     * 
+     * @throws IOException
+     */
+    void verifyWritable(long nextId) throws IOException;
+    
+    
+    /**
+     * Deletes any journal for this partition that occurred before the given time
+     * @param earliestEventTimeToDelete
+     * @throws IOException
+     */
+    void deleteOldEvents(long earliestEventTimeToDelete) throws IOException;
+
+    /**
+     * Returns the size of this partition in bytes
+     * @return
+     */
+    long getPartitionSize();
+    
+    /**
+     * Returns the size of the journals in the entire container
+     * @return
+     */
+    long getContainerSize();
+    
+    /**
+     * Deletes the oldest journal from this partition
+     * @throws IOException
+     */
+    void deleteOldest() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
index c0a56c4..14d5b17 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
@@ -52,10 +52,23 @@ public interface PartitionManager {
      * @param writeAction specifies whether or not the action writes to the repository
      * @return
      */
-    <T> Set<T> withEachPartition(PartitionAction<T> action) throws IOException;
+//    <T> Set<T> withEachPartition(PartitionAction<T> action, boolean writeAction) throws IOException;
     
     /**
-     * Performs the given Action on each partition and returns the set of results. Unlike
+     * Performs the given Action on each partition and returns the set of results. This method does 
+     * not use the thread pool in order to perform the request in parallel. This is desirable for 
+     * very quick functions, as the thread pool can be fully utilized, resulting in a quick function 
+     * taking far longer than it should.
+     * 
+     * @param action the action to perform
+     * @param writeAction specifies whether or not the action writes to the repository
+     * @return
+     */
+    <T> Set<T> withEachPartitionSerially(PartitionAction<T> action, boolean writeAction) throws IOException;
+    
+    
+    /**
+     * Performs the given Action on each partition. Unlike
      * {@link #withEachPartition(PartitionAction))}, this method does not use the thread pool
      * in order to perform the request in parallel. This is desirable for very quick functions,
      * as the thread pool can be fully utilized, resulting in a quick function taking far longer
@@ -65,7 +78,7 @@ public interface PartitionManager {
      * @param writeAction specifies whether or not the action writes to the repository
      * @return
      */
-    <T> Set<T> withEachPartitionSerially(PartitionAction<T> action) throws IOException;
+    void withEachPartitionSerially(VoidPartitionAction action, boolean writeAction) throws IOException;
     
     /**
      * Performs the given Action to each partition, optionally waiting for the action to complete
@@ -74,7 +87,13 @@ public interface PartitionManager {
      * @param async if <code>true</code>, will perform the action asynchronously; if <code>false</code>, will
      *  wait for the action to complete before returning
      */
-    void withEachPartition(VoidPartitionAction action, boolean async);
+//    void withEachPartition(VoidPartitionAction action, boolean async);
     
     void shutdown();
+    
+    /**
+     * Triggers the Partition Manager to delete events from journals and indices based on the sizes of the containers
+     * and overall size of the repository
+     */
+    void deleteEventsBasedOnSize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
index 10af697..9b5c442 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
@@ -19,19 +19,18 @@ package org.apache.nifi.provenance.journaling.partition;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
 import org.apache.nifi.provenance.journaling.index.IndexManager;
@@ -43,16 +42,26 @@ public class QueuingPartitionManager implements PartitionManager {
     
     private static final Logger logger = LoggerFactory.getLogger(QueuingPartitionManager.class);
     
+    private final IndexManager indexManager;
     private final JournalingRepositoryConfig config;
     private final BlockingQueue<Partition> partitionQueue;
     private final JournalingPartition[] partitionArray;
-    private final ExecutorService executor;
+    private final AtomicLong eventIdGenerator;
     private volatile boolean shutdown = false;
     
-    private final AtomicInteger blacklistedCount = new AtomicInteger(0);
+    private final Set<Partition> blackListedPartitions = Collections.synchronizedSet(new HashSet<Partition>());
     
-    public QueuingPartitionManager(final IndexManager indexManager, final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
+    public QueuingPartitionManager(final IndexManager indexManager, final AtomicLong eventIdGenerator, final JournalingRepositoryConfig config, final ScheduledExecutorService workerExecutor, final ExecutorService compressionExecutor) throws IOException {
+        this.indexManager = indexManager;
         this.config = config;
+        this.eventIdGenerator = eventIdGenerator;
+        
+        // We can consider using a PriorityQueue here instead. Keep track of how many Partitions are being written
+        // to for each container, as a container usually maps to a physical drive. Then, prioritize the queue
+        // so that the partitions that belong to Container A get a higher priority than those belonging to Container B
+        // if there are currently more partitions on Container B being written to (i.e., we prefer a partition for the
+        // container that is the least used at this moment). Would require significant performance testing to see if it
+        // really provides any benefit.
         this.partitionQueue = new LinkedBlockingQueue<>(config.getPartitionCount());
         this.partitionArray = new JournalingPartition[config.getPartitionCount()];
         
@@ -61,16 +70,24 @@ public class QueuingPartitionManager implements PartitionManager {
             containerTuples.add(new Tuple<>(entry.getKey(), entry.getValue()));
         }
         
+        final Map<String, AtomicLong> containerSizes = new HashMap<>();
+        for ( final String containerName : config.getContainers().keySet() ) {
+            containerSizes.put(containerName, new AtomicLong(0L));
+        }
+        
         for (int i=0; i < config.getPartitionCount(); i++) {
             final Tuple<String, File> tuple = containerTuples.get(i % containerTuples.size());
             final File section = new File(tuple.getValue(), String.valueOf(i));
             
-            final JournalingPartition partition = new JournalingPartition(indexManager, tuple.getKey(), i, section, config, executor);
+            final String containerName = tuple.getKey();
+            final JournalingPartition partition = new JournalingPartition(indexManager, containerName, i, 
+                    section, config, containerSizes.get(containerName), compressionExecutor);
+            partition.restore();
             partitionQueue.offer(partition);
             partitionArray[i] = partition;
         }
         
-        this.executor = executor;
+        workerExecutor.scheduleWithFixedDelay(new CheckBlackListedPartitions(), 30, 30, TimeUnit.SECONDS);
     }
     
     @Override
@@ -82,32 +99,63 @@ public class QueuingPartitionManager implements PartitionManager {
         }
     }
     
-    private Partition nextPartition() {
+    private Partition nextPartition(final boolean writeAction) {
         Partition partition = null;
         
-        while(partition == null) {
-            if (shutdown) {
-                throw new RuntimeException("Journaling Provenance Repository is shutting down");
-            }
-            
-            try {
-                partition = partitionQueue.poll(1, TimeUnit.SECONDS);
-            } catch (final InterruptedException ie) {
-            }
-            
-            if ( partition == null ) {
-                if ( blacklistedCount.get() >= config.getPartitionCount() ) {
-                    throw new RuntimeException("Cannot persist to the Journal Provenance Repository because all partitions have been blacklisted due to write failures");
+        final List<Partition> partitionsSkipped = new ArrayList<>();
+        try {
+            while (partition == null) {
+                if (shutdown) {
+                    throw new RuntimeException("Journaling Provenance Repository is shutting down");
+                }
+                
+                try {
+                    partition = partitionQueue.poll(1, TimeUnit.SECONDS);
+                } catch (final InterruptedException ie) {
+                }
+                
+                if ( partition == null ) {
+                    if ( blackListedPartitions.size() >= config.getPartitionCount() ) {
+                        throw new RuntimeException("Cannot persist to the Journal Provenance Repository because all partitions have been blacklisted due to write failures");
+                    }
+                    
+                    // we are out of partitions. Add back all of the partitions that we skipped so we 
+                    // can try them again.
+                    partitionQueue.addAll(partitionsSkipped);
+                    partitionsSkipped.clear();
+                } else if (writeAction) {
+                    // determine if the container is full.
+                    final String containerName = partition.getContainerName();
+                    long desiredMaxContainerCapacity = config.getMaxCapacity(containerName);
+                    
+                    // If no max capacity set for the container itself, use 1/N of repo max
+                    // where N is the number of containers
+                    if ( desiredMaxContainerCapacity == config.getMaxStorageCapacity() ) {
+                        desiredMaxContainerCapacity = config.getMaxStorageCapacity() / config.getContainers().size();
+                    }
+                    
+                    // if the partition is more than 10% over its desired capacity, we don't want to write to it.
+                    if ( partition.getContainerSize() > 1.1 * desiredMaxContainerCapacity ) {
+                        partitionsSkipped.add(partition);
+                        continue;
+                    }
                 }
             }
+        } finally {
+            partitionQueue.addAll( partitionsSkipped );
         }
         
         return partition;
     }
     
+    
+    private void blackList(final Partition partition) {
+        blackListedPartitions.add(partition);
+    }
+    
     @Override
     public <T> T withPartition(final PartitionAction<T> action, final boolean writeAction) throws IOException {
-        final Partition partition = nextPartition();
+        final Partition partition = nextPartition(writeAction);
 
         boolean ioe = false;
         try {
@@ -117,8 +165,7 @@ public class QueuingPartitionManager implements PartitionManager {
             throw e;
         } finally {
             if ( ioe && writeAction ) {
-                // We failed to write to this Partition. This partition will no longer be usable until NiFi is restarted!
-                blacklistedCount.incrementAndGet();
+                blackList(partition);
             } else {
                 partitionQueue.offer(partition);
             }
@@ -127,7 +174,7 @@ public class QueuingPartitionManager implements PartitionManager {
     
     @Override
     public void withPartition(final VoidPartitionAction action, final boolean writeAction) throws IOException {
-        final Partition partition = nextPartition();
+        final Partition partition = nextPartition(writeAction);
 
         boolean ioe = false;
         try {
@@ -137,8 +184,7 @@ public class QueuingPartitionManager implements PartitionManager {
             throw e;
         } finally {
             if ( ioe && writeAction ) {
-                // We failed to write to this Partition. This partition will no longer be usable until NiFi is restarted!
-                blacklistedCount.incrementAndGet();
+                blackList(partition);
             } else {
                 partitionQueue.offer(partition);
             }
@@ -146,89 +192,234 @@ public class QueuingPartitionManager implements PartitionManager {
     }
 
     
+//    @Override
+//    public <T> Set<T> withEachPartition(final PartitionAction<T> action) throws IOException {
+//        if ( writeAction && blackListedPartitions.size() > 0 ) {
+//            throw new IOException("Cannot perform action {} because at least one partition has been blacklisted (i.e., writint to the partition failed)");
+//        }
+//
+//        final Set<T> results = new HashSet<>(partitionArray.length);
+//        
+//        final Map<Partition, Future<T>> futures = new HashMap<>(partitionArray.length);
+//        for ( final Partition partition : partitionArray ) {
+//            final Callable<T> callable = new Callable<T>() {
+//                @Override
+//                public T call() throws Exception {
+//                    return action.perform(partition);
+//                }
+//            };
+//            
+//            final Future<T> future = executor.submit(callable);
+//            futures.put(partition, future);
+//        }
+//        
+//        for ( final Map.Entry<Partition, Future<T>> entry : futures.entrySet() ) {
+//            try {
+//                final T result = entry.getValue().get();
+//                results.add(result);
+//            } catch (final ExecutionException ee) {
+//                final Throwable cause = ee.getCause();
+//                if ( cause instanceof IOException ) {
+//                    throw (IOException) cause;
+//                } else {
+//                    throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
+//                }
+//            } catch (InterruptedException e) {
+//                throw new RuntimeException(e);
+//            }
+//        }
+//        
+//        return results;
+//    }
+    
     @Override
-    public <T> Set<T> withEachPartition(final PartitionAction<T> action) throws IOException {
-        final Set<T> results = new HashSet<>(partitionArray.length);
-        
-        // TODO: Do not use blacklisted partitions.
-        final Map<Partition, Future<T>> futures = new HashMap<>(partitionArray.length);
-        for ( final Partition partition : partitionArray ) {
-            final Callable<T> callable = new Callable<T>() {
-                @Override
-                public T call() throws Exception {
-                    return action.perform(partition);
-                }
-            };
-            
-            final Future<T> future = executor.submit(callable);
-            futures.put(partition, future);
+    public <T> Set<T> withEachPartitionSerially(final PartitionAction<T> action, final boolean writeAction) throws IOException {
+        if ( writeAction && blackListedPartitions.size() > 0 ) {
+            throw new IOException("Cannot perform action {} because at least one partition has been blacklisted (i.e., writint to the partition failed)");
         }
         
-        for ( final Map.Entry<Partition, Future<T>> entry : futures.entrySet() ) {
-            try {
-                final T result = entry.getValue().get();
-                results.add(result);
-            } catch (final ExecutionException ee) {
-                final Throwable cause = ee.getCause();
-                if ( cause instanceof IOException ) {
-                    throw (IOException) cause;
-                } else {
-                    throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
-                }
-            } catch (InterruptedException e) {
-                throw new RuntimeException(e);
-            }
+        final Set<T> results = new HashSet<>(partitionArray.length);
+        for ( final Partition partition : partitionArray ) {
+            results.add( action.perform(partition) );
         }
         
         return results;
     }
     
     @Override
-    public <T> Set<T> withEachPartitionSerially(final PartitionAction<T> action) throws IOException {
-        // TODO: Do not use blacklisted partitions.
-        final Set<T> results = new HashSet<>(partitionArray.length);
+    public void withEachPartitionSerially(final VoidPartitionAction action, final boolean writeAction) throws IOException {
+        if ( writeAction && blackListedPartitions.size() > 0 ) {
+            throw new IOException("Cannot perform action {} because at least one partition has been blacklisted (i.e., writint to the partition failed)");
+        }
+        
         for ( final Partition partition : partitionArray ) {
-            results.add( action.perform(partition) );
+            action.perform(partition);
+        }
+    }
+    
+//    @Override
+//    public void withEachPartition(final VoidPartitionAction action, final boolean async) {
+//        // TODO: skip blacklisted partitions
+//        final Map<Partition, Future<?>> futures = new HashMap<>(partitionArray.length);
+//        for ( final Partition partition : partitionArray ) {
+//            final Runnable runnable = new Runnable() {
+//                @Override
+//                public void run() {
+//                    try {
+//                        action.perform(partition);
+//                    } catch (final Throwable t) {
+//                        logger.error("Failed to perform action against " + partition + " due to " + t);
+//                        if ( logger.isDebugEnabled() ) {
+//                            logger.error("", t);
+//                        }
+//                    }
+//                }
+//            };
+//            
+//            final Future<?> future = executor.submit(runnable);
+//            futures.put(partition, future);
+//        }
+//        
+//        if ( !async ) {
+//            for ( final Map.Entry<Partition, Future<?>> entry : futures.entrySet() ) {
+//                try {
+//                    // throw any exception thrown by runnable
+//                    entry.getValue().get();
+//                } catch (final ExecutionException ee) {
+//                    final Throwable cause = ee.getCause();
+//                    throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
+//                } catch (InterruptedException e) {
+//                    throw new RuntimeException(e);
+//                }
+//            }
+//        }
+//    }
+    
+    private long getTotalSize() {
+        long totalSize = 0L;
+
+        for ( final JournalingPartition partition : partitionArray ) {
+            totalSize += partition.getPartitionSize();
         }
         
-        return results;
+        for ( final String containerName : config.getContainers().keySet() ) {
+            totalSize += indexManager.getSize(containerName);
+        }
+        
+        return totalSize;
+    }
+    
+    
+    /**
+     * Responsible for looking at partitions that have been marked as blacklisted and checking if they
+     * are able to be written to now. If so, adds them back to the partition queue; otherwise, leaves
+     * them as blacklisted
+     */
+    private class CheckBlackListedPartitions implements Runnable {
+        @Override
+        public void run() {
+            final Set<Partition> reclaimed = new HashSet<>();
+            
+            final Set<Partition> partitions = new HashSet<>(blackListedPartitions);
+            for ( final Partition partition : partitions ) {
+                final long nextId = eventIdGenerator.get();
+                if ( nextId <= 0 ) {
+                    // we don't have an ID to use yet. Don't attempt to do anything yet.
+                    return;
+                }
+                
+                try {
+                    partition.verifyWritable(nextId);
+                    reclaimed.add(partition);
+                 } catch (final IOException ioe) {
+                     logger.debug("{} is still blackListed due to {}", partition, ioe);
+                 }
+            }
+            
+            // any partition that is reclaimable is now removed from the set of blacklisted
+            // partitions and added back to our queue of partitions
+            blackListedPartitions.removeAll(reclaimed);
+            partitionQueue.addAll(reclaimed);
+        }
     }
     
+    
     @Override
-    public void withEachPartition(final VoidPartitionAction action, final boolean async) {
-        // TODO: Do not use blacklisted partitions.
-        final Map<Partition, Future<?>> futures = new HashMap<>(partitionArray.length);
-        for ( final Partition partition : partitionArray ) {
-            final Runnable runnable = new Runnable() {
-                @Override
-                public void run() {
+    public void deleteEventsBasedOnSize() {
+        final Map<String, List<JournalingPartition>> containerPartitionMap = new HashMap<>();
+        
+        for ( final JournalingPartition partition : partitionArray ) {
+            final String container = partition.getContainerName();
+            List<JournalingPartition> list = containerPartitionMap.get(container);
+            if ( list == null ) {
+                list = new ArrayList<>();
+                containerPartitionMap.put(container, list);
+            }
+            
+            list.add(partition);
+        }
+        
+        int iterations = 0;
+        for ( final String containerName : config.getContainers().keySet() ) {
+            // continue as long as we need to delete data from this container.
+            while (true) {
+                // don't hammer the disks if we can't delete anything
+                if ( iterations++ > 0 ) {
+                    try {
+                        Thread.sleep(1000L);
+                    } catch (final InterruptedException ie) {}
+                }
+                
+                final List<JournalingPartition> containerPartitions = containerPartitionMap.get(containerName);
+                final long containerSize = containerPartitions.get(0).getContainerSize();
+                final long maxContainerCapacity = config.getMaxCapacity(containerName);
+                if ( containerSize < maxContainerCapacity ) {
+                    break;
+                }
+                
+                logger.debug("Container {} exceeds max capacity of {} bytes with a size of {} bytes; deleting oldest events", containerName, maxContainerCapacity, containerSize);
+                
+                // container is too large. Delete oldest journal from each partition in this container.
+                for ( final Partition partition : containerPartitions ) {
                     try {
-                        action.perform(partition);
-                    } catch (final Throwable t) {
-                        logger.error("Failed to perform action against " + partition + " due to " + t);
+                        partition.deleteOldest();
+                    } catch (final IOException ioe) {
+                        logger.error("Failed to delete events from {} due to {}", partition, ioe.toString());
                         if ( logger.isDebugEnabled() ) {
-                            logger.error("", t);
+                            logger.error("", ioe);
                         }
                     }
                 }
-            };
-            
-            final Future<?> future = executor.submit(runnable);
-            futures.put(partition, future);
+            }
         }
         
-        if ( !async ) {
-            for ( final Map.Entry<Partition, Future<?>> entry : futures.entrySet() ) {
+        long totalSize;
+        iterations = 0;
+        while ((totalSize = getTotalSize()) >= config.getMaxStorageCapacity()) {
+            logger.debug("Provenance Repository exceeds max capacity of {} bytes with a size of {}; deleting oldest events", config.getMaxStorageCapacity(), totalSize);
+            
+            // don't hammer the disks if we can't delete anything
+            if ( iterations++ > 0 ) {
                 try {
-                    // throw any exception thrown by runnable
-                    entry.getValue().get();
-                } catch (final ExecutionException ee) {
-                    final Throwable cause = ee.getCause();
-                    throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
-                } catch (InterruptedException e) {
-                    throw new RuntimeException(e);
+                    Thread.sleep(1000L);
+                } catch (final InterruptedException ie) {}
+            }
+
+            for ( final Partition partition : partitionArray ) {
+                try {
+                    partition.deleteOldest();
+                } catch (final IOException ioe) {
+                    logger.error("Failed to delete events from {} due to {}", partition, ioe.toString());
+                    if ( logger.isDebugEnabled() ) {
+                        logger.error("", ioe);
+                    }
                 }
             }
+            
+            // don't hammer the disks if we can't delete anything
+            try {
+                Thread.sleep(1000L);
+            } catch (final InterruptedException ie) {}
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/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 4edc6ad..c5516aa 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
@@ -16,10 +16,14 @@
  */
 package org.apache.nifi.provenance.journaling.query;
 
+import java.io.Closeable;
+
+import org.apache.nifi.provenance.ProvenanceEventRepository;
+import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
 import org.apache.nifi.provenance.search.Query;
 import org.apache.nifi.provenance.search.QuerySubmission;
 
-public interface QueryManager {
+public interface QueryManager extends Closeable {
     /**
      * Submits an asynchronous request to process the given query, returning an
      * identifier that can be used to fetch the results at a later time
@@ -39,4 +43,51 @@ public interface QueryManager {
      * @return
      */
     QuerySubmission retrieveQuerySubmission(String queryIdentifier);
+    
+    /**
+     * Returns the {@link ComputeLineageSubmission} associated with the given
+     * identifier
+     *
+     * @param lineageIdentifier
+     * @return
+     */
+    ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier);
+    
+    /**
+     * Submits a Lineage Computation to be completed and returns the
+     * AsynchronousLineageResult that indicates the status of the request and
+     * the results, if the computation is complete.
+     *
+     * @param flowFileUuid the UUID of the FlowFile for which the Lineage should
+     * be calculated
+     * @return a {@link ComputeLineageSubmission} object that can be used to
+     * check if the computing is complete and if so get the results
+     */
+    ComputeLineageSubmission submitLineageComputation(String flowFileUuid);
+    
+    /**
+     * Submits a request to expand the parents of the event with the given id
+     *
+     * @param eventRepo the repository from which to obtain the event information
+     * @param eventId the one-up id of the Event to expand
+     * @return
+     *
+     * @throws IllegalArgumentException if the given identifier identifies a
+     * Provenance Event that has a Type that is not expandable or if the
+     * identifier cannot be found
+     */
+    ComputeLineageSubmission submitExpandParents(final ProvenanceEventRepository eventRepo, long eventId);
+
+    /**
+     * Submits a request to expand the children of the event with the given id
+     *
+     * @param eventRepo the repository from which to obtain the event information
+     * @param eventId the one-up id of the Event
+     * @return
+     *
+     * @throws IllegalArgumentException if the given identifier identifies a
+     * Provenance Event that has a Type that is not expandable or if the
+     * identifier cannot be found
+     */
+    ComputeLineageSubmission submitExpandChildren(final ProvenanceEventRepository eventRepo, long eventId);
 }


[11/12] incubator-nifi git commit: NIFI-388: Initial implementation of prov repo; not yet finished but pushing so that the code is not lost

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.class
new file mode 100644
index 0000000..d7c093f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO$Formatter.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO$Formatter.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO$Formatter.class
new file mode 100644
index 0000000..ea2ea48
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO$Formatter.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.class
new file mode 100644
index 0000000..51fc897
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.class
new file mode 100644
index 0000000..edebe8e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.class
new file mode 100644
index 0000000..e25a3f2
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.class
new file mode 100644
index 0000000..354d6d3
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.class
new file mode 100644
index 0000000..241625a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.class
new file mode 100644
index 0000000..398bb9b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.class
new file mode 100644
index 0000000..5a39b6f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.class
new file mode 100644
index 0000000..990dcaf
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.class
new file mode 100644
index 0000000..3ca7f1e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.class
new file mode 100644
index 0000000..6411749
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.class
new file mode 100644
index 0000000..6422776
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.class
new file mode 100644
index 0000000..9632ba5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.class
new file mode 100644
index 0000000..a007034
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.class
new file mode 100644
index 0000000..81f90f8
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.class
new file mode 100644
index 0000000..0191736
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.class
new file mode 100644
index 0000000..66278be
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.class
new file mode 100644
index 0000000..717bd5d
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.class
new file mode 100644
index 0000000..826b456
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.class
new file mode 100644
index 0000000..fd8b930
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.class
new file mode 100644
index 0000000..d5b2b04
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.class
new file mode 100644
index 0000000..217ab8b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.class
new file mode 100644
index 0000000..88dc8e5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.class
new file mode 100644
index 0000000..e6d4a4d
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.class
new file mode 100644
index 0000000..4b7c464
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.class
new file mode 100644
index 0000000..a2c1774
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.class
new file mode 100644
index 0000000..78348ca
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.class
new file mode 100644
index 0000000..00cd50a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/Entity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/Entity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/Entity.class
new file mode 100644
index 0000000..e0786bc
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/Entity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.class
new file mode 100644
index 0000000..64426df
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.class
new file mode 100644
index 0000000..c02c5d4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.class
new file mode 100644
index 0000000..3558037
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.class
new file mode 100644
index 0000000..e64fe58
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.class
new file mode 100644
index 0000000..89779b7
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.class
new file mode 100644
index 0000000..a4a536f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.class
new file mode 100644
index 0000000..567191d
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.class
new file mode 100644
index 0000000..0bbf4d5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.class
new file mode 100644
index 0000000..6f6a8e4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.class
new file mode 100644
index 0000000..af254bd
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.class
new file mode 100644
index 0000000..3cd8f78
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.class
new file mode 100644
index 0000000..a72f158
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.class
new file mode 100644
index 0000000..eb7872c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.class
new file mode 100644
index 0000000..2459f06
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.class
new file mode 100644
index 0000000..d2bbd17
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.class
new file mode 100644
index 0000000..ea1bf33
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.class
new file mode 100644
index 0000000..6e6d3f6
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.class
new file mode 100644
index 0000000..8cc8f3e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.class
new file mode 100644
index 0000000..7040fc5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.class
new file mode 100644
index 0000000..fda40d9
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.class
new file mode 100644
index 0000000..2ccda7e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.class
new file mode 100644
index 0000000..e8e12e3
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.class
new file mode 100644
index 0000000..cec6043
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.class
new file mode 100644
index 0000000..3fefd86
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.class
new file mode 100644
index 0000000..e7f7a26
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.class
new file mode 100644
index 0000000..fa6a8fc
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.class
new file mode 100644
index 0000000..da9489c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.class
new file mode 100644
index 0000000..bcd987c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.class
new file mode 100644
index 0000000..337b7d6
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.class
new file mode 100644
index 0000000..5c67d8e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.class
new file mode 100644
index 0000000..71a0441
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.class
new file mode 100644
index 0000000..76ee09a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.class
new file mode 100644
index 0000000..4d50806
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.class
new file mode 100644
index 0000000..b76751f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserEntity.class
new file mode 100644
index 0000000..4fe06b5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.class
new file mode 100644
index 0000000..4df503c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.class
new file mode 100644
index 0000000..4e95aae
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.class
new file mode 100644
index 0000000..d58f42a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 7cb2874..54f0807 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -48,6 +48,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import javax.net.ssl.SSLContext;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
@@ -128,14 +129,12 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
 import org.apache.nifi.groups.StandardProcessGroup;
-import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepository;
 import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.logging.ProcessorLog;
 import org.apache.nifi.logging.ProcessorLogObserver;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarClassLoader;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.nar.NarThreadContextClassLoader;
 import org.apache.nifi.processor.Processor;
@@ -165,6 +164,7 @@ import org.apache.nifi.reporting.EventAccess;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.ReflectionUtils;
@@ -184,7 +184,6 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index fa55623..fcbfe46 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -62,8 +62,17 @@ nifi.content.repository.always.sync=${nifi.content.repository.always.sync}
 nifi.content.viewer.url=${nifi.content.viewer.url}
 
 # Provenance Repository Properties
+# Choose implementation of Provenance Repository
 nifi.provenance.repository.implementation=${nifi.provenance.repository.implementation}
 
+# For "Persistent Provenance Repository"
+#nifi.provenance.repository.implementation=org.apache.nifi.provenance.PersistentProvenanceRepository
+# For in-memory provenance repository
+#nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository
+# For "Journaling Provenance Repository"
+#nifi.provenance.repository.implementation=org.apache.nifi.provenance.journaling.JournalingProvenanceRepository
+
+
 # Persistent Provenance Repository Properties
 nifi.provenance.repository.directory.default=${nifi.provenance.repository.directory.default}
 nifi.provenance.repository.max.storage.time=${nifi.provenance.repository.max.storage.time}
@@ -83,9 +92,35 @@ nifi.provenance.repository.indexed.attributes=${nifi.provenance.repository.index
 # but should provide better performance
 nifi.provenance.repository.index.shard.size=${nifi.provenance.repository.index.shard.size}
 
+
+
 # Volatile Provenance Respository Properties
 nifi.provenance.repository.buffer.size=${nifi.provenance.repository.buffer.size}
 
+
+
+# Journaling Provenance Repository Properties
+#nifi.provenance.repository.directory.default=${nifi.provenance.repository.directory.default}
+#nifi.provenance.repository.max.storage.time=${nifi.provenance.repository.max.storage.time}
+#nifi.provenance.repository.max.storage.size=${nifi.provenance.repository.max.storage.size}
+#nifi.provenance.repository.rollover.time=${nifi.provenance.repository.rollover.time}
+#nifi.provenance.repository.rollover.size=${nifi.provenance.repository.rollover.size}
+#nifi.provenance.repository.compress.on.rollover=${nifi.provenance.repository.compress.on.rollover}
+#nifi.provenance.repository.always.sync=${nifi.provenance.repository.always.sync}
+#nifi.provenance.repository.journal.count=${nifi.provenance.repository.journal.count}
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are: 
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, ContentType, Relationship, Details
+#nifi.provenance.repository.indexed.fields=${nifi.provenance.repository.indexed.fields}
+# FlowFile Attributes that should be indexed and made searchable
+#nifi.provenance.repository.indexed.attributes=${nifi.provenance.repository.indexed.attributes}
+# Larger values for block size will result in better compression but slower read performance
+#nifi.provenance.repository.writer.block.size=5000
+#nifi.provenance.repository.worker.threads=2
+#nifi.provenance.repository.query.threads=${nifi.provenance.repository.query.threads}
+#nifi.provenance.repository.compression.threads=2
+
+
+
 # Component Status Repository
 nifi.components.status.repository.implementation=${nifi.components.status.repository.implementation}
 nifi.components.status.repository.buffer.size=${nifi.components.status.repository.buffer.size}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
index cdb98c8..d4f9e44 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournaledStorageLocation.java
@@ -23,11 +23,11 @@ import org.apache.nifi.provenance.StorageLocation;
 public class JournaledStorageLocation implements StorageLocation, Comparable<JournaledStorageLocation>, Comparator<JournaledStorageLocation> {
     private final String containerName;
     private final String sectionName;
-    private final String journalId;
+    private final Long journalId;
     private final int blockIndex;
     private final long eventId;
     
-    public JournaledStorageLocation(final String containerName, final String sectionName, final String journalId, final int blockIndex, final long eventId) {
+    public JournaledStorageLocation(final String containerName, final String sectionName, final Long journalId, final int blockIndex, final long eventId) {
         this.containerName = containerName;
         this.sectionName = sectionName;
         this.journalId = journalId;
@@ -43,7 +43,7 @@ public class JournaledStorageLocation implements StorageLocation, Comparable<Jou
         return sectionName;
     }
     
-    public String getJournalId() {
+    public Long getJournalId() {
         return journalId;
     }
     
@@ -56,6 +56,11 @@ public class JournaledStorageLocation implements StorageLocation, Comparable<Jou
     }
 
     @Override
+    public String toString() {
+        return "JournaledStorageLocation [containerName=" + containerName + ", sectionName=" + sectionName + ", journalId=" + journalId + ", eventId=" + eventId + "]";
+    }
+
+    @Override
     public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
         final int containerVal = o1.getContainerName().compareTo(o2.getContainerName());
         if ( containerVal != 0 ) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
index cc97ee9..0aa20df 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
@@ -30,11 +30,13 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.nifi.events.EventReporter;
@@ -48,6 +50,8 @@ import org.apache.nifi.provenance.StandardProvenanceEventRecord;
 import org.apache.nifi.provenance.StorageLocation;
 import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.IndexAction;
 import org.apache.nifi.provenance.journaling.index.IndexManager;
 import org.apache.nifi.provenance.journaling.index.LuceneIndexManager;
 import org.apache.nifi.provenance.journaling.index.QueryUtils;
@@ -66,24 +70,36 @@ 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.reporting.Severity;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
+// TODO: Add header info to journals. Magic header. Prov repo implementation. Version. Encoding. Encoding Version.
+// TODO: EXPIRE : backpressure if unable to delete fast enough! I.e., if size is greater than 110% of size specified
+// TODO: Ensure number of partitions does not go below the current value. If it does, use the existing value
+// so that we don't lose events.
 public class JournalingProvenanceRepository implements ProvenanceEventRepository {
-    public static final String BLOCK_SIZE = "nifi.provenance.block.size";
+    public static final String WORKER_THREAD_POOL_SIZE = "nifi.provenance.repository.worker.threads";
+    public static final String BLOCK_SIZE = "nifi.provenance.repository.writer.block.size";
     
     private static final Logger logger = LoggerFactory.getLogger(JournalingProvenanceRepository.class);
     
     private final JournalingRepositoryConfig config;
     private final AtomicLong idGenerator = new AtomicLong(0L);
-    private final ScheduledExecutorService executor;
     
-    private EventReporter eventReporter;    // effectively final
-    private PartitionManager partitionManager;  // effectively final
-    private QueryManager queryManager;    // effectively final
-    private IndexManager indexManager;    // effectively final
+    // the follow member variables are effectively final. They are initialized
+    // in the initialize method rather than the constructor because we want to ensure
+    // that they only not created every time that the Java Service Loader instantiates the class.
+    private ScheduledExecutorService workerExecutor;
+    private ExecutorService queryExecutor;
+    private ExecutorService compressionExecutor;
+    private EventReporter eventReporter;
+    private PartitionManager partitionManager;
+    private QueryManager queryManager;
+    private IndexManager indexManager;
     
     public JournalingProvenanceRepository() throws IOException {
         this(createConfig());
@@ -91,19 +107,22 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     
     public JournalingProvenanceRepository(final JournalingRepositoryConfig config) throws IOException {
         this.config = config;
-        this.executor = Executors.newScheduledThreadPool(config.getThreadPoolSize(), new ThreadFactory() {
-            private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
-            
+    }
+
+    private static ThreadFactory createThreadFactory(final String namePrefix) {
+        final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+        final AtomicInteger counter = new AtomicInteger(0);
+        
+        return new ThreadFactory() {
             @Override
             public Thread newThread(final Runnable r) {
                 final Thread thread = defaultFactory.newThread(r);
-                thread.setName("Provenance Repository Worker Thread");
+                thread.setName(namePrefix + "-" + counter.incrementAndGet());
                 return thread;
             }
-        });
+        };
     }
     
-    
     private static JournalingRepositoryConfig createConfig()  {
         final NiFiProperties properties = NiFiProperties.getInstance();
         final Map<String, Path> storageDirectories = properties.getProvenanceRepositoryPaths();
@@ -116,6 +135,7 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
         final String rolloverSize = properties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_SIZE, "100 MB");
         final String shardSize = properties.getProperty(NiFiProperties.PROVENANCE_INDEX_SHARD_SIZE, "500 MB");
         final int queryThreads = properties.getIntegerProperty(NiFiProperties.PROVENANCE_QUERY_THREAD_POOL_SIZE, 2);
+        final int workerThreads = properties.getIntegerProperty(WORKER_THREAD_POOL_SIZE, 4);
         final int journalCount = properties.getIntegerProperty(NiFiProperties.PROVENANCE_JOURNAL_COUNT, 16);
 
         final long storageMillis = FormatUtils.getTimeDuration(storageTime, TimeUnit.MILLISECONDS);
@@ -126,7 +146,7 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
         final boolean compressOnRollover = Boolean.parseBoolean(properties.getProperty(NiFiProperties.PROVENANCE_COMPRESS_ON_ROLLOVER));
         final String indexedFieldString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_FIELDS);
         final String indexedAttrString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_ATTRIBUTES);
-        final int blockSize = properties.getIntegerProperty(BLOCK_SIZE, 1000);
+        final int blockSize = properties.getIntegerProperty(BLOCK_SIZE, 5000);
         
         final Boolean alwaysSync = Boolean.parseBoolean(properties.getProperty("nifi.provenance.repository.always.sync", "false"));
 
@@ -152,7 +172,8 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
         config.setJournalRolloverPeriod(rolloverMillis, TimeUnit.MILLISECONDS);
         config.setEventExpiration(storageMillis, TimeUnit.MILLISECONDS);
         config.setMaxStorageCapacity(maxStorageBytes);
-        config.setThreadPoolSize(queryThreads);
+        config.setQueryThreadPoolSize(queryThreads);
+        config.setWorkerThreadPoolSize(workerThreads);
         config.setPartitionCount(journalCount);
         config.setBlockSize(blockSize);
         
@@ -169,9 +190,43 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     public synchronized void initialize(final EventReporter eventReporter) throws IOException {
         this.eventReporter = eventReporter;
         
-        this.indexManager = new LuceneIndexManager(config, executor);
-        this.partitionManager = new QueuingPartitionManager(indexManager, config, executor);
-        this.queryManager = new StandardQueryManager(partitionManager, config, 10);
+        // We use 3 different thread pools here because we don't want to threads from 1 pool to interfere with
+        // each other. This is because the worker threads can be long running, and they shouldn't tie up the
+        // compression threads. Likewise, there may be MANY compression tasks, which could delay the worker
+        // threads. And the query threads need to run immediately when a user submits a query - they cannot
+        // wait until we finish compressing data and sync'ing the repository!
+        final int workerThreadPoolSize = Math.max(2, config.getWorkerThreadPoolSize());
+        this.workerExecutor = Executors.newScheduledThreadPool(workerThreadPoolSize, createThreadFactory("Provenance Repository Worker Thread"));
+        
+        final int queryThreadPoolSize = Math.max(2, config.getQueryThreadPoolSize());
+        this.queryExecutor = Executors.newScheduledThreadPool(queryThreadPoolSize, createThreadFactory("Provenance Repository Query Thread"));
+        
+        final int compressionThreads = Math.max(1, config.getCompressionThreadPoolSize());
+        this.compressionExecutor = Executors.newFixedThreadPool(compressionThreads, createThreadFactory("Provenance Repository Compression Thread"));
+        
+        this.indexManager = new LuceneIndexManager(config, workerExecutor, queryExecutor);
+        this.partitionManager = new QueuingPartitionManager(indexManager, idGenerator, config, workerExecutor, compressionExecutor);
+        this.queryManager = new StandardQueryManager(indexManager, queryExecutor, config, 10);
+        
+        final Long maxEventId = getMaxEventId();
+        if ( maxEventId != null && maxEventId > 0 ) {
+            this.idGenerator.set(maxEventId);   // maxEventId returns 1 greater than the last event id written
+        }
+        
+        // the partition manager may have caused journals to be re-indexed. We will sync the
+        // index manager to make sure that we are completely in sync before allowing any new data
+        // to be written to the repo.
+        indexManager.sync();
+        
+        final long expirationFrequencyNanos = config.getExpirationFrequency(TimeUnit.NANOSECONDS);
+        workerExecutor.scheduleWithFixedDelay(new ExpireOldEvents(), expirationFrequencyNanos, expirationFrequencyNanos, TimeUnit.NANOSECONDS);
+        
+        workerExecutor.scheduleWithFixedDelay(new Runnable() {
+            @Override
+            public void run() {
+                partitionManager.deleteEventsBasedOnSize();
+            }
+        }, expirationFrequencyNanos, expirationFrequencyNanos, TimeUnit.NANOSECONDS);
     }
 
     @Override
@@ -186,12 +241,19 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
 
     @Override
     public void registerEvents(final Collection<ProvenanceEventRecord> events) throws IOException {
-        partitionManager.withPartition(new VoidPartitionAction() {
-            @Override
-            public void perform(final Partition partition) throws IOException {
-                partition.registerEvents(events, idGenerator.getAndAdd(events.size()));
+        try {
+            partitionManager.withPartition(new VoidPartitionAction() {
+                @Override
+                public void perform(final Partition partition) throws IOException {
+                    partition.registerEvents(events, idGenerator.getAndAdd(events.size()));
+                }
+            }, true);
+        } catch (final IOException ioe) {
+            if ( eventReporter != null ) {
+                eventReporter.reportEvent(Severity.ERROR, "Provenance Repository", "Failed to persist " + events.size() + " events to Provenance Repository due to " + ioe);
             }
-        }, true);
+            throw ioe;
+        }
     }
 
     @Override
@@ -216,11 +278,11 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     public List<StoredProvenanceEvent> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
         // Must generate query to determine the appropriate StorageLocation objects and then call
         // getEvent(List<StorageLocation>)
-        final Set<List<JournaledStorageLocation>> resultSet = partitionManager.withEachPartition(
-            new PartitionAction<List<JournaledStorageLocation>>() {
+        final Set<List<JournaledStorageLocation>> resultSet = indexManager.withEachIndex(
+            new IndexAction<List<JournaledStorageLocation>>() {
                 @Override
-                public List<JournaledStorageLocation> perform(final Partition partition) throws IOException {
-                    return partition.getEvents(firstRecordId, maxRecords);
+                public List<JournaledStorageLocation> perform(final EventIndexSearcher searcher) throws IOException {
+                    return searcher.getEvents(firstRecordId, maxRecords);
                 }
         });
         
@@ -267,7 +329,12 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
             final Callable<List<StoredProvenanceEvent>> callable = new Callable<List<StoredProvenanceEvent>>() {
                 @Override
                 public List<StoredProvenanceEvent> call() throws Exception {
-                    try(final TocReader tocReader = new StandardTocReader(new File(journalFile.getParentFile(), journalFile.getName() + ".toc"));
+                    final File tocFile = QueryUtils.getTocFile(journalFile);
+                    if ( !journalFile.exists() || !tocFile.exists() ) {
+                        return Collections.emptyList();
+                    }
+                    
+                    try(final TocReader tocReader = new StandardTocReader(tocFile);
                         final JournalReader reader = new StandardJournalReader(journalFile)) 
                     {
                         final List<StoredProvenanceEvent> storedEvents = new ArrayList<>(locationsForFile.size());
@@ -284,7 +351,7 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
                 }
             };
             
-            final Future<List<StoredProvenanceEvent>> future = executor.submit(callable);
+            final Future<List<StoredProvenanceEvent>> future = queryExecutor.submit(callable);
             futures.add(future);
         }
         
@@ -331,7 +398,7 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
             public Long perform(final Partition partition) throws IOException {
                 return partition.getMaxEventId();
             }
-        });
+        }, false);
         
         Long maxId = null;
         for ( final Long id : maxIds ) {
@@ -360,26 +427,22 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
 
     @Override
     public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) {
-        // TODO Auto-generated method stub
-        return null;
+        return queryManager.submitLineageComputation(flowFileUuid);
     }
 
     @Override
     public ComputeLineageSubmission retrieveLineageSubmission(final String lineageIdentifier) {
-        // TODO Auto-generated method stub
-        return null;
+        return queryManager.retrieveLineageSubmission(lineageIdentifier);
     }
 
     @Override
     public ComputeLineageSubmission submitExpandParents(final long eventId) {
-        // TODO Auto-generated method stub
-        return null;
+        return queryManager.submitExpandParents(this, eventId);
     }
 
     @Override
     public ComputeLineageSubmission submitExpandChildren(final long eventId) {
-        // TODO Auto-generated method stub
-        return null;
+        return queryManager.submitExpandChildren(this, eventId);
     }
 
     @Override
@@ -388,16 +451,40 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
             partitionManager.shutdown();
         }
         
-        indexManager.close();
+        if ( indexManager != null ) {
+            try {
+                indexManager.close();
+            } catch (final IOException ioe) {
+                logger.warn("Failed to shutdown Index Manager due to {}", ioe.toString());
+                if ( logger.isDebugEnabled() ) {
+                    logger.warn("", ioe);
+                }
+            }
+        }
         
-        // TODO: make sure that all are closed here!
+        if ( queryManager != null ) {
+            try {
+                queryManager.close();
+            } catch (final IOException ioe) {
+                logger.warn("Failed to shutdown Query Manager due to {}", ioe.toString());
+                if ( logger.isDebugEnabled() ) {
+                    logger.warn("", ioe);
+                }
+            }
+        }
         
-        executor.shutdown();
+        compressionExecutor.shutdown();
+        workerExecutor.shutdown();
+        queryExecutor.shutdown();
     }
 
     @Override
     public List<SearchableField> getSearchableFields() {
-        return config.getSearchableFields();
+        final List<SearchableField> searchableFields = new ArrayList<>(config.getSearchableFields());
+        // we exclude the Event Time because it is always searchable and is a bit special in its handling
+        // because it dictates in some cases which index files we look at
+        searchableFields.remove(SearchableFields.EventTime);
+        return searchableFields;
     }
 
     @Override
@@ -413,7 +500,7 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
             public Long perform(final Partition partition) throws IOException {
                 return partition.getEarliestEventTime();
             }
-        });
+        }, false);
         
         // Find the latest timestamp for each of the "earliest" timestamps.
         // This is a bit odd, but we're doing it for a good reason:
@@ -437,4 +524,43 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
         return latest;
     }
 
+
+    
+    private class ExpireOldEvents implements Runnable {
+        @Override
+        public void run() {
+            final long now = System.currentTimeMillis();
+            final long expirationThreshold = now - config.getEventExpiration(TimeUnit.MILLISECONDS);
+            
+            try {
+                indexManager.deleteOldEvents(expirationThreshold);
+            } catch (final IOException ioe) {
+                logger.error("Failed to delete expired events from index due to {}", ioe.toString());
+                if ( logger.isDebugEnabled() ) {
+                    logger.error("", ioe);
+                }
+            }
+            
+            try {
+                partitionManager.withEachPartitionSerially(new VoidPartitionAction() {
+                    @Override
+                    public void perform(final Partition partition) throws IOException {
+                        try {
+                            partition.deleteOldEvents(expirationThreshold);
+                        } catch (final IOException ioe) {
+                            logger.error("Failed to delete expired events from Partition {} due to {}", partition, ioe.toString());
+                            if ( logger.isDebugEnabled() ) {
+                                logger.error("", ioe);
+                            }
+                        }
+                    }
+                }, false);
+            } catch (IOException ioe) {
+                logger.error("Failed to delete expired events from journals due to {}", ioe.toString());
+                if ( logger.isDebugEnabled() ) {
+                    logger.error("", ioe);
+                }
+            }
+        }
+    }
 }


[07/12] incubator-nifi git commit: continuing to implement

Posted by ma...@apache.org.
continuing to implement


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/a5f557ad
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/a5f557ad
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/a5f557ad

Branch: refs/heads/journaling-prov-repo
Commit: a5f557ad966c4fa70ae0a0239e3bf70dcd788ff0
Parents: b95e756
Author: Mark Payne <ma...@hotmail.com>
Authored: Sun Feb 15 10:45:29 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Sun Feb 15 10:45:29 2015 -0500

----------------------------------------------------------------------
 .../JournalingProvenanceRepository.java         |  30 ++-
 .../config/JournalingRepositoryConfig.java      |   5 +
 .../journaling/index/EventIndexSearcher.java    |   2 +
 .../journaling/index/IndexManager.java          |  41 +++
 .../journaling/index/LuceneIndexManager.java    | 178 +++++++++++++
 .../journaling/index/LuceneIndexSearcher.java   |  23 ++
 .../journaling/index/LuceneIndexWriter.java     |   7 +-
 .../journaling/index/MultiIndexSearcher.java    | 112 ++++++++
 .../journals/StandardJournalWriter.java         |  40 ++-
 .../partition/JournalingPartition.java          | 254 ++++++++++---------
 .../journaling/partition/PartitionManager.java  |  13 +
 .../partition/QueuingPartitionManager.java      |  16 +-
 .../journaling/tasks/CompressionTask.java       |   2 +-
 .../journaling/toc/StandardTocWriter.java       |  25 +-
 14 files changed, 607 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
index 7911d73..cc97ee9 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/JournalingProvenanceRepository.java
@@ -30,9 +30,10 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -47,6 +48,8 @@ import org.apache.nifi.provenance.StandardProvenanceEventRecord;
 import org.apache.nifi.provenance.StorageLocation;
 import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
+import org.apache.nifi.provenance.journaling.index.LuceneIndexManager;
 import org.apache.nifi.provenance.journaling.index.QueryUtils;
 import org.apache.nifi.provenance.journaling.journals.JournalReader;
 import org.apache.nifi.provenance.journaling.journals.StandardJournalReader;
@@ -75,11 +78,12 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     
     private final JournalingRepositoryConfig config;
     private final AtomicLong idGenerator = new AtomicLong(0L);
-    private final ExecutorService executor;
+    private final ScheduledExecutorService executor;
     
     private EventReporter eventReporter;    // effectively final
     private PartitionManager partitionManager;  // effectively final
     private QueryManager queryManager;    // effectively final
+    private IndexManager indexManager;    // effectively final
     
     public JournalingProvenanceRepository() throws IOException {
         this(createConfig());
@@ -87,7 +91,16 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     
     public JournalingProvenanceRepository(final JournalingRepositoryConfig config) throws IOException {
         this.config = config;
-        this.executor = Executors.newFixedThreadPool(config.getThreadPoolSize());
+        this.executor = Executors.newScheduledThreadPool(config.getThreadPoolSize(), new ThreadFactory() {
+            private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+            
+            @Override
+            public Thread newThread(final Runnable r) {
+                final Thread thread = defaultFactory.newThread(r);
+                thread.setName("Provenance Repository Worker Thread");
+                return thread;
+            }
+        });
     }
     
     
@@ -156,7 +169,8 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     public synchronized void initialize(final EventReporter eventReporter) throws IOException {
         this.eventReporter = eventReporter;
         
-        this.partitionManager = new QueuingPartitionManager(config, executor);
+        this.indexManager = new LuceneIndexManager(config, executor);
+        this.partitionManager = new QueuingPartitionManager(indexManager, config, executor);
         this.queryManager = new StandardQueryManager(partitionManager, config, 10);
     }
 
@@ -312,7 +326,7 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
 
     @Override
     public Long getMaxEventId() throws IOException {
-        final Set<Long> maxIds = partitionManager.withEachPartition(new PartitionAction<Long>() {
+        final Set<Long> maxIds = partitionManager.withEachPartitionSerially(new PartitionAction<Long>() {
             @Override
             public Long perform(final Partition partition) throws IOException {
                 return partition.getMaxEventId();
@@ -374,6 +388,10 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
             partitionManager.shutdown();
         }
         
+        indexManager.close();
+        
+        // TODO: make sure that all are closed here!
+        
         executor.shutdown();
     }
 
@@ -390,7 +408,7 @@ public class JournalingProvenanceRepository implements ProvenanceEventRepository
     @Override
     public Long getEarliestEventTime() throws IOException {
         // Get the earliest event timestamp for each partition
-        final Set<Long> earliestTimes = partitionManager.withEachPartition(new PartitionAction<Long>() {
+        final Set<Long> earliestTimes = partitionManager.withEachPartitionSerially(new PartitionAction<Long>() {
             @Override
             public Long perform(final Partition partition) throws IOException {
                 return partition.getEarliestEventTime();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
index 8998932..18871c7 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
@@ -35,6 +35,7 @@ public class JournalingRepositoryConfig {
     private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB
     private int partitionCount = 16;
     private int blockSize = 5000;
+    private int indexesPerContainer = 2;
 
     private List<SearchableField> searchableFields = new ArrayList<>();
     private List<SearchableField> searchableAttributes = new ArrayList<>();
@@ -51,6 +52,10 @@ public class JournalingRepositoryConfig {
         return readOnly;
     }
 
+    public int getIndexesPerContainer() {
+        return indexesPerContainer;
+    }
+    
     /**
      * Specifies where the repository will store data
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
index b669c53..753378d 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
@@ -33,4 +33,6 @@ public interface EventIndexSearcher extends Closeable {
     SearchResult search(Query query) throws IOException;
     
     List<JournaledStorageLocation> getEvents(long minEventId, int maxResults) throws IOException;
+    
+    Long getMaxEventId(String container, String section) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
new file mode 100644
index 0000000..141b84a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public interface IndexManager extends Closeable {
+
+    /**
+     * Returns an EventIndexWriter for the given container.
+     * @param container
+     * @return
+     */
+    EventIndexWriter getIndexWriter(final String container);
+
+    /**
+     * Returns the max event ID that has been indexed for the given container and section.
+     * 
+     * @param container
+     * @param section
+     * @return
+     */
+    Long getMaxEventId(String container, String section) throws IOException;
+    
+    EventIndexSearcher newIndexSearcher(String containerName) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
new file mode 100644
index 0000000..d10fd00
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LuceneIndexManager implements IndexManager {
+    private static final Logger logger = LoggerFactory.getLogger(LuceneIndexManager.class);
+    
+    private final JournalingRepositoryConfig config;
+    private final ScheduledExecutorService executor;
+    
+    private final Map<String, List<LuceneIndexWriter>> writers = new HashMap<>();
+    private final Map<String, AtomicLong> writerIndexes = new HashMap<>();
+    
+    public LuceneIndexManager(final JournalingRepositoryConfig config, final ScheduledExecutorService executor) throws IOException {
+        this.config = config;
+        this.executor = executor;
+        
+        final int rolloverSeconds = (int) config.getJournalRolloverPeriod(TimeUnit.SECONDS);
+        if ( !config.isReadOnly() ) {
+            for ( final Map.Entry<String, File> entry : config.getContainers().entrySet() ) {
+                final String containerName = entry.getKey();
+                final File container = entry.getValue();
+                
+                final List<LuceneIndexWriter> writerList = new ArrayList<>(config.getIndexesPerContainer());
+                writers.put(containerName, writerList);
+                writerIndexes.put(containerName, new AtomicLong(0L));
+                
+                for ( int i=0; i < config.getIndexesPerContainer(); i++ ){
+                    final File indexDir = new File(container, "indices/" + i);
+                    writerList.add(new LuceneIndexWriter(indexDir, config));
+                }
+                
+                executor.scheduleWithFixedDelay(new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            sync(containerName);
+                        } catch (final Throwable t) {
+                            logger.error("Failed to sync Provenance Repository Container {} due to {}", containerName, t);
+                            if ( logger.isDebugEnabled() ) {
+                                logger.error("", t);
+                            }
+                        }
+                    }
+                }, rolloverSeconds, rolloverSeconds, TimeUnit.SECONDS);
+            }
+        }
+    }
+    
+    @Override
+    public EventIndexSearcher newIndexSearcher(final String containerName) throws IOException {
+        final File containerDir = config.getContainers().get(containerName);
+        if ( containerDir == null ) {
+            throw new IllegalArgumentException();
+        }
+        
+        final List<EventIndexSearcher> searchers = new ArrayList<>();
+        
+        try {
+            if (config.isReadOnly()) {
+                for (int i=0; i < config.getIndexesPerContainer(); i++) {
+                    final File indexDir = new File(containerName, "indices/" + i);
+                    searchers.add(new LuceneIndexSearcher(indexDir));
+                }
+            } else {
+                final List<LuceneIndexWriter> writerList = writers.get(containerName);
+                for ( final LuceneIndexWriter writer : writerList ) {
+                    searchers.add(writer.newIndexSearcher());
+                }
+            }
+        } catch (final IOException ioe) {
+            // If we failed to create a searcher, we need to close all that we've already created.
+            for ( final EventIndexSearcher searcher : searchers ) {
+                try {
+                    searcher.close();
+                } catch (final IOException ioe2) {
+                    ioe.addSuppressed(ioe2);
+                }
+            }
+            
+            throw ioe;
+        }
+        
+        return new MultiIndexSearcher(searchers);
+    }
+    
+    @Override
+    public LuceneIndexWriter getIndexWriter(final String container) {
+        if (config.isReadOnly() ) {
+            throw new IllegalStateException("Cannot obtain Index Writer because repository is read-only");
+        }
+        
+        final AtomicLong index = writerIndexes.get(container);
+        if (index == null ) {
+            throw new IllegalArgumentException();
+        }
+        
+        final long curVal = index.get();
+        final List<LuceneIndexWriter> writerList = writers.get(container);
+        return writerList.get((int) (curVal % writerList.size()));
+    }
+
+    @Override
+    public Long getMaxEventId(final String container, final String section) throws IOException {
+        final List<LuceneIndexWriter> writerList = writers.get(container);
+        if ( writerList == null ) {
+            return null;
+        }
+
+        Long max = null;
+        for ( final LuceneIndexWriter writer : writerList ) {
+            try (final EventIndexSearcher searcher = writer.newIndexSearcher()) {
+                final Long maxForWriter = searcher.getMaxEventId(container, section);
+                if ( maxForWriter != null ) {
+                    if (max == null || maxForWriter.longValue() > max.longValue() ) {
+                        max = maxForWriter;
+                    }
+                }
+            }
+        }
+        
+        return max;
+    }
+
+    
+    private void sync(final String containerName) throws IOException {
+        final AtomicLong index = writerIndexes.get(containerName);
+        final long curIndex = index.getAndIncrement();
+        
+        final List<LuceneIndexWriter> writerList = writers.get(containerName);
+        final EventIndexWriter toSync = writerList.get((int) (curIndex % writerList.size()));
+        toSync.sync();
+    }
+
+    @Override
+    public void close() throws IOException {
+        for ( final List<LuceneIndexWriter> writerList : writers.values() ) {
+            for ( final LuceneIndexWriter writer : writerList ) {
+                try {
+                    writer.close();
+                } catch (final IOException ioe) {
+                    logger.warn("Failed to close {} due to {}", writer, ioe);
+                    if ( logger.isDebugEnabled() ) {
+                        logger.warn("", ioe);
+                    }
+                }
+            }
+        }        
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
index 32dc7c3..a9dd1a5 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
@@ -31,6 +32,7 @@ import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortField.Type;
+import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
@@ -110,4 +112,25 @@ public class LuceneIndexSearcher implements EventIndexSearcher {
         return getLocations(topDocs);
     }
 
+    @Override
+    public Long getMaxEventId(final String container, final String section) throws IOException {
+        final BooleanQuery query = new BooleanQuery();
+        
+        if ( container != null ) {
+            query.add(new TermQuery(new Term(IndexedFieldNames.CONTAINER_NAME, container)), Occur.MUST);
+        }
+        
+        if ( section != null ) {
+            query.add(new TermQuery(new Term(IndexedFieldNames.SECTION_NAME, section)), Occur.MUST);
+        }
+        
+        final TopDocs topDocs = searcher.search(query, 1, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG, true)));
+        final List<JournaledStorageLocation> locations = getLocations(topDocs);
+        if ( locations.isEmpty() ) {
+            return null;
+        }
+        
+        return locations.get(0).getEventId();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
index e955ae5..b61ad34 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
@@ -18,9 +18,11 @@ package org.apache.nifi.provenance.journaling.index;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
@@ -117,6 +119,7 @@ public class LuceneIndexWriter implements EventIndexWriter {
     public void index(final Collection<JournaledProvenanceEvent> events) throws IOException {
         long maxId = this.indexMaxId.get();
         
+        final List<Document> documents = new ArrayList<>(events.size());
         for ( final JournaledProvenanceEvent event : events ) {
             maxId = event.getEventId();
 
@@ -189,8 +192,10 @@ public class LuceneIndexWriter implements EventIndexWriter {
                 }
             }
 
-            indexWriter.addDocument(doc);
+            documents.add(doc);
         }
+        
+        indexWriter.addDocuments(documents);
 
         // Update the index's max id
         boolean updated = false;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
new file mode 100644
index 0000000..d086ff5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
+import org.apache.nifi.provenance.search.Query;
+
+public class MultiIndexSearcher implements EventIndexSearcher {
+    private final List<EventIndexSearcher> searchers;
+    
+    public MultiIndexSearcher(final List<EventIndexSearcher> searchers) {
+        this.searchers = searchers;
+    }
+
+    @Override
+    public void close() throws IOException {
+        IOException suppressed = null;
+        
+        for ( final EventIndexSearcher searcher : searchers ) {
+            try {
+                searcher.close();
+            } catch (final IOException ioe) {
+                if ( suppressed == null ) {
+                    suppressed = ioe;
+                } else {
+                    suppressed.addSuppressed(ioe);
+                }
+            }
+        }
+        
+        if ( suppressed != null ) {
+            throw suppressed;
+        }
+    }
+
+    @Override
+    public SearchResult search(final Query query) throws IOException {
+        int totalHitCount = 0;
+        final List<JournaledStorageLocation> locations = new ArrayList<>();
+        
+        for ( final EventIndexSearcher searcher : searchers ) {
+            final SearchResult result = searcher.search(query);
+            totalHitCount += result.getTotalCount();
+            locations.addAll(result.getLocations());
+        }
+        
+        return new SearchResult(locations, totalHitCount);
+    }
+
+    @Override
+    public List<JournaledStorageLocation> getEvents(final long minEventId, final int maxResults) throws IOException {
+        final List<JournaledStorageLocation> locations = new ArrayList<>();
+        int results = 0;
+        
+        // Perform search against all searchers and aggregate results.
+        for ( final EventIndexSearcher searcher : searchers ) {
+            final List<JournaledStorageLocation> searchLocations = searcher.getEvents(minEventId, maxResults);
+            locations.addAll(searchLocations);
+            if ( !searchLocations.isEmpty() ) {
+                results++;
+            }
+        }
+        
+        // Results from this call are sorted. If we have only 0 or 1 searchers that had results, then
+        // we don't need to sort anything. Otherwise, we need to sort and return just the first X
+        // number of results.
+        if ( results > 1 ) {
+            Collections.sort(locations);
+        }
+        
+        if ( locations.size() > maxResults ) {
+            return locations.subList(0, maxResults);
+        }
+        
+        return locations;
+    }
+
+    @Override
+    public Long getMaxEventId(final String container, final String section) throws IOException {
+        Long max = null;
+        for ( final EventIndexSearcher searcher : searchers ) {
+            final Long maxForWriter = searcher.getMaxEventId(container, section);
+            if ( maxForWriter != null ) {
+                if (max == null || maxForWriter.longValue() > max.longValue() ) {
+                    max = maxForWriter;
+                }
+            }
+        }
+        
+        return max;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
index 5a289fe..af5f8de 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
@@ -16,8 +16,6 @@
  */
 package org.apache.nifi.provenance.journaling.journals;
 
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
@@ -29,6 +27,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.journaling.io.Serializer;
 import org.apache.nifi.remote.io.CompressionOutputStream;
+import org.apache.nifi.stream.io.BufferedOutputStream;
+import org.apache.nifi.stream.io.ByteArrayOutputStream;
 import org.apache.nifi.stream.io.ByteCountingOutputStream;
 
 
@@ -96,6 +96,9 @@ public class StandardJournalWriter implements JournalWriter {
     private OutputStream compressedStream;
     private ByteCountingOutputStream out;
     
+    private long recordBytes = 256L;
+    private long recordCount = 1L;
+    
     
     public StandardJournalWriter(final long journalId, final File journalFile, final boolean compressed, final Serializer serializer) throws IOException {
         this.journalId = journalId;
@@ -132,16 +135,38 @@ public class StandardJournalWriter implements JournalWriter {
     @Override
     public void close() throws IOException {
         finishBlock();
-        
-        if ( compressedStream != null ) {
+
+        IOException suppressed = null;
+        try {
             compressedStream.flush();
             compressedStream.close();
+        } catch (final IOException ioe) {
+            suppressed = ioe;
+        }
+        
+        try {
+            try {
+                uncompressedStream.flush();
+            } finally {
+                uncompressedStream.close();
+            }
+        } catch (final IOException ioe) {
+            if ( suppressed != null ) {
+                ioe.addSuppressed(suppressed);
+            }
+            throw ioe;
+        }
+        
+        if ( suppressed != null ) {
+            throw suppressed;
         }
     }
 
     @Override
     public void write(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
-        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        final int avgRecordSize = (int) (recordBytes / recordCount);
+        
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream(avgRecordSize);
         final DataOutputStream serializerDos = new DataOutputStream(baos);
         
         final BufferedOutputStream bos = new BufferedOutputStream(out);
@@ -153,10 +178,13 @@ public class StandardJournalWriter implements JournalWriter {
                 serializer.serialize(event, serializerDos);
                 serializerDos.flush();
                 
-                final int recordLength = 8 + baos.size();   // record length is length of ID (8 bytes) plus length of serialized record
+                final int serializedLength = baos.size();
+                final int recordLength = 8 + serializedLength;   // record length is length of ID (8 bytes) plus length of serialized record
                 outDos.writeInt(recordLength);
                 outDos.writeLong(id++);
                 baos.writeTo(outDos);
+                recordBytes += recordLength;
+                recordCount++;
                 baos.reset();
                 
                 eventCount++;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
index 651c41e..1ace37f 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/JournalingPartition.java
@@ -35,8 +35,11 @@ import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
 import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
 import org.apache.nifi.provenance.journaling.index.EventIndexSearcher;
+import org.apache.nifi.provenance.journaling.index.EventIndexWriter;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
 import org.apache.nifi.provenance.journaling.index.LuceneIndexSearcher;
 import org.apache.nifi.provenance.journaling.index.LuceneIndexWriter;
+import org.apache.nifi.provenance.journaling.index.MultiIndexSearcher;
 import org.apache.nifi.provenance.journaling.index.QueryUtils;
 import org.apache.nifi.provenance.journaling.io.StandardEventSerializer;
 import org.apache.nifi.provenance.journaling.journals.JournalReader;
@@ -55,13 +58,12 @@ public class JournalingPartition implements Partition {
     private static final String JOURNAL_FILE_EXTENSION = ".journal";
     
     private final String containerName;
-    private final String sectionName;
+    private final int sectionIndex;
     
     private final File section;
     private final File journalsDir;
     private final JournalingRepositoryConfig config;
     private final ExecutorService executor;
-    private final LuceneIndexWriter indexWriter;
     
     private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
     private final Lock readLock = rwLock.readLock();
@@ -73,9 +75,12 @@ public class JournalingPartition implements Partition {
     private volatile long maxEventId = -1L;
     private volatile Long earliestEventTime = null;
     
-    public JournalingPartition(final String containerName, final String sectionName, final File sectionDir, final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
+    private final IndexManager indexManager;
+    
+    public JournalingPartition(final IndexManager indexManager, final String containerName, final int sectionIndex, final File sectionDir, final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
+        this.indexManager = indexManager;
         this.containerName = containerName;
-        this.sectionName = sectionName;
+        this.sectionIndex = sectionIndex;
         this.section = sectionDir;
         this.journalsDir = new File(section, "journals");
         this.config = config;
@@ -88,22 +93,11 @@ public class JournalingPartition implements Partition {
         if ( journalsDir.exists() && journalsDir.isFile() ) {
             throw new IOException("Could not create directory " + section + " because a file already exists with this name");
         }
-        
-        if ( config.isReadOnly() ) {
-            indexWriter = null;
-        } else {
-            final File indexDir = new File(section, "index");
-            indexWriter = new LuceneIndexWriter(indexDir, config);
-        }
     }
     
     
     public EventIndexSearcher newIndexSearcher() throws IOException {
-        if (config.isReadOnly()) {
-            return new LuceneIndexSearcher(new File(section, "index"));
-        }
-        
-        return indexWriter.newIndexSearcher();
+        return indexManager.newIndexSearcher(containerName);
     }
     
     protected JournalWriter getJournalWriter(final long firstEventId) throws IOException {
@@ -118,6 +112,11 @@ public class JournalingPartition implements Partition {
         return journalWriter;
     }
     
+    // MUST be called with writeLock or readLock held.
+    private EventIndexWriter getIndexWriter() {
+        return indexManager.getIndexWriter(containerName);
+    }
+    
     @Override
     public List<JournaledProvenanceEvent> registerEvents(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
         writeLock.lock();
@@ -139,12 +138,13 @@ public class JournalingPartition implements Partition {
             final List<JournaledProvenanceEvent> storedEvents = new ArrayList<>(events.size());
             long id = firstEventId;
             for (final ProvenanceEventRecord event : events) {
-                final JournaledStorageLocation location = new JournaledStorageLocation(containerName, sectionName, 
+                final JournaledStorageLocation location = new JournaledStorageLocation(containerName, String.valueOf(sectionIndex), 
                         String.valueOf(writer.getJournalId()), tocWriter.getCurrentBlockIndex(), id++);
                 final JournaledProvenanceEvent storedEvent = new JournaledProvenanceEvent(event, location);
                 storedEvents.add(storedEvent);
             }
             
+            final EventIndexWriter indexWriter = getIndexWriter();
             indexWriter.index(storedEvents);
             
             if ( config.isAlwaysSync() ) {
@@ -196,13 +196,28 @@ public class JournalingPartition implements Partition {
     
     // MUST be called with write lock held.
     private void rollover(final long firstEventId) throws IOException {
+        // TODO: Rework how rollover works because we now have index manager!!
+        
         // if we have a writer already, close it and initiate rollover actions
         if ( journalWriter != null ) {
             journalWriter.finishBlock();
             journalWriter.close();
             tocWriter.close();
-            indexWriter.sync();
-        
+
+            final EventIndexWriter curWriter = getIndexWriter();
+            executor.submit(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        curWriter.sync();
+                    } catch (final IOException e) {
+                        
+                        // TODO Auto-generated catch block
+                        e.printStackTrace();
+                    }
+                }
+            });
+            
             if ( config.isCompressOnRollover() ) {
                 final File finishedFile = journalWriter.getJournalFile();
                 final File finishedTocFile = tocWriter.getFile();
@@ -213,7 +228,7 @@ public class JournalingPartition implements Partition {
         // create new writers and reset state.
         final File journalFile = new File(journalsDir, firstEventId + JOURNAL_FILE_EXTENSION);
         journalWriter = new StandardJournalWriter(firstEventId, journalFile, false, new StandardEventSerializer());
-        tocWriter = new StandardTocWriter(QueryUtils.getTocFile(journalFile), false);
+        tocWriter = new StandardTocWriter(QueryUtils.getTocFile(journalFile), false, config.isAlwaysSync());
         tocWriter.addBlockOffset(journalWriter.getSize());
         numEventsAtEndOfLastBlock = 0;
     }
@@ -237,112 +252,123 @@ public class JournalingPartition implements Partition {
     
     @Override
     public void restore() throws IOException {
-        // delete or rename files if stopped during rollover; compress any files that haven't been compressed
-        if ( !config.isReadOnly() ) {
-            final File[] children = journalsDir.listFiles();
-            if ( children != null ) {
-                // find the latest journal.
-                File latestJournal = null;
-                long latestJournalId = -1L;
-                
-                final List<File> journalFiles = new ArrayList<>();
-                
-                // find any journal files that either haven't been compressed or were partially compressed when
-                // we last shutdown and then restart compression.
-                for ( final File file : children ) {
-                    final String filename = file.getName();
-                    if ( !filename.contains(JOURNAL_FILE_EXTENSION) ) {
-                        continue;
-                    }
+        writeLock.lock();
+        try {
+            // delete or rename files if stopped during rollover; compress any files that haven't been compressed
+            if ( !config.isReadOnly() ) {
+                final File[] children = journalsDir.listFiles();
+                if ( children != null ) {
+                    // find the latest journal.
+                    File latestJournal = null;
+                    long latestJournalId = -1L;
+                    
+                    final List<File> journalFiles = new ArrayList<>();
                     
-                    final Long journalId = getJournalId(file);
-                    if ( journalId != null && journalId > latestJournalId ) {
-                        latestJournal = file;
-                        latestJournalId = journalId;
+                    // find any journal files that either haven't been compressed or were partially compressed when
+                    // we last shutdown and then restart compression.
+                    for ( final File file : children ) {
+                        final String filename = file.getName();
+                        if ( !filename.contains(JOURNAL_FILE_EXTENSION) ) {
+                            continue;
+                        }
+                        
+                        final Long journalId = getJournalId(file);
+                        if ( journalId != null && journalId > latestJournalId ) {
+                            latestJournal = file;
+                            latestJournalId = journalId;
+                        }
+                        
+                        journalFiles.add(file);
+                        
+                        if ( !config.isCompressOnRollover() ) {
+                            continue;
+                        }
+                        
+                        if ( filename.endsWith(CompressionTask.FILE_EXTENSION) ) {
+                            final File uncompressedFile = new File(journalsDir, filename.replace(CompressionTask.FILE_EXTENSION, ""));
+                            if ( uncompressedFile.exists() ) {
+                                // both the compressed and uncompressed version of this journal exist. The Compression Task was
+                                // not complete when we shutdown. Delete the compressed journal and toc and re-start the Compression Task.
+                                final File tocFile = QueryUtils.getTocFile(uncompressedFile);
+                                executor.submit(new CompressionTask(uncompressedFile, getJournalId(uncompressedFile), tocFile));
+                            } else {
+                                // The compressed file exists but the uncompressed file does not. This means that we have finished
+                                // writing the compressed file and deleted the original journal file but then shutdown before
+                                // renaming the compressed file to the original filename. We can simply rename the compressed file
+                                // to the original file and then address the TOC file.
+                                final boolean rename = CompressionTask.rename(file, uncompressedFile);
+                                if ( !rename ) {
+                                    logger.warn("{} During recovery, failed to rename {} to {}", this, file, uncompressedFile);
+                                    continue;
+                                }
+                                
+                                // Check if the compressed TOC file exists. If not, we are finished.
+                                // If it does exist, then we know that it is complete, as described above, so we will go
+                                // ahead and replace the uncompressed version.
+                                final File tocFile = QueryUtils.getTocFile(uncompressedFile);
+                                final File compressedTocFile = new File(tocFile.getParentFile(), tocFile.getName() + CompressionTask.FILE_EXTENSION);
+                                if ( !compressedTocFile.exists() ) {
+                                    continue;
+                                }
+                                
+                                tocFile.delete();
+                                
+                                final boolean renamedTocFile = CompressionTask.rename(compressedTocFile, tocFile);
+                                if ( !renamedTocFile ) {
+                                    logger.warn("{} During recovery, failed to rename {} to {}", this, compressedTocFile, tocFile);
+                                }
+                            }
+                        }
                     }
                     
-                    journalFiles.add(file);
+                    // Get the first event in the earliest journal file so that we know what the earliest time available is
+                    Collections.sort(journalFiles, new Comparator<File>() {
+                        @Override
+                        public int compare(final File o1, final File o2) {
+                            return Long.compare(getJournalId(o1), getJournalId(o2));
+                        }
+                    });
                     
-                    if ( !config.isCompressOnRollover() ) {
-                        continue;
+                    for ( final File journal : journalFiles ) {
+                        try (final JournalReader reader = new StandardJournalReader(journal)) {
+                            final ProvenanceEventRecord record = reader.nextEvent();
+                            this.earliestEventTime = record.getEventTime();
+                            break;
+                        } catch (final IOException ioe) {
+                        }
                     }
                     
-                    if ( filename.endsWith(CompressionTask.FILE_EXTENSION) ) {
-                        final File uncompressedFile = new File(journalsDir, filename.replace(CompressionTask.FILE_EXTENSION, ""));
-                        if ( uncompressedFile.exists() ) {
-                            // both the compressed and uncompressed version of this journal exist. The Compression Task was
-                            // not complete when we shutdown. Delete the compressed journal and toc and re-start the Compression Task.
-                            final File tocFile = QueryUtils.getTocFile(uncompressedFile);
-                            executor.submit(new CompressionTask(uncompressedFile, getJournalId(uncompressedFile), tocFile));
-                        } else {
-                            // The compressed file exists but the uncompressed file does not. This means that we have finished
-                            // writing the compressed file and deleted the original journal file but then shutdown before
-                            // renaming the compressed file to the original filename. We can simply rename the compressed file
-                            // to the original file and then address the TOC file.
-                            final boolean rename = CompressionTask.rename(file, uncompressedFile);
-                            if ( !rename ) {
-                                logger.warn("{} During recovery, failed to rename {} to {}", this, file, uncompressedFile);
-                                continue;
-                            }
-                            
-                            // Check if the compressed TOC file exists. If not, we are finished.
-                            // If it does exist, then we know that it is complete, as described above, so we will go
-                            // ahead and replace the uncompressed version.
-                            final File tocFile = QueryUtils.getTocFile(uncompressedFile);
-                            final File compressedTocFile = new File(tocFile.getParentFile(), tocFile.getName() + CompressionTask.FILE_EXTENSION);
-                            if ( !compressedTocFile.exists() ) {
-                                continue;
-                            }
-                            
-                            tocFile.delete();
-                            
-                            final boolean renamedTocFile = CompressionTask.rename(compressedTocFile, tocFile);
-                            if ( !renamedTocFile ) {
-                                logger.warn("{} During recovery, failed to rename {} to {}", this, compressedTocFile, tocFile);
-                            }
+                    // Whatever was the last journal for this partition, we need to remove anything for that journal
+                    // from the index and re-add them, and then sync the index. This allows us to avoid syncing
+                    // the index each time (we sync only on rollover) but allows us to still ensure that we index
+                    // all events.
+                    if ( latestJournal != null ) {
+                        try {
+                            reindex(latestJournal);
+                        } catch (final EOFException eof) {
                         }
                     }
                 }
-                
-                // Get the first event in the earliest journal file so that we know what the earliest time available is
-                Collections.sort(journalFiles, new Comparator<File>() {
-                    @Override
-                    public int compare(final File o1, final File o2) {
-                        return Long.compare(getJournalId(o1), getJournalId(o2));
-                    }
-                });
-                
-                for ( final File journal : journalFiles ) {
-                    try (final JournalReader reader = new StandardJournalReader(journal)) {
-                        final ProvenanceEventRecord record = reader.nextEvent();
-                        this.earliestEventTime = record.getEventTime();
-                        break;
-                    } catch (final IOException ioe) {
-                    }
-                }
-                
-                // Whatever was the last journal for this partition, we need to remove anything for that journal
-                // from the index and re-add them, and then sync the index. This allows us to avoid syncing
-                // the index each time (we sync only on rollover) but allows us to still ensure that we index
-                // all events.
-                if ( latestJournal != null ) {
-                    try {
-                        reindex(latestJournal);
-                    } catch (final EOFException eof) {
-                    }
-                }
             }
+        } finally {
+            writeLock.unlock();
         }
     }
 
     
     private void reindex(final File journalFile) throws IOException {
-        try (final TocJournalReader reader = new TocJournalReader(containerName, sectionName, String.valueOf(getJournalId(journalFile)), journalFile)) {
-            indexWriter.delete(containerName, sectionName, String.valueOf(getJournalId(journalFile)));
+        // TODO: Rework how recovery works because we now have index manager!!
+        try (final TocJournalReader reader = new TocJournalReader(containerName, String.valueOf(sectionIndex), String.valueOf(getJournalId(journalFile)), journalFile)) {
+            // We don't know which index contains the data for this journal, so remove the journal
+            // from both.
+            for (final LuceneIndexWriter indexWriter : indexWriters ) {
+                indexWriter.delete(containerName, String.valueOf(sectionIndex), String.valueOf(getJournalId(journalFile)));
+            }
             
             long maxId = -1L;
             final List<JournaledProvenanceEvent> storedEvents = new ArrayList<>(1000);
             JournaledProvenanceEvent event;
+            final LuceneIndexWriter indexWriter = indexWriters[0];
             while ((event = reader.nextJournaledEvent()) != null ) {
                 storedEvents.add(event);
                 maxId = event.getEventId();
@@ -365,7 +391,7 @@ public class JournalingPartition implements Partition {
     
     @Override
     public List<JournaledStorageLocation> getEvents(final long minEventId, final int maxRecords) throws IOException {
-        try (final EventIndexSearcher searcher = indexWriter.newIndexSearcher()) {
+        try (final EventIndexSearcher searcher = newIndexSearcher()) {
             return searcher.getEvents(minEventId, maxRecords);
         }
     }
@@ -401,16 +427,6 @@ public class JournalingPartition implements Partition {
             }
         }
         
-        if ( indexWriter != null ) {
-            try {
-                indexWriter.close();
-            } catch (final IOException ioe) {
-                logger.warn("Failed to close {} due to {}", indexWriter, ioe);
-                if ( logger.isDebugEnabled() ) {
-                    logger.warn("", ioe);
-                }
-            }
-        }
     }
     
     @Override
@@ -425,6 +441,6 @@ public class JournalingPartition implements Partition {
     
     @Override
     public String toString() {
-        return "Partition[section=" + sectionName + "]";
+        return "Partition[section=" + sectionIndex + "]";
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
index edbf75b..c0a56c4 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/PartitionManager.java
@@ -55,6 +55,19 @@ public interface PartitionManager {
     <T> Set<T> withEachPartition(PartitionAction<T> action) throws IOException;
     
     /**
+     * Performs the given Action on each partition and returns the set of results. Unlike
+     * {@link #withEachPartition(PartitionAction))}, this method does not use the thread pool
+     * in order to perform the request in parallel. This is desirable for very quick functions,
+     * as the thread pool can be fully utilized, resulting in a quick function taking far longer
+     * than it should.
+     * 
+     * @param action the action to perform
+     * @param writeAction specifies whether or not the action writes to the repository
+     * @return
+     */
+    <T> Set<T> withEachPartitionSerially(PartitionAction<T> action) throws IOException;
+    
+    /**
      * Performs the given Action to each partition, optionally waiting for the action to complete
      * @param action
      * @param writeAction

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
index 51d90e2..10af697 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/partition/QueuingPartitionManager.java
@@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.index.IndexManager;
 import org.apache.nifi.util.Tuple;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,7 +51,7 @@ public class QueuingPartitionManager implements PartitionManager {
     
     private final AtomicInteger blacklistedCount = new AtomicInteger(0);
     
-    public QueuingPartitionManager(final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
+    public QueuingPartitionManager(final IndexManager indexManager, final JournalingRepositoryConfig config, final ExecutorService executor) throws IOException {
         this.config = config;
         this.partitionQueue = new LinkedBlockingQueue<>(config.getPartitionCount());
         this.partitionArray = new JournalingPartition[config.getPartitionCount()];
@@ -64,7 +65,7 @@ public class QueuingPartitionManager implements PartitionManager {
             final Tuple<String, File> tuple = containerTuples.get(i % containerTuples.size());
             final File section = new File(tuple.getValue(), String.valueOf(i));
             
-            final JournalingPartition partition = new JournalingPartition(tuple.getKey(), String.valueOf(i), section, config, executor);
+            final JournalingPartition partition = new JournalingPartition(indexManager, tuple.getKey(), i, section, config, executor);
             partitionQueue.offer(partition);
             partitionArray[i] = partition;
         }
@@ -183,6 +184,17 @@ public class QueuingPartitionManager implements PartitionManager {
     }
     
     @Override
+    public <T> Set<T> withEachPartitionSerially(final PartitionAction<T> action) throws IOException {
+        // TODO: Do not use blacklisted partitions.
+        final Set<T> results = new HashSet<>(partitionArray.length);
+        for ( final Partition partition : partitionArray ) {
+            results.add( action.perform(partition) );
+        }
+        
+        return results;
+    }
+    
+    @Override
     public void withEachPartition(final VoidPartitionAction action, final boolean async) {
         // TODO: Do not use blacklisted partitions.
         final Map<Partition, Future<?>> futures = new HashMap<>(partitionArray.length);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
index c23a405..a6a487b 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/tasks/CompressionTask.java
@@ -133,7 +133,7 @@ public class CompressionTask implements Runnable {
             try (final JournalReader journalReader = new StandardJournalReader(journalFile);
                 final JournalWriter compressedWriter = new StandardJournalWriter(journalId, compressedFile, true, new StandardEventSerializer());
                 final TocReader tocReader = new StandardTocReader(tocFile);
-                final TocWriter compressedTocWriter = new StandardTocWriter(compressedTocFile, true)) {
+                final TocWriter compressedTocWriter = new StandardTocWriter(compressedTocFile, true, false)) {
                 
                 compress(journalReader, compressedWriter, tocReader, compressedTocWriter);
                 compressedWriter.sync();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/a5f557ad/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
index 6058282..fea6057 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/toc/StandardTocWriter.java
@@ -40,6 +40,7 @@ public class StandardTocWriter implements TocWriter {
     
     private final File file;
     private final FileOutputStream fos;
+    private final boolean alwaysSync;
     private int index = 0;
     
     /**
@@ -48,7 +49,7 @@ public class StandardTocWriter implements TocWriter {
      * @param compressionFlag whether or not the journal is compressed
      * @throws FileNotFoundException 
      */
-    public StandardTocWriter(final File file, final boolean compressionFlag) throws IOException {
+    public StandardTocWriter(final File file, final boolean compressionFlag, final boolean alwaysSync) throws IOException {
         if ( file.exists() ) {
             throw new FileAlreadyExistsException(file.getAbsolutePath());
         }
@@ -59,11 +60,17 @@ public class StandardTocWriter implements TocWriter {
         
         this.file = file;
         fos = new FileOutputStream(file);
-        
-        fos.write(VERSION);
-        fos.write(compressionFlag ? 1 : 0);
+        this.alwaysSync = alwaysSync;
+
+        final byte[] header = new byte[2];
+        header[0] = VERSION;
+        header[1] = (byte) (compressionFlag ? 1 : 0);
+        fos.write(header);
         fos.flush();
-        fos.getFD().sync();
+        
+        if ( alwaysSync ) {
+            fos.getFD().sync();
+        }
     }
     
     @Override
@@ -73,7 +80,9 @@ public class StandardTocWriter implements TocWriter {
         dos.writeLong(offset);
         dos.flush();
         
-        fos.getFD().sync();
+        if ( alwaysSync ) {
+            fos.getFD().sync();
+        }
     }
     
     @Override
@@ -83,6 +92,10 @@ public class StandardTocWriter implements TocWriter {
 
     @Override
     public void close() throws IOException {
+        if (alwaysSync) {
+            fos.getFD().sync();
+        }
+        
         fos.close();
     }
     


[05/12] incubator-nifi git commit: bug fixes and additional pieces of repo implemented

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
index f46b5fc..716c8b7 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java
@@ -152,19 +152,25 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
     private final AtomicBoolean repoDirty = new AtomicBoolean(false);
     // we keep the last 1000 records on hand so that when the UI is opened and it asks for the last 1000 records we don't need to 
     // read them. Since this is a very cheap operation to keep them, it's worth the tiny expense for the improved user experience.
-    private final RingBuffer<ProvenanceEventRecord> latestRecords = new RingBuffer<>(1000);
+    private final RingBuffer<StoredProvenanceEvent> latestRecords = new RingBuffer<>(1000);
     private EventReporter eventReporter;
+    private long rolloverCheckMillis = 10;
 
     public PersistentProvenanceRepository() throws IOException {
         this(createRepositoryConfiguration());
     }
 
     public PersistentProvenanceRepository(final RepositoryConfiguration configuration) throws IOException {
+        this(configuration, 10);
+    }
+    
+    PersistentProvenanceRepository(final RepositoryConfiguration configuration, final int rolloverCheckMillis) throws IOException {
         if (configuration.getStorageDirectories().isEmpty()) {
             throw new IllegalArgumentException("Must specify at least one storage directory");
         }
 
         this.configuration = configuration;
+        this.rolloverCheckMillis = rolloverCheckMillis;
 
         for (final File file : configuration.getStorageDirectories()) {
             final Path storageDirectory = file.toPath();
@@ -205,8 +211,9 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
         rolloverExecutor = Executors.newFixedThreadPool(numRolloverThreads, new NamedThreadFactory("Provenance Repository Rollover Thread"));
     }
 
+    
     @Override
-    public void initialize(final EventReporter eventReporter) throws IOException {
+    public synchronized void initialize(final EventReporter eventReporter) throws IOException {
         if (initialized.getAndSet(true)) {
             return;
         }
@@ -244,7 +251,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                         }
                     }
                 }
-            }, 10L, 10L, TimeUnit.SECONDS);
+            }, rolloverCheckMillis, rolloverCheckMillis, TimeUnit.MILLISECONDS);
 
             scheduledExecService.scheduleWithFixedDelay(new RemoveExpiredQueryResults(), 30L, 3L, TimeUnit.SECONDS);
             scheduledExecService.scheduleWithFixedDelay(new Runnable() {
@@ -350,13 +357,13 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
     }
 
     @Override
-    public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
+    public void registerEvents(final Collection<ProvenanceEventRecord> events) {
         persistRecord(events);
     }
 
     @Override
-    public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
-        final List<ProvenanceEventRecord> records = new ArrayList<>(maxRecords);
+    public List<StoredProvenanceEvent> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
+        final List<StoredProvenanceEvent> records = new ArrayList<>(maxRecords);
 
         final List<Path> paths = getPathsForId(firstRecordId);
         if (paths == null || paths.isEmpty()) {
@@ -368,7 +375,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 StandardProvenanceEventRecord record;
                 while (records.size() < maxRecords && ((record = reader.nextRecord()) != null)) {
                     if (record.getEventId() >= firstRecordId) {
-                        records.add(record);
+                        records.add(new IdEnrichedProvenanceEvent(record));
                     }
                 }
             } catch (final EOFException | FileNotFoundException fnfe) {
@@ -1133,7 +1140,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
         return mergedFile;
     }
 
-    static File mergeJournals(final List<File> journalFiles, final File storageDir, final File mergedFile, final EventReporter eventReporter, final RingBuffer<ProvenanceEventRecord> ringBuffer) throws IOException {
+    static File mergeJournals(final List<File> journalFiles, final File storageDir, final File mergedFile, final EventReporter eventReporter, final RingBuffer<StoredProvenanceEvent> ringBuffer) throws IOException {
         final long startNanos = System.nanoTime();
         if (journalFiles.isEmpty()) {
             return null;
@@ -1210,7 +1217,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                     final RecordReader reader = entry.getValue();
 
                     writer.writeRecord(record, record.getEventId());
-                    ringBuffer.add(record);
+                    ringBuffer.add(new IdEnrichedProvenanceEvent(record));
                     records++;
 
                     // Remove this entry from the map
@@ -1325,8 +1332,8 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1);
 
             if (latestRecords.getSize() >= query.getMaxResults()) {
-                final List<ProvenanceEventRecord> latestList = latestRecords.asList();
-                final List<ProvenanceEventRecord> trimmed;
+                final List<StoredProvenanceEvent> latestList = latestRecords.asList();
+                final List<StoredProvenanceEvent> trimmed;
                 if (latestList.size() > query.getMaxResults()) {
                     trimmed = latestList.subList(latestList.size() - query.getMaxResults(), latestList.size());
                 } else {
@@ -1335,7 +1342,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
 
                 final Long maxEventId = getMaxEventId();
                 if (maxEventId == null) {
-                    result.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0L);
+                    result.getResult().update(Collections.<StoredProvenanceEvent>emptyList(), 0L);
                 }
                 Long minIndexedId = indexConfig.getMinIdIndexed();
                 if (minIndexedId == null) {
@@ -1361,7 +1368,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
         querySubmissionMap.put(query.getIdentifier(), result);
 
         if (indexDirectories.isEmpty()) {
-            result.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0L);
+            result.getResult().update(Collections.<StoredProvenanceEvent>emptyList(), 0L);
         } else {
             for (final File indexDir : indexDirectories) {
                 queryExecService.submit(new QueryRunnable(query, result, indexDir, retrievalCount));
@@ -1582,7 +1589,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             if (event == null) {
                 final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
                 lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-                submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+                submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
                 return submission;
             }
 
@@ -1619,7 +1626,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             if (event == null) {
                 final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
                 lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-                submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+                submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
                 return submission;
             }
 
@@ -1661,12 +1668,21 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
     }
 
     @Override
-    public ProvenanceEventRecord getEvent(final long id) throws IOException {
-        final List<ProvenanceEventRecord> records = getEvents(id, 1);
+    public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
+        if ( !(location instanceof EventIdLocation) ) {
+            throw new IllegalArgumentException("Invalid StorageLocation for this repository");
+        }
+        
+        return getEvent(((EventIdLocation) location).getId());
+    }
+    
+    @Override
+    public StoredProvenanceEvent getEvent(final long id) throws IOException {
+        final List<StoredProvenanceEvent> records = getEvents(id, 1);
         if (records.isEmpty()) {
             return null;
         }
-        final ProvenanceEventRecord record = records.get(0);
+        final StoredProvenanceEvent record = records.get(0);
         if (record.getEventId() != id) {
             return null;
         }
@@ -1729,7 +1745,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             // get the max indexed event id
             final Long maxEventId = indexConfig.getMaxIdIndexed();
             if (maxEventId == null) {
-                submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+                submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList(), 0);
                 return;
             }
 
@@ -1743,7 +1759,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
                 }
                 final long totalNumDocs = maxEventId - minIndexedId;
 
-                final List<ProvenanceEventRecord> mostRecent = getEvents(startIndex, maxResults);
+                final List<StoredProvenanceEvent> mostRecent = getEvents(startIndex, maxResults);
                 submission.getResult().update(mostRecent, totalNumDocs);
             } catch (final IOException ioe) {
                 logger.error("Failed to retrieve records from Provenance Repository: " + ioe.toString());
@@ -1818,7 +1834,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             }
 
             try {
-                final Set<ProvenanceEventRecord> matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, indexDir, null, flowFileUuids);
+                final Set<StoredProvenanceEvent> matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, indexDir, null, flowFileUuids);
                 final StandardLineageResult result = submission.getResult();
                 result.update(matchingRecords);
 
@@ -1889,4 +1905,26 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
             return thread;
         }
     }
+
+    @Override
+    public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
+        final List<StoredProvenanceEvent> storedEvents = new ArrayList<>(storageLocations.size());
+        for ( final StorageLocation location : storageLocations ) {
+            final StoredProvenanceEvent event = getEvent(location);
+            if ( event != null ) {
+                storedEvents.add(event);
+            }
+        }
+        return storedEvents;
+    }
+
+    @Override
+    public Long getEarliestEventTime() throws IOException {
+        final List<StoredProvenanceEvent> firstEvents = getEvents(0, 1);
+        if ( firstEvents == null || firstEvents.isEmpty() ) {
+            return null;
+        }
+        
+        return firstEvents.get(0).getEventTime();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
index af5fe50..be1caf0 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java
@@ -28,22 +28,22 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.StandardProvenanceEventRecord;
-import org.apache.nifi.provenance.serialization.RecordReader;
-import org.apache.nifi.provenance.serialization.RecordReaders;
-
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
+import org.apache.nifi.provenance.IdEnrichedProvenanceEvent;
+import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
+import org.apache.nifi.provenance.serialization.RecordReader;
+import org.apache.nifi.provenance.serialization.RecordReaders;
 
 public class DocsReader {
 
     public DocsReader(final List<File> storageDirectories) {
     }
 
-    public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
+    public Set<StoredProvenanceEvent> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
         if (retrievalCount.get() >= maxResults) {
             return Collections.emptySet();
         }
@@ -59,13 +59,13 @@ public class DocsReader {
         return read(docs, allProvenanceLogFiles, retrievalCount, maxResults);
     }
 
-    public Set<ProvenanceEventRecord> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
+    public Set<StoredProvenanceEvent> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults) throws IOException {
         LuceneUtil.sortDocsForRetrieval(docs);
 
         RecordReader reader = null;
         String lastStorageFilename = null;
         long lastByteOffset = 0L;
-        final Set<ProvenanceEventRecord> matchingRecords = new LinkedHashSet<>();
+        final Set<StoredProvenanceEvent> matchingRecords = new LinkedHashSet<>();
 
         try {
             for (final Document d : docs) {
@@ -78,7 +78,8 @@ public class DocsReader {
                         try {
                             reader.skipTo(byteOffset);
                             final StandardProvenanceEventRecord record = reader.nextRecord();
-                            matchingRecords.add(record);
+                            matchingRecords.add(new IdEnrichedProvenanceEvent(record));
+                            
                             if (retrievalCount.incrementAndGet() >= maxResults) {
                                 break;
                             }
@@ -107,7 +108,7 @@ public class DocsReader {
                                 reader.skip(byteOffset);
 
                                 final StandardProvenanceEventRecord record = reader.nextRecord();
-                                matchingRecords.add(record);
+                                matchingRecords.add(new IdEnrichedProvenanceEvent(record));
                                 if (retrievalCount.incrementAndGet() >= maxResults) {
                                     break;
                                 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
index e2854c3..de7da44 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java
@@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.nifi.provenance.PersistentProvenanceRepository;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.StandardQueryResult;
-
+import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexNotFoundException;
 import org.apache.lucene.search.IndexSearcher;
@@ -53,7 +53,7 @@ public class IndexSearch {
         }
 
         final StandardQueryResult sqr = new StandardQueryResult(provenanceQuery, 1);
-        final Set<ProvenanceEventRecord> matchingRecords;
+        final Set<StoredProvenanceEvent> matchingRecords;
 
         try (final DirectoryReader directoryReader = DirectoryReader.open(FSDirectory.open(indexDirectory))) {
             final IndexSearcher searcher = new IndexSearcher(directoryReader);
@@ -65,7 +65,7 @@ public class IndexSearch {
 
             TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults());
             if (topDocs.totalHits == 0) {
-                sqr.update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+                sqr.update(Collections.<StoredProvenanceEvent>emptyList(), 0);
                 return sqr;
             }
 
@@ -76,7 +76,7 @@ public class IndexSearch {
             return sqr;
         } catch (final IndexNotFoundException e) {
             // nothing has been indexed yet.
-            sqr.update(Collections.<ProvenanceEventRecord>emptyList(), 0);
+            sqr.update(Collections.<StoredProvenanceEvent>emptyList(), 0);
             return sqr;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
index 54cde15..54407ed 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java
@@ -25,10 +25,6 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.nifi.provenance.PersistentProvenanceRepository;
-import org.apache.nifi.provenance.ProvenanceEventRecord;
-import org.apache.nifi.provenance.SearchableFields;
-
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
@@ -39,6 +35,9 @@ import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
+import org.apache.nifi.provenance.PersistentProvenanceRepository;
+import org.apache.nifi.provenance.SearchableFields;
+import org.apache.nifi.provenance.StoredProvenanceEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,7 +47,7 @@ public class LineageQuery {
     public static final int MAX_LINEAGE_UUIDS = 100;
     private static final Logger logger = LoggerFactory.getLogger(LineageQuery.class);
 
-    public static Set<ProvenanceEventRecord> computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final File indexDirectory, final String lineageIdentifier, final Collection<String> flowFileUuids) throws IOException {
+    public static Set<StoredProvenanceEvent> computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final File indexDirectory, final String lineageIdentifier, final Collection<String> flowFileUuids) throws IOException {
         if (requireNonNull(flowFileUuids).size() > MAX_LINEAGE_UUIDS) {
             throw new IllegalArgumentException(String.format("Cannot compute lineage for more than %s FlowFiles. This lineage contains %s.", MAX_LINEAGE_UUIDS, flowFileUuids.size()));
         }
@@ -97,7 +96,7 @@ public class LineageQuery {
             final long searchEnd = System.nanoTime();
 
             final DocsReader docsReader = new DocsReader(repo.getConfiguration().getStorageDirectories());
-            final Set<ProvenanceEventRecord> recs = docsReader.read(uuidQueryTopDocs, indexReader, repo.getAllLogFiles(), new AtomicInteger(0), Integer.MAX_VALUE);
+            final Set<StoredProvenanceEvent> recs = docsReader.read(uuidQueryTopDocs, indexReader, repo.getAllLogFiles(), new AtomicInteger(0), Integer.MAX_VALUE);
             final long readDocsEnd = System.nanoTime();
             logger.debug("Finished Lineage Query; Lucene search took {} millis, reading records took {} millis", TimeUnit.NANOSECONDS.toMillis(searchEnd - searchStart), TimeUnit.NANOSECONDS.toMillis(readDocsEnd - searchEnd));
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
index 7305872..84fd9ee 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
@@ -170,7 +170,7 @@ public class TestPersistentProvenanceRepository {
         config.setCompressOnRollover(false);
         config.setJournalCount(10);
         config.setQueryThreadPoolSize(10);
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final Map<String, String> attributes = new HashMap<>();
@@ -218,12 +218,12 @@ public class TestPersistentProvenanceRepository {
 
         System.out.println("Closing and re-initializing");
         repo.close();
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
         System.out.println("Re-initialized");
 
         final long fetchStart = System.nanoTime();
-        final List<ProvenanceEventRecord> records = repo.getEvents(0L, 1000000);
+        final List<StoredProvenanceEvent> records = repo.getEvents(0L, 1000000);
         final long fetchMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - fetchStart);
         assertEquals(1000000, records.size());
         final long fetchRecsPerMilli = 1000000 / fetchMillis;
@@ -238,7 +238,7 @@ public class TestPersistentProvenanceRepository {
         final RepositoryConfiguration config = createConfiguration();
         config.setMaxEventFileCapacity(1L);
         config.setMaxEventFileLife(1, TimeUnit.SECONDS);
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final Map<String, String> attributes = new HashMap<>();
@@ -262,9 +262,9 @@ public class TestPersistentProvenanceRepository {
         repo.close();
         Thread.sleep(500L); // Give the repo time to shutdown (i.e., close all file handles, etc.)
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
-        final List<ProvenanceEventRecord> recoveredRecords = repo.getEvents(0L, 12);
+        final List<StoredProvenanceEvent> recoveredRecords = repo.getEvents(0L, 12);
 
         assertEquals(10, recoveredRecords.size());
         for (int i = 0; i < 10; i++) {
@@ -285,7 +285,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setMaxEventFileLife(2, TimeUnit.SECONDS);
         config.setSearchableFields(searchableFields);
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final Map<String, String> attributes = new HashMap<>();
@@ -339,7 +339,7 @@ public class TestPersistentProvenanceRepository {
         final RepositoryConfiguration config = createConfiguration();
         config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -383,7 +383,7 @@ public class TestPersistentProvenanceRepository {
         final RepositoryConfiguration config = createConfiguration();
         config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
         config.setCompressOnRollover(true);
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -421,7 +421,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "10000000-0000-0000-0000-000000000000";
@@ -480,7 +480,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -543,7 +543,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -625,7 +625,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -681,7 +681,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000001";
@@ -736,7 +736,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000001";
@@ -795,7 +795,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String childId = "00000000-0000-0000-0000-000000000000";
@@ -845,7 +845,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String childId = "00000000-0000-0000-0000-000000000000";
@@ -892,7 +892,7 @@ public class TestPersistentProvenanceRepository {
     public void testCorrectProvenanceEventIdOnRestore() throws IOException {
         final RepositoryConfiguration config = createConfiguration();
         config.setMaxEventFileLife(1, TimeUnit.SECONDS);
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -918,7 +918,7 @@ public class TestPersistentProvenanceRepository {
 
         repo.close();
 
-        final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config);
+        final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config, 75);
         secondRepo.initialize(getEventReporter());
 
         final ProvenanceEventRecord event11 = builder.build();
@@ -938,7 +938,7 @@ public class TestPersistentProvenanceRepository {
         config.setMaxEventFileCapacity(1024L * 1024L);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
 
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -1002,7 +1002,7 @@ public class TestPersistentProvenanceRepository {
         final RepositoryConfiguration config = createConfiguration();
         config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
         config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
@@ -1070,7 +1070,7 @@ public class TestPersistentProvenanceRepository {
     public void testMergeJournals() throws IOException, InterruptedException {
         final RepositoryConfiguration config = createConfiguration();
         config.setMaxEventFileLife(3, TimeUnit.SECONDS);
-        repo = new PersistentProvenanceRepository(config);
+        repo = new PersistentProvenanceRepository(config, 75);
         repo.initialize(getEventReporter());
 
         final Map<String, String> attributes = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml
index 647c4ec..c3a3dd3 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-provenance-repository-nar/pom.xml
@@ -31,5 +31,9 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-volatile-provenance-repository</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-journaling-provenance-repository</artifactId>
+        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
index 777130e..a20ca75 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java
@@ -24,7 +24,6 @@ import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
@@ -119,7 +118,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
     @Override
     public void registerEvent(final ProvenanceEventRecord event) {
         final long id = idGenerator.getAndIncrement();
-        ringBuffer.add(new IdEnrichedProvEvent(event, id));
+        ringBuffer.add(new IdEnrichedProvenanceEvent(event, id));
     }
 
     @Override
@@ -413,11 +412,11 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
     public List<StoredProvenanceEvent> getEvents(final List<StorageLocation> storageLocations) throws IOException {
         final List<StoredProvenanceEvent> events = new ArrayList<>(storageLocations.size());
         for ( final StorageLocation location : storageLocations ) {
-            if ( !(location instanceof IdLocation) ) {
+            if ( !(location instanceof EventIdLocation) ) {
                 throw new IllegalArgumentException("Illegal Storage Location");
             }
             
-            final long id = ((IdLocation) location).getId();
+            final long id = ((EventIdLocation) location).getId();
             final StoredProvenanceEvent event = getEvent(id);
             if ( event != null ) {
                 events.add(event);
@@ -428,11 +427,11 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
 
     @Override
     public StoredProvenanceEvent getEvent(final StorageLocation location) throws IOException {
-        if ( !(location instanceof IdLocation) ) {
+        if ( !(location instanceof EventIdLocation) ) {
             throw new IllegalArgumentException("Illegal Storage Location");
         }
         
-        final long id = ((IdLocation) location).getId();
+        final long id = ((EventIdLocation) location).getId();
         return getEvent(id);
     }
 
@@ -452,7 +451,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         if (event == null) {
             final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.<String>emptyList(), 1);
             lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-            submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+            submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
             return submission;
         }
 
@@ -478,7 +477,7 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         if (event == null) {
             final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1);
             lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
-            submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList());
+            submission.getResult().update(Collections.<StoredProvenanceEvent>emptyList());
             return submission;
         }
 
@@ -614,187 +613,4 @@ public class VolatileProvenanceRepository implements ProvenanceEventRepository {
         }
     }
 
-    private static class IdEnrichedProvEvent implements StoredProvenanceEvent {
-
-        private final ProvenanceEventRecord record;
-        private final long id;
-
-        public IdEnrichedProvEvent(final ProvenanceEventRecord record, final long id) {
-            this.record = record;
-            this.id = id;
-        }
-
-        @Override
-        public long getEventId() {
-            return id;
-        }
-
-        @Override
-        public long getEventTime() {
-            return record.getEventTime();
-        }
-
-        @Override
-        public long getFlowFileEntryDate() {
-            return record.getFlowFileEntryDate();
-        }
-
-        @Override
-        public long getLineageStartDate() {
-            return record.getLineageStartDate();
-        }
-
-        @Override
-        public Set<String> getLineageIdentifiers() {
-            return record.getLineageIdentifiers();
-        }
-
-        @Override
-        public long getFileSize() {
-            return record.getFileSize();
-        }
-
-        @Override
-        public Long getPreviousFileSize() {
-            return record.getPreviousFileSize();
-        }
-
-        @Override
-        public long getEventDuration() {
-            return record.getEventDuration();
-        }
-
-        @Override
-        public ProvenanceEventType getEventType() {
-            return record.getEventType();
-        }
-
-        @Override
-        public Map<String, String> getAttributes() {
-            return record.getAttributes();
-        }
-
-        @Override
-        public Map<String, String> getPreviousAttributes() {
-            return record.getPreviousAttributes();
-        }
-
-        @Override
-        public Map<String, String> getUpdatedAttributes() {
-            return record.getUpdatedAttributes();
-        }
-
-        @Override
-        public String getComponentId() {
-            return record.getComponentId();
-        }
-
-        @Override
-        public String getComponentType() {
-            return record.getComponentType();
-        }
-
-        @Override
-        public String getTransitUri() {
-            return record.getTransitUri();
-        }
-
-        @Override
-        public String getSourceSystemFlowFileIdentifier() {
-            return record.getSourceSystemFlowFileIdentifier();
-        }
-
-        @Override
-        public String getFlowFileUuid() {
-            return record.getFlowFileUuid();
-        }
-
-        @Override
-        public List<String> getParentUuids() {
-            return record.getParentUuids();
-        }
-
-        @Override
-        public List<String> getChildUuids() {
-            return record.getChildUuids();
-        }
-
-        @Override
-        public String getAlternateIdentifierUri() {
-            return record.getAlternateIdentifierUri();
-        }
-
-        @Override
-        public String getDetails() {
-            return record.getDetails();
-        }
-
-        @Override
-        public String getRelationship() {
-            return record.getRelationship();
-        }
-
-        @Override
-        public String getSourceQueueIdentifier() {
-            return record.getSourceQueueIdentifier();
-        }
-
-        @Override
-        public String getContentClaimSection() {
-            return record.getContentClaimSection();
-        }
-
-        @Override
-        public String getPreviousContentClaimSection() {
-            return record.getPreviousContentClaimSection();
-        }
-
-        @Override
-        public String getContentClaimContainer() {
-            return record.getContentClaimContainer();
-        }
-
-        @Override
-        public String getPreviousContentClaimContainer() {
-            return record.getPreviousContentClaimContainer();
-        }
-
-        @Override
-        public String getContentClaimIdentifier() {
-            return record.getContentClaimIdentifier();
-        }
-
-        @Override
-        public String getPreviousContentClaimIdentifier() {
-            return record.getPreviousContentClaimIdentifier();
-        }
-
-        @Override
-        public Long getContentClaimOffset() {
-            return record.getContentClaimOffset();
-        }
-
-        @Override
-        public Long getPreviousContentClaimOffset() {
-            return record.getPreviousContentClaimOffset();
-        }
-
-        @Override
-        public StorageLocation getStorageLocation() {
-            return new IdLocation(getEventId());
-        }
-    }
-    
-    private static class IdLocation implements StorageLocation {
-        private final long id;
-        
-        public IdLocation(final long id) {
-            this.id = id;
-        }
-        
-        public long getId() {
-            return id;
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/b95e7569/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml
index 8a835d6..d7bfba7 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/pom.xml
@@ -25,6 +25,7 @@
     <modules>
         <module>nifi-persistent-provenance-repository</module>
         <module>nifi-volatile-provenance-repository</module>
+        <module>nifi-journaling-provenance-repository</module>
         <module>nifi-provenance-repository-nar</module>
     </modules>
     <dependencyManagement>
@@ -39,6 +40,11 @@
                 <artifactId>nifi-volatile-provenance-repository</artifactId>
                 <version>0.0.2-incubating-SNAPSHOT</version>
             </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-journaling-provenance-repository</artifactId>
+                <version>0.0.2-incubating-SNAPSHOT</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 </project>


[10/12] incubator-nifi git commit: NIFI-388: Initial implementation of prov repo; not yet finished but pushing so that the code is not lost

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
index 18871c7..c1a7de8 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/config/JournalingRepositoryConfig.java
@@ -28,6 +28,8 @@ import org.apache.nifi.provenance.search.SearchableField;
 
 public class JournalingRepositoryConfig {
     private Map<String, File> containers = new HashMap<>();
+    private Map<String, Long> containerCapacities = new HashMap<>();
+    
     private long expirationMillis = TimeUnit.MILLISECONDS.convert(24, TimeUnit.HOURS);
     private long storageCapacity = 1024L * 1024L * 1024L;   // 1 GB
     private long rolloverMillis = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES);
@@ -35,13 +37,16 @@ public class JournalingRepositoryConfig {
     private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB
     private int partitionCount = 16;
     private int blockSize = 5000;
-    private int indexesPerContainer = 2;
-
+    private int indexesPerContainer = 1;
+    private long expirationFrequency = TimeUnit.MINUTES.toNanos(2L);
+    
     private List<SearchableField> searchableFields = new ArrayList<>();
     private List<SearchableField> searchableAttributes = new ArrayList<>();
     private boolean compress = true;
     private boolean alwaysSync = false;
-    private int threadPoolSize = 4;
+    private int workerThreadPoolSize = 2;
+    private int queryThreadPoolSize = 2;
+    private int compressionThreadPoolSize = 1;
     private boolean readOnly = false;
 
     public void setReadOnly(final boolean readOnly) {
@@ -74,6 +79,27 @@ public class JournalingRepositoryConfig {
         this.containers = new HashMap<>(containers);
     }
 
+    
+    public long getMaxCapacity(final String containerName) {
+        final Long maxCapacity = containerCapacities.get(containerName);
+        if ( maxCapacity == null ) {
+            return getMaxStorageCapacity();
+        } else {
+            return maxCapacity;
+        }
+    }
+
+    public void setMaxContainerCapacities(final Map<String, Long> containerCapacities) {
+        this.containerCapacities = new HashMap<>(containerCapacities);
+    }
+
+    public void setMaxContainerCapacity(final String containerName, final long maxCapacity) {
+        if ( maxCapacity < 1 ) {
+            throw new IllegalArgumentException("Cannot set max container capacity for " + containerName + " to " + maxCapacity + " bytes");
+        }
+
+        this.containerCapacities.put(containerName, maxCapacity);
+    }
     /**
      * Returns the maximum amount of time that a given record will stay in the
      * repository
@@ -207,15 +233,28 @@ public class JournalingRepositoryConfig {
         this.compress = compress;
     }
 
-    public int getThreadPoolSize() {
-        return threadPoolSize;
+    public int getWorkerThreadPoolSize() {
+        return workerThreadPoolSize;
+    }
+
+    public void setWorkerThreadPoolSize(final int workerThreadPoolSize) {
+        if (workerThreadPoolSize < 1) {
+            throw new IllegalArgumentException();
+        }
+        this.workerThreadPoolSize = workerThreadPoolSize;
     }
+    
+    
 
-    public void setThreadPoolSize(final int queryThreadPoolSize) {
+    public int getQueryThreadPoolSize() {
+        return queryThreadPoolSize;
+    }
+
+    public void setQueryThreadPoolSize(int queryThreadPoolSize) {
         if (queryThreadPoolSize < 1) {
             throw new IllegalArgumentException();
         }
-        this.threadPoolSize = queryThreadPoolSize;
+        this.queryThreadPoolSize = queryThreadPoolSize;
     }
 
     /**
@@ -308,7 +347,7 @@ public class JournalingRepositoryConfig {
      * Returns the minimum number of Provenance Events that should be written to a single Block.
      * Events are written out in blocks, which are later optionally compressed. A larger block size
      * will potentially result in better compression. However, a smaller block size will result
-     * in better performance when reading the data. The default value is 100 events per block.
+     * in better performance when reading the data. The default value is 5000 events per block.
      * 
      * @return
      */
@@ -320,7 +359,7 @@ public class JournalingRepositoryConfig {
      * Sets the minimum number of Provenance Events that should be written to a single Block.
      * Events are written out in blocks, which are later optionally compressed. A larger block size
      * will potentially result in better compression. However, a smaller block size will result
-     * in better performance when reading the data. The default value is 100 events per block.
+     * in better performance when reading the data. The default value is 5000 events per block.
      * 
      * @return
      */
@@ -330,4 +369,20 @@ public class JournalingRepositoryConfig {
         }
         this.blockSize = blockSize;
     }
+
+    public int getCompressionThreadPoolSize() {
+        return compressionThreadPoolSize;
+    }
+
+    public void setCompressionThreadPoolSize(final int compressionThreadPoolSize) {
+        this.compressionThreadPoolSize = compressionThreadPoolSize;
+    }
+    
+    public long getExpirationFrequency(final TimeUnit unit) {
+        return unit.convert(expirationFrequency, TimeUnit.NANOSECONDS);
+    }
+    
+    public void setExpirationFrequency(final long period, final TimeUnit unit) {
+        this.expirationFrequency = unit.toNanos(period);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
index 753378d..85e02c0 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexSearcher.java
@@ -18,6 +18,7 @@ package org.apache.nifi.provenance.journaling.index;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
@@ -32,7 +33,53 @@ public interface EventIndexSearcher extends Closeable {
      */
     SearchResult search(Query query) throws IOException;
     
+    /**
+     * Returns the locations of all events for a FlowFile that has a FlowFile UUID in the collection of
+     * UUIDs provided, if the event time occurs between earliestTime and latestTime. The return value is 
+     * ordered in the order in which the records should be read from the journals in order to obtain 
+     * maximum efficiency
+     * 
+     * @param flowFileUuids
+     * @param earliestTime
+     * @param latestTime
+     * 
+     * @return
+     * @throws IOException
+     */
+    List<JournaledStorageLocation> getEventsForFlowFiles(Collection<String> flowFileUuids, long earliestTime, long latestTime) throws IOException;
+    
+    /**
+     * Returns the locations of events that have Event ID's at least equal to minEventId, and returns
+     * up to the given number of results
+     * 
+     * @param minEventId
+     * @param maxResults
+     * @return
+     * @throws IOException
+     */
     List<JournaledStorageLocation> getEvents(long minEventId, int maxResults) throws IOException;
     
+    /**
+     * Returns the largest event id that is known by the index being searched
+     * @param container
+     * @param section
+     * @return
+     * @throws IOException
+     */
     Long getMaxEventId(String container, String section) throws IOException;
+    
+    /**
+     * Returns the locations of the latest events for the index being searched
+     * @param numEvents
+     * @return
+     * @throws IOException
+     */
+    List<JournaledStorageLocation> getLatestEvents(int numEvents) throws IOException;
+    
+    /**
+     * Returns the total number of events that exist for the index being searched
+     * @return
+     * @throws IOException
+     */
+    long getNumberOfEvents() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
index 1f231e9..a151838 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/EventIndexWriter.java
@@ -44,5 +44,23 @@ public interface EventIndexWriter extends Closeable {
      * @param journalId
      * @throws IOException
      */
-    void delete(String containerName, String section, String journalId) throws IOException;
+    void delete(String containerName, String section, Long journalId) throws IOException;
+    
+    /**
+     * Deletes any records that belong to the given container and section but have a journal Id less
+     * than the specified value
+     * @param containerName
+     * @param section
+     * @param journalId
+     * @throws IOException
+     */
+    void deleteEventsBefore(String containerName, String section, Long journalId) throws IOException;
+    
+    
+    /**
+     * Removes all events from the index that occurred before the given time
+     * @param earliestEventTimeToDelete
+     * @throws IOException
+     */
+    void deleteOldEvents(long earliestEventTimeToDelete) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexAction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexAction.java
new file mode 100644
index 0000000..6486d56
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexAction.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.IOException;
+
+public interface IndexAction<T> {
+    T perform(EventIndexSearcher searcher) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
index 141b84a..34d1b18 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/IndexManager.java
@@ -17,7 +17,9 @@
 package org.apache.nifi.provenance.journaling.index;
 
 import java.io.Closeable;
+import java.io.File;
 import java.io.IOException;
+import java.util.Set;
 
 public interface IndexManager extends Closeable {
 
@@ -37,5 +39,112 @@ public interface IndexManager extends Closeable {
      */
     Long getMaxEventId(String container, String section) throws IOException;
     
+    /**
+     * Returns the total number of indices for all containers
+     * @return
+     */
+    int getNumberOfIndices();
+    
+    /**
+     * Returns a new {@link EventIndexSearcher} that can be used to search the indices for
+     * the given container
+     * 
+     * @param containerName the containerName to search
+     * 
+     * @return
+     * @throws IOException
+     */
     EventIndexSearcher newIndexSearcher(String containerName) throws IOException;
+    
+    /**
+     * Executes the given action against each index and returns a Set of results,
+     * where each result is obtained from performing the given action against one index
+     * 
+     * @param action the action to perform
+     * @return
+     * @throws IOException
+     */
+    <T> Set<T> withEachIndex(IndexAction<T> action) throws IOException;
+    
+    /**
+     * Performs the given action against each index, waiting for the action to complete
+     * against each index before returning
+     * 
+     * @param action the action to perform against each index
+     * @throws IOException
+     */
+    void withEachIndex(VoidIndexAction action) throws IOException;
+    
+    /**
+     * Performs the given action against each index
+     * 
+     * @param action the action to perform
+     * 
+     * @param async if true, the method will return immediatley and the actions will occur
+     * in the background. If <code>false</code>, the method will not return until the action
+     * has been performed against all indices
+     * 
+     * @throws IOException
+     */
+    void withEachIndex(VoidIndexAction action, boolean async) throws IOException;
+    
+    /**
+     * Removes any events that have a Storage Location that includes the provided containerName, secitonIndex, and journalId,
+     * and then re-adds all of the events that are in the given journalFile.
+     * @param containerName
+     * @param sectionIndex
+     * @param journalId
+     * @param journalFile
+     * @throws IOException
+     */
+    void reindex(final String containerName, final int sectionIndex, final Long journalId, final File journalFile) throws IOException;
+    
+    /**
+     * Syncs all indices
+     * @throws IOException
+     */
+    void sync() throws IOException;
+    
+    /**
+     * Returns the total number of events in all indices
+     * @return
+     */
+    long getNumberOfEvents() throws IOException;
+    
+    /**
+     * Removes all events from all indices that occurred before the given time
+     * @param earliestEventTimeToDelete
+     * @throws IOException
+     */
+    void deleteOldEvents(long earliestEventTimeToDelete) throws IOException;
+    
+    /**
+     * Deletes any events from the index that belong to the given container, section, and journal id
+     * 
+     * @param containerName
+     * @param sectionIndex
+     * @param journalId
+     * 
+     * @throws IOException
+     */
+    void deleteEvents(String containerName, int sectionIndex, Long journalId) throws IOException;
+    
+    /**
+     * Deletes any events from the index that belong to the given container and section but have
+     * a journal id before the value specified
+     * 
+     * @param containerName
+     * @param sectionIndex
+     * @param journalId
+     * 
+     * @throws IOException
+     */
+    void deleteEventsBefore(String containerName, int sectionIndex, Long journalId) throws IOException;
+    
+    /**
+     * Returns the size (in bytes) of the index for the given container
+     * @param containerName
+     * @return
+     */
+    long getSize(String containerName);
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
index d10fd00..e212342 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexManager.java
@@ -20,13 +20,21 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.nifi.provenance.journaling.JournaledProvenanceEvent;
 import org.apache.nifi.provenance.journaling.config.JournalingRepositoryConfig;
+import org.apache.nifi.provenance.journaling.toc.TocJournalReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,14 +42,14 @@ public class LuceneIndexManager implements IndexManager {
     private static final Logger logger = LoggerFactory.getLogger(LuceneIndexManager.class);
     
     private final JournalingRepositoryConfig config;
-    private final ScheduledExecutorService executor;
+    private final ExecutorService queryExecutor;
     
     private final Map<String, List<LuceneIndexWriter>> writers = new HashMap<>();
     private final Map<String, AtomicLong> writerIndexes = new HashMap<>();
     
-    public LuceneIndexManager(final JournalingRepositoryConfig config, final ScheduledExecutorService executor) throws IOException {
+    public LuceneIndexManager(final JournalingRepositoryConfig config, final ScheduledExecutorService workerExecutor, final ExecutorService queryExecutor) throws IOException {
         this.config = config;
-        this.executor = executor;
+        this.queryExecutor = queryExecutor;
         
         final int rolloverSeconds = (int) config.getJournalRolloverPeriod(TimeUnit.SECONDS);
         if ( !config.isReadOnly() ) {
@@ -58,7 +66,7 @@ public class LuceneIndexManager implements IndexManager {
                     writerList.add(new LuceneIndexWriter(indexDir, config));
                 }
                 
-                executor.scheduleWithFixedDelay(new Runnable() {
+                workerExecutor.scheduleWithFixedDelay(new Runnable() {
                     @Override
                     public void run() {
                         try {
@@ -150,6 +158,15 @@ public class LuceneIndexManager implements IndexManager {
         return max;
     }
 
+    @Override
+    public void sync() throws IOException {
+        for ( final List<LuceneIndexWriter> writerList : writers.values() ) {
+            for ( final LuceneIndexWriter writer : writerList ) {
+                writer.sync();
+            }
+        }
+    }
+    
     
     private void sync(final String containerName) throws IOException {
         final AtomicLong index = writerIndexes.get(containerName);
@@ -175,4 +192,190 @@ public class LuceneIndexManager implements IndexManager {
             }
         }        
     }
+    
+    @Override
+    public <T> Set<T> withEachIndex(final IndexAction<T> action) throws IOException {
+        final Set<T> results = new HashSet<>();
+        final Map<String, Future<T>> futures = new HashMap<>();
+        final Set<String> containerNames = config.getContainers().keySet();
+        for (final String containerName : containerNames) {
+            final Callable<T> callable = new Callable<T>() {
+                @Override
+                public T call() throws Exception {
+                    try (final EventIndexSearcher searcher = newIndexSearcher(containerName)) {
+                        return action.perform(searcher);
+                    }
+                }
+            };
+            
+            final Future<T> future = queryExecutor.submit(callable);
+            futures.put(containerName, future);
+        }
+        
+        for ( final Map.Entry<String, Future<T>> entry : futures.entrySet() ) {
+            try {
+                final T result = entry.getValue().get();
+                results.add(result);
+            } catch (final ExecutionException ee) {
+                final Throwable cause = ee.getCause();
+                if ( cause instanceof IOException ) {
+                    throw (IOException) cause;
+                } else {
+                    throw new RuntimeException("Failed to query Container " + entry.getKey() + " due to " + cause, cause);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        
+        return results;
+    }
+    
+    @Override
+    public void withEachIndex(final VoidIndexAction action) throws IOException {
+        withEachIndex(action, false);
+    }
+    
+    @Override
+    public void withEachIndex(final VoidIndexAction action, final boolean async) throws IOException {
+        final Map<String, Future<?>> futures = new HashMap<>();
+        final Set<String> containerNames = config.getContainers().keySet();
+
+        for (final String containerName : containerNames) {
+            final Callable<Object> callable = new Callable<Object>() {
+                @Override
+                public Object call() throws IOException {
+                    try (final EventIndexSearcher searcher = newIndexSearcher(containerName)) {
+                        action.perform(searcher);
+                        return null;
+                    } catch (final Throwable t) {
+                        if ( async ) {
+                            logger.error("Failed to perform action against container " + containerName + " due to " + t, t);
+                            if ( logger.isDebugEnabled() ) {
+                                logger.error("", t);
+                            }
+                            
+                            return null;
+                        } else {
+                            throw new IOException("Failed to perform action against container " + containerName + " due to " + t, t);
+                        }
+                    }
+                }
+            };
+            
+            final Future<?> future = queryExecutor.submit(callable);
+            futures.put(containerName, future);
+        }
+        
+        if ( !async ) {
+            for ( final Map.Entry<String, Future<?>> entry : futures.entrySet() ) {
+                try {
+                    // throw any exception thrown by runnable
+                    entry.getValue().get();
+                } catch (final ExecutionException ee) {
+                    final Throwable cause = ee.getCause();
+                    if ( cause instanceof IOException ) {
+                        throw ((IOException) cause);
+                    }
+                    
+                    throw new RuntimeException("Failed to query Partition " + entry.getKey() + " due to " + cause, cause);
+                } catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        }
+    }
+    
+    @Override
+    public int getNumberOfIndices() {
+        return config.getContainers().size();
+    }
+    
+    @Override
+    public void deleteEvents(final String containerName, final int sectionIndex, final Long journalId) throws IOException {
+        final List<LuceneIndexWriter> writerList = writers.get(containerName);
+        for ( final LuceneIndexWriter writer : writerList ) {
+            writer.delete(containerName, String.valueOf(sectionIndex), journalId);
+        }
+    }
+    
+    @Override
+    public void deleteEventsBefore(final String containerName, final int sectionIndex, final Long journalId) throws IOException {
+        final List<LuceneIndexWriter> writerList = writers.get(containerName);
+        for ( final LuceneIndexWriter writer : writerList ) {
+            writer.deleteEventsBefore(containerName, String.valueOf(sectionIndex), journalId);
+        }
+    }
+    
+    @Override
+    public void reindex(final String containerName, final int sectionIndex, final Long journalId, final File journalFile) throws IOException {
+        deleteEvents(containerName, sectionIndex, journalId);
+        
+        final LuceneIndexWriter writer = getIndexWriter(containerName);
+        try (final TocJournalReader reader = new TocJournalReader(containerName, String.valueOf(sectionIndex), journalId, journalFile)) {
+            final List<JournaledProvenanceEvent> events = new ArrayList<>(1000);
+            JournaledProvenanceEvent event;
+            
+            while ((event = reader.nextJournaledEvent()) != null) {
+                events.add(event);
+                if ( events.size() >= 1000 ) {
+                    writer.index(events);
+                    events.clear();
+                }
+            }
+            
+            if (!events.isEmpty() ) {
+                writer.index(events);
+            }
+        }
+    }
+    
+    @Override
+    public long getNumberOfEvents() throws IOException {
+        final AtomicLong totalCount = new AtomicLong(0L);
+        withEachIndex(new VoidIndexAction() {
+            @Override
+            public void perform(final EventIndexSearcher searcher) throws IOException {
+                totalCount.addAndGet(searcher.getNumberOfEvents());
+            }
+        });
+        
+        return totalCount.get();
+    }
+    
+    @Override
+    public void deleteOldEvents(final long earliestEventTimeToDelete) throws IOException {
+        for ( final String containerName : config.getContainers().keySet() ) {
+            final List<LuceneIndexWriter> writerList = writers.get(containerName);
+            for ( final LuceneIndexWriter writer : writerList ) {
+                writer.deleteOldEvents(earliestEventTimeToDelete);
+            }
+        }
+    }
+    
+    
+    @Override
+    public long getSize(final String containerName) {
+        final File containerFile = config.getContainers().get(containerName);
+        final File indicesDir = new File(containerFile, "indices");
+        
+        return getSize(indicesDir);
+    }
+    
+    private long getSize(final File file) {
+        if ( file.isDirectory() ) {
+            long totalSize = 0L;
+            
+            final File[] children = file.listFiles();
+            if ( children != null ) {
+                for ( final File child : children ) {
+                    totalSize += getSize(child);
+                }
+            }
+            
+            return totalSize;
+        } else {
+            return file.length();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
index a9dd1a5..94bd3f8 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexSearcher.java
@@ -19,6 +19,7 @@ package org.apache.nifi.provenance.journaling.index;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
@@ -27,6 +28,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Sort;
@@ -34,7 +36,9 @@ import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.SortField.Type;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopFieldDocs;
 import org.apache.lucene.store.FSDirectory;
+import org.apache.nifi.provenance.SearchableFields;
 import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
 import org.apache.nifi.provenance.search.Query;
 
@@ -43,16 +47,20 @@ public class LuceneIndexSearcher implements EventIndexSearcher {
     private final IndexSearcher searcher;
     private final FSDirectory fsDirectory;
     
+    private final String description;
+    
     public LuceneIndexSearcher(final File indexDirectory) throws IOException {
         this.fsDirectory = FSDirectory.open(indexDirectory);
         this.reader = DirectoryReader.open(fsDirectory);
         this.searcher = new IndexSearcher(reader);
+        this.description = "LuceneIndexSearcher[indexDirectory=" + indexDirectory + "]";
     }
     
-    public LuceneIndexSearcher(final DirectoryReader reader) {
+    public LuceneIndexSearcher(final DirectoryReader reader, final File indexDirectory) {
         this.reader = reader;
         this.searcher = new IndexSearcher(reader);
         this.fsDirectory = null;
+        this.description = "LuceneIndexSearcher[indexDirectory=" + indexDirectory + "]";
     }
     
     @Override
@@ -76,29 +84,35 @@ public class LuceneIndexSearcher implements EventIndexSearcher {
     private JournaledStorageLocation createLocation(final Document document) {
         final String containerName = document.get(IndexedFieldNames.CONTAINER_NAME);
         final String sectionName = document.get(IndexedFieldNames.SECTION_NAME);
-        final String journalId = document.get(IndexedFieldNames.JOURNAL_ID);
+        final long journalId = document.getField(IndexedFieldNames.JOURNAL_ID).numericValue().longValue();
         final int blockIndex = document.getField(IndexedFieldNames.BLOCK_INDEX).numericValue().intValue();
         final long eventId = document.getField(IndexedFieldNames.EVENT_ID).numericValue().longValue();
         
         return new JournaledStorageLocation(containerName, sectionName, journalId, blockIndex, eventId);
     }
     
-    private List<JournaledStorageLocation> getLocations(final TopDocs topDocs) throws IOException {
+    private List<JournaledStorageLocation> getOrderedLocations(final TopDocs topDocs) throws IOException {
         final ScoreDoc[] scoreDocs = topDocs.scoreDocs;
         final List<JournaledStorageLocation> locations = new ArrayList<>(scoreDocs.length);
-        for ( final ScoreDoc scoreDoc : scoreDocs ) {
+        populateLocations(topDocs, locations);
+        
+        return locations;
+    }
+    
+    
+    private void populateLocations(final TopDocs topDocs, final Collection<JournaledStorageLocation> locations) throws IOException {
+        for ( final ScoreDoc scoreDoc : topDocs.scoreDocs ) {
             final Document document = reader.document(scoreDoc.doc);
             locations.add(createLocation(document));
         }
-        
-        return locations;
     }
     
+    
     @Override
     public SearchResult search(final Query provenanceQuery) throws IOException {
         final org.apache.lucene.search.Query luceneQuery = QueryUtils.convertQueryToLucene(provenanceQuery);
         final TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults());
-        final List<JournaledStorageLocation> locations = getLocations(topDocs);
+        final List<JournaledStorageLocation> locations = getOrderedLocations(topDocs);
         
         return new SearchResult(locations, topDocs.totalHits);
     }
@@ -109,7 +123,7 @@ public class LuceneIndexSearcher implements EventIndexSearcher {
         query.add(NumericRangeQuery.newLongRange(IndexedFieldNames.EVENT_ID, minEventId, null, true, true), Occur.MUST);
         
         final TopDocs topDocs = searcher.search(query, maxResults, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG)));
-        return getLocations(topDocs);
+        return getOrderedLocations(topDocs);
     }
 
     @Override
@@ -125,7 +139,7 @@ public class LuceneIndexSearcher implements EventIndexSearcher {
         }
         
         final TopDocs topDocs = searcher.search(query, 1, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG, true)));
-        final List<JournaledStorageLocation> locations = getLocations(topDocs);
+        final List<JournaledStorageLocation> locations = getOrderedLocations(topDocs);
         if ( locations.isEmpty() ) {
             return null;
         }
@@ -133,4 +147,45 @@ public class LuceneIndexSearcher implements EventIndexSearcher {
         return locations.get(0).getEventId();
     }
 
+    @Override
+    public List<JournaledStorageLocation> getEventsForFlowFiles(final Collection<String> flowFileUuids, final long earliestTime, final long latestTime) throws IOException {
+        // Create a query for all Events related to the FlowFiles of interest. We do this by adding all ID's as
+        // "SHOULD" clauses and then setting the minimum required to 1.
+        final BooleanQuery flowFileIdQuery;
+        if (flowFileUuids == null || flowFileUuids.isEmpty()) {
+            flowFileIdQuery = null;
+        } else {
+            flowFileIdQuery = new BooleanQuery();
+            for (final String flowFileUuid : flowFileUuids) {
+                flowFileIdQuery.add(new TermQuery(new Term(SearchableFields.FlowFileUUID.getSearchableFieldName(), flowFileUuid)), Occur.SHOULD);
+            }
+            flowFileIdQuery.setMinimumNumberShouldMatch(1);
+        }
+        
+        flowFileIdQuery.add(NumericRangeQuery.newLongRange(SearchableFields.EventTime.getSearchableFieldName(), 
+                earliestTime, latestTime, true, true), Occur.MUST);
+        
+        final TopDocs topDocs = searcher.search(flowFileIdQuery, 1000);
+        return getOrderedLocations(topDocs);
+    }
+    
+    
+    @Override
+    public List<JournaledStorageLocation> getLatestEvents(final int numEvents) throws IOException {
+        final MatchAllDocsQuery query = new MatchAllDocsQuery();
+        
+        final TopFieldDocs topDocs = searcher.search(query, numEvents, new Sort(new SortField(IndexedFieldNames.EVENT_ID, Type.LONG, true)));
+        final List<JournaledStorageLocation> locations = getOrderedLocations(topDocs);
+        return locations;
+    }
+    
+    @Override
+    public String toString() {
+        return description;
+    }
+
+    @Override
+    public long getNumberOfEvents() {
+        return reader.numDocs();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
index b61ad34..d1b2c0e 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/LuceneIndexWriter.java
@@ -21,10 +21,12 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -33,6 +35,7 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.LongField;
 import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.ConcurrentMergeScheduler;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
@@ -40,6 +43,8 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
@@ -61,6 +66,7 @@ public class LuceneIndexWriter implements EventIndexWriter {
     private final JournalingRepositoryConfig config;
     private final Set<SearchableField> nonAttributeSearchableFields;
     private final Set<SearchableField> attributeSearchableFields;
+    private final File indexDir;
     
     private final Directory directory;
     private final Analyzer analyzer;
@@ -68,6 +74,7 @@ public class LuceneIndexWriter implements EventIndexWriter {
     private final AtomicLong indexMaxId = new AtomicLong(-1L);
     
     public LuceneIndexWriter(final File indexDir, final JournalingRepositoryConfig config) throws IOException {
+        this.indexDir = indexDir;
         this.config = config;
         
         attributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(config.getSearchableAttributes()));
@@ -76,12 +83,20 @@ public class LuceneIndexWriter implements EventIndexWriter {
         directory = FSDirectory.open(indexDir);
         analyzer = new StandardAnalyzer();
         final IndexWriterConfig writerConfig = new IndexWriterConfig(Version.LATEST, analyzer);
+        // Increase number of concurrent merges since we are on SSD:
+        final ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler();
+        writerConfig.setMergeScheduler(cms);
+        final int mergeThreads = Math.max(2, Math.min(4, config.getWorkerThreadPoolSize() / 2));
+        cms.setMaxMergesAndThreads(mergeThreads, mergeThreads);
+        
         indexWriter = new IndexWriter(directory, writerConfig);
     }
     
     public EventIndexSearcher newIndexSearcher() throws IOException {
+        logger.trace("Creating index searcher for {}", indexWriter);
+        
         final DirectoryReader reader = DirectoryReader.open(indexWriter, false);
-        return new LuceneIndexSearcher(reader);
+        return new LuceneIndexSearcher(reader, indexDir);
     }
 
     @Override
@@ -119,6 +134,8 @@ public class LuceneIndexWriter implements EventIndexWriter {
     public void index(final Collection<JournaledProvenanceEvent> events) throws IOException {
         long maxId = this.indexMaxId.get();
         
+        final long startNanos = System.nanoTime();
+        
         final List<Document> documents = new ArrayList<>(events.size());
         for ( final JournaledProvenanceEvent event : events ) {
             maxId = event.getEventId();
@@ -154,7 +171,7 @@ public class LuceneIndexWriter implements EventIndexWriter {
             final JournaledStorageLocation location = event.getStorageLocation();
             doc.add(new StringField(IndexedFieldNames.CONTAINER_NAME, location.getContainerName(), Store.YES));
             doc.add(new StringField(IndexedFieldNames.SECTION_NAME, location.getSectionName(), Store.YES));
-            doc.add(new StringField(IndexedFieldNames.JOURNAL_ID, location.getJournalId(), Store.YES));
+            doc.add(new LongField(IndexedFieldNames.JOURNAL_ID, location.getJournalId(), Store.YES));
             doc.add(new LongField(IndexedFieldNames.BLOCK_INDEX, location.getBlockIndex(), Store.YES));
             doc.add(new LongField(IndexedFieldNames.EVENT_ID, location.getEventId(), Store.YES));
 
@@ -207,22 +224,59 @@ public class LuceneIndexWriter implements EventIndexWriter {
                 updated = true;
             }
         } while (!updated);
+        
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+        logger.debug("Indexed {} events in {} millis with {}", events.size(), millis, this);
     }
     
     
     @Override
-    public void delete(final String containerName, final String section, final String journalId) throws IOException {
+    public void delete(final String containerName, final String section, final Long journalId) throws IOException {
+        final BooleanQuery query = new BooleanQuery();
+        query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.CONTAINER_NAME, containerName)), Occur.MUST));
+        query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.SECTION_NAME, section)), Occur.MUST));
+        query.add(NumericRangeQuery.newLongRange(IndexedFieldNames.JOURNAL_ID, journalId, journalId, true, true), Occur.MUST);
+        
+        final long start = System.nanoTime();
+        indexWriter.deleteDocuments(query);
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+        logger.info("Deleted events from {} that matched container={}, section={}, journal={} in {} millis", indexWriter, containerName, section, journalId, millis);
+    }
+    
+    @Override
+    public void deleteEventsBefore(final String containerName, final String section, final Long journalId) throws IOException {
         final BooleanQuery query = new BooleanQuery();
         query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.CONTAINER_NAME, containerName)), Occur.MUST));
         query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.SECTION_NAME, section)), Occur.MUST));
-        query.add(new BooleanClause(new TermQuery(new Term(IndexedFieldNames.JOURNAL_ID, journalId)), Occur.MUST));
+        query.add(NumericRangeQuery.newLongRange(IndexedFieldNames.JOURNAL_ID, 0L, journalId, true, false), Occur.MUST);
         
+        final long start = System.nanoTime();
         indexWriter.deleteDocuments(query);
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+        logger.info("Deleted events from {} that matched container={}, section={}, journal less than {} in {} millis", indexWriter, containerName, section, journalId, millis);
     }
     
     
     @Override
+    public void deleteOldEvents(final long earliestEventTimeToDelete) throws IOException {
+        final Query query = NumericRangeQuery.newLongRange(SearchableFields.EventTime.getSearchableFieldName(), 0L, earliestEventTimeToDelete, true, true);
+        
+        final long start = System.nanoTime();
+        indexWriter.deleteDocuments(query);
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+        logger.info("Deleted events from {} that ocurred before {}; deletion took {} millis", this, new Date(earliestEventTimeToDelete), millis);
+    }
+    
+    @Override
     public void sync() throws IOException {
+        final long start = System.nanoTime();
         indexWriter.commit();
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+        logger.info("Successfully sync'ed {} in {} millis", this, millis);
+    }
+    
+    @Override
+    public String toString() {
+        return "LuceneIndexWriter[indexDir=" + indexDir + "]";
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
index d086ff5..4accf50 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/MultiIndexSearcher.java
@@ -18,7 +18,9 @@ package org.apache.nifi.provenance.journaling.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 
 import org.apache.nifi.provenance.journaling.JournaledStorageLocation;
@@ -63,6 +65,7 @@ public class MultiIndexSearcher implements EventIndexSearcher {
             locations.addAll(result.getLocations());
         }
         
+        Collections.sort(locations);
         return new SearchResult(locations, totalHitCount);
     }
 
@@ -109,4 +112,52 @@ public class MultiIndexSearcher implements EventIndexSearcher {
         return max;
     }
 
+    @Override
+    public List<JournaledStorageLocation> getEventsForFlowFiles(final Collection<String> flowFileUuids, final long earliestTime, final long latestTime) throws IOException {
+        final List<JournaledStorageLocation> locations = new ArrayList<>();
+        for ( final EventIndexSearcher searcher : searchers ) {
+            final List<JournaledStorageLocation> indexLocations = searcher.getEventsForFlowFiles(flowFileUuids, earliestTime, latestTime);
+            if ( indexLocations != null && !indexLocations.isEmpty() ) {
+                locations.addAll(indexLocations);
+            }
+        }
+        
+        Collections.sort(locations);
+        return locations;
+    }
+    
+    @Override
+    public List<JournaledStorageLocation> getLatestEvents(final int numEvents) throws IOException {
+        final List<JournaledStorageLocation> locations = new ArrayList<>();
+        for ( final EventIndexSearcher searcher : searchers ) {
+            final List<JournaledStorageLocation> indexLocations = searcher.getLatestEvents(numEvents);
+            if ( indexLocations != null && !indexLocations.isEmpty() ) {
+                locations.addAll(indexLocations);
+            }
+        }
+        
+        Collections.sort(locations, new Comparator<JournaledStorageLocation>() {
+            @Override
+            public int compare(final JournaledStorageLocation o1, final JournaledStorageLocation o2) {
+                return Long.compare(o1.getEventId(), o2.getEventId());
+            }
+        });
+        return locations;
+    }
+    
+    @Override
+    public long getNumberOfEvents() throws IOException {
+        long totalCount = 0;
+        
+        for ( final EventIndexSearcher searcher : searchers ) {
+            totalCount += searcher.getNumberOfEvents();
+        }
+        
+        return totalCount;
+    }
+    
+    @Override
+    public String toString() {
+        return searchers.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
index 4ae4b16..d8dd5eb 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/QueryUtils.java
@@ -91,6 +91,15 @@ public class QueryUtils {
     }
     
     
+    /**
+     * Orders the given StorageLocations so that we have a Map where the Key is a Journal file and the value is a List of JournaledStorageLocation in the order
+     * in which they should be read from the journal for optimal performance
+     * @param locations
+     * @param config
+     * @return
+     * @throws FileNotFoundException
+     * @throws IOException
+     */
     public static Map<File, List<JournaledStorageLocation>> orderLocations(final List<StorageLocation> locations, final JournalingRepositoryConfig config) throws FileNotFoundException, IOException {
         final Map<File, List<JournaledStorageLocation>> map = new HashMap<>();
         

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/VoidIndexAction.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/VoidIndexAction.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/VoidIndexAction.java
new file mode 100644
index 0000000..a1ae22f
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/index/VoidIndexAction.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.provenance.journaling.index;
+
+import java.io.IOException;
+
+public interface VoidIndexAction {
+    void perform(EventIndexSearcher searcher) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
index 535d1dd..3345f50 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/JournalReader.java
@@ -44,4 +44,12 @@ public interface JournalReader extends Closeable {
      * @return
      */
     long getPosition();
+    
+    /**
+     * Retrieves the last event from the journal, given the offset of the last Block in the journal
+     * @param blockOffset
+     * @return
+     * @throws IOException
+     */
+    ProvenanceEventRecord getLastEvent(long blockOffset) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
index 2ec5131..df4a2d0 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalReader.java
@@ -62,13 +62,21 @@ public class StandardJournalReader implements JournalReader {
     private void resetStreams() throws IOException {
         final InputStream bufferedIn = new BufferedInputStream(new FileInputStream(file));
         compressedStream = new ByteCountingInputStream(bufferedIn);
-        final DataInputStream dis = new DataInputStream(compressedStream);
-        final String codecName = dis.readUTF();
-        serializationVersion = dis.readInt();
-        compressed = dis.readBoolean();
-        deserializer = Deserializers.getDeserializer(codecName);
-        
-        resetDecompressedStream();
+        try {
+            final DataInputStream dis = new DataInputStream(compressedStream);
+            final String codecName = dis.readUTF();
+            serializationVersion = dis.readInt();
+            compressed = dis.readBoolean();
+            deserializer = Deserializers.getDeserializer(codecName);
+            
+            resetDecompressedStream();
+        } catch (final Exception e) {
+            try {
+                compressedStream.close();
+            } catch (final IOException ignore) {}
+            
+            throw new IOException("Failed to reset data stream when reading" + file, e);
+        }
     }
     
     private void resetDecompressedStream() throws IOException {
@@ -80,6 +88,15 @@ public class StandardJournalReader implements JournalReader {
     }
     
     @Override
+    public void close() throws IOException {
+        compressedStream.close();
+        
+        if ( compressedStream != decompressedStream ) {
+            decompressedStream.close();
+        }
+    }
+    
+    @Override
     public ProvenanceEventRecord nextEvent() throws IOException {
         return nextEvent(true);
     }
@@ -145,7 +162,7 @@ public class StandardJournalReader implements JournalReader {
         // of the file. We do this because we know that the ID's are always increasing, so if we need an ID less
         // than the previous ID, we have to go backward in the file. We can't do this with streams, so start the
         // stream over.
-        if ( eventId < lastEventIdRead ) {
+        if ( eventId <= lastEventIdRead ) {
             close();
             resetStreams();
         }
@@ -167,12 +184,36 @@ public class StandardJournalReader implements JournalReader {
     }
 
     @Override
-    public void close() throws IOException {
-        decompressedStream.close();
-    }
-    
-    @Override
     public String toString() {
         return "StandardJournalReader[" + file + "]";
     }
+
+    @Override
+    public ProvenanceEventRecord getLastEvent(final long blockOffset) throws IOException {
+        if ( blockOffset > compressedStream.getBytesConsumed() ) {
+            close();
+            resetStreams();
+        }
+        
+        final long bytesToSkip = blockOffset - compressedStream.getBytesConsumed();
+        if ( bytesToSkip > 0 ) {
+            StreamUtils.skip(compressedStream, bytesToSkip);
+            resetDecompressedStream();
+        }
+        
+        ProvenanceEventRecord lastReadRecord = null;
+        ProvenanceEventRecord event;
+        while ((event = nextEvent()) != null) {
+            lastReadRecord = event;
+        }
+        
+        // If we weren't able to read anything and the block offset was given, just start over
+        // and read the entire thing, returning the last event.
+        if ( lastReadRecord == null && blockOffset > 0L ) {
+            return getLastEvent(0L);
+        }
+        
+        // return the last even that we read, whether or not it was null
+        return lastReadRecord;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
index af5f8de..8d322b9 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-journaling-provenance-repository/src/main/java/org/apache/nifi/provenance/journaling/journals/StandardJournalWriter.java
@@ -16,11 +16,17 @@
  */
 package org.apache.nifi.provenance.journaling.journals;
 
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
+import java.io.EOFException;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.file.FileAlreadyExistsException;
 import java.util.Collection;
 import java.util.concurrent.TimeUnit;
 
@@ -30,6 +36,8 @@ import org.apache.nifi.remote.io.CompressionOutputStream;
 import org.apache.nifi.stream.io.BufferedOutputStream;
 import org.apache.nifi.stream.io.ByteArrayOutputStream;
 import org.apache.nifi.stream.io.ByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -82,11 +90,14 @@ import org.apache.nifi.stream.io.ByteCountingOutputStream;
  * 
  */
 public class StandardJournalWriter implements JournalWriter {
+    private static final Logger logger = LoggerFactory.getLogger(StandardJournalWriter.class);
+    
     private final long journalId;
     private final File journalFile;
     private final boolean compressed;
     private final Serializer serializer;
     private final long creationTime = System.nanoTime();
+    private final String description;
     
     private int eventCount;
     private boolean blockStarted = false;
@@ -101,10 +112,28 @@ public class StandardJournalWriter implements JournalWriter {
     
     
     public StandardJournalWriter(final long journalId, final File journalFile, final boolean compressed, final Serializer serializer) throws IOException {
+        if ( journalFile.exists() ) {
+            // Check if there is actually any data here.
+            try (final InputStream fis = new FileInputStream(journalFile);
+                 final InputStream bufferedIn = new BufferedInputStream(fis);
+                 final DataInputStream dis = new DataInputStream(bufferedIn) ) {
+                dis.readUTF();
+                dis.readInt();
+                dis.readBoolean();
+                final int nextByte = dis.read();
+                if ( nextByte > -1 ) {
+                    throw new FileAlreadyExistsException(journalFile.getAbsolutePath());
+                }
+            } catch (final EOFException eof) {
+                // If we catch an EOF, there's no real data here, so we can overwrite the file.
+            }
+        }
+        
         this.journalId = journalId;
         this.journalFile = journalFile;
         this.compressed = compressed;
         this.serializer = serializer;
+        this.description = "Journal Writer for " + journalFile;
         this.fos = new FileOutputStream(journalFile);
         
         uncompressedStream = new ByteCountingOutputStream(fos);
@@ -164,6 +193,7 @@ public class StandardJournalWriter implements JournalWriter {
 
     @Override
     public void write(final Collection<ProvenanceEventRecord> events, final long firstEventId) throws IOException {
+        final long start = System.nanoTime();
         final int avgRecordSize = (int) (recordBytes / recordCount);
         
         final ByteArrayOutputStream baos = new ByteArrayOutputStream(avgRecordSize);
@@ -192,6 +222,9 @@ public class StandardJournalWriter implements JournalWriter {
         } finally {
             outDos.flush();
         }
+        
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
+        logger.debug("Finished writing {} events to {} in {} millis", events.size(), this, millis);
     }
     
 
@@ -202,7 +235,10 @@ public class StandardJournalWriter implements JournalWriter {
 
     @Override
     public void sync() throws IOException {
+        final long start = System.nanoTime();
         fos.getFD().sync();
+        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()- start);
+        logger.debug("Successfully sync'ed {} in {} millis", this, millis);
     }
 
     @Override
@@ -259,6 +295,6 @@ public class StandardJournalWriter implements JournalWriter {
     
     @Override
     public String toString() {
-        return "Journal Writer for " + journalFile;
+        return description;
     }
 }


[12/12] incubator-nifi git commit: NIFI-388: Initial implementation of prov repo; not yet finished but pushing so that the code is not lost

Posted by ma...@apache.org.
NIFI-388: Initial implementation of prov repo; not yet finished but pushing so that the code is not lost


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/f23f36d7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/f23f36d7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/f23f36d7

Branch: refs/heads/journaling-prov-repo
Commit: f23f36d73bab207cf31f446e41895b626ccfc883
Parents: a5f557a
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Feb 27 15:45:24 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Feb 27 15:45:24 2015 -0500

----------------------------------------------------------------------
 .../org/apache/nifi/web/api/dto/AboutDTO.class  | Bin 0 -> 1662 bytes
 .../org/apache/nifi/web/api/dto/BannerDTO.class | Bin 0 -> 1689 bytes
 .../nifi/web/api/dto/BulletinBoardDTO.class     | Bin 0 -> 2495 bytes
 .../apache/nifi/web/api/dto/BulletinDTO.class   | Bin 0 -> 4909 bytes
 .../nifi/web/api/dto/BulletinQueryDTO.class     | Bin 0 -> 3439 bytes
 .../apache/nifi/web/api/dto/ClusterDTO.class    | Bin 0 -> 2537 bytes
 .../nifi/web/api/dto/ConnectableDTO.class       | Bin 0 -> 4566 bytes
 .../apache/nifi/web/api/dto/ConnectionDTO.class | Bin 0 -> 7690 bytes
 .../api/dto/ControllerConfigurationDTO.class    | Bin 0 -> 4966 bytes
 .../apache/nifi/web/api/dto/ControllerDTO.class | Bin 0 -> 7819 bytes
 .../apache/nifi/web/api/dto/CounterDTO.class    | Bin 0 -> 2880 bytes
 .../apache/nifi/web/api/dto/CountersDTO.class   | Bin 0 -> 2558 bytes
 .../nifi/web/api/dto/DocumentedTypeDTO.class    | Bin 0 -> 2608 bytes
 .../nifi/web/api/dto/FlowSnippetDTO.class       | Bin 0 -> 5853 bytes
 .../org/apache/nifi/web/api/dto/FunnelDTO.class | Bin 0 -> 675 bytes
 .../org/apache/nifi/web/api/dto/LabelDTO.class  | Bin 0 -> 3299 bytes
 .../nifi/web/api/dto/NiFiComponentDTO.class     | Bin 0 -> 2976 bytes
 .../org/apache/nifi/web/api/dto/NodeDTO.class   | Bin 0 -> 6504 bytes
 .../apache/nifi/web/api/dto/NodeEventDTO.class  | Bin 0 -> 2607 bytes
 .../web/api/dto/NodeSystemDiagnosticsDTO.class  | Bin 0 -> 1573 bytes
 .../org/apache/nifi/web/api/dto/PortDTO.class   | Bin 0 -> 6031 bytes
 .../apache/nifi/web/api/dto/PositionDTO.class   | Bin 0 -> 1938 bytes
 .../nifi/web/api/dto/PreviousValueDTO.class     | Bin 0 -> 2637 bytes
 .../nifi/web/api/dto/ProcessGroupDTO.class      | Bin 0 -> 6157 bytes
 .../ProcessorConfigDTO$AllowableValueDTO.class  | Bin 0 -> 5609 bytes
 ...ocessorConfigDTO$PropertyDescriptorDTO.class | Bin 0 -> 6991 bytes
 .../nifi/web/api/dto/ProcessorConfigDTO.class   | Bin 0 -> 9530 bytes
 .../apache/nifi/web/api/dto/ProcessorDTO.class  | Bin 0 -> 6680 bytes
 .../nifi/web/api/dto/ProcessorHistoryDTO.class  | Bin 0 -> 2481 bytes
 .../nifi/web/api/dto/PropertyHistoryDTO.class   | Bin 0 -> 1615 bytes
 .../nifi/web/api/dto/RelationshipDTO.class      | Bin 0 -> 2247 bytes
 .../api/dto/RemoteProcessGroupContentsDTO.class | Bin 0 -> 2031 bytes
 .../web/api/dto/RemoteProcessGroupDTO.class     | Bin 0 -> 9213 bytes
 .../web/api/dto/RemoteProcessGroupPortDTO.class | Bin 0 -> 6148 bytes
 .../apache/nifi/web/api/dto/RevisionDTO.class   | Bin 0 -> 1824 bytes
 .../apache/nifi/web/api/dto/SnippetDTO.class    | Bin 0 -> 8071 bytes
 ...temDiagnosticsDTO$GarbageCollectionDTO.class | Bin 0 -> 5421 bytes
 .../SystemDiagnosticsDTO$StorageUsageDTO.class  | Bin 0 -> 6420 bytes
 .../nifi/web/api/dto/SystemDiagnosticsDTO.class | Bin 0 -> 10209 bytes
 .../apache/nifi/web/api/dto/TemplateDTO.class   | Bin 0 -> 3709 bytes
 .../org/apache/nifi/web/api/dto/UserDTO.class   | Bin 0 -> 5877 bytes
 .../apache/nifi/web/api/dto/UserGroupDTO.class  | Bin 0 -> 3157 bytes
 .../nifi/web/api/dto/action/ActionDTO.class     | Bin 0 -> 6039 bytes
 .../nifi/web/api/dto/action/HistoryDTO.class    | Bin 0 -> 3142 bytes
 .../web/api/dto/action/HistoryQueryDTO.class    | Bin 0 -> 4725 bytes
 .../component/details/ComponentDetailsDTO.class | Bin 0 -> 952 bytes
 .../component/details/ProcessorDetailsDTO.class | Bin 0 -> 1390 bytes
 .../details/RemoteProcessGroupDetailsDTO.class  | Bin 0 -> 1414 bytes
 .../dto/action/details/ActionDetailsDTO.class   | Bin 0 -> 903 bytes
 .../action/details/ConfigureDetailsDTO.class    | Bin 0 -> 2071 bytes
 .../dto/action/details/ConnectDetailsDTO.class  | Bin 0 -> 3608 bytes
 .../api/dto/action/details/MoveDetailsDTO.class | Bin 0 -> 2450 bytes
 .../dto/action/details/PurgeDetailsDTO.class    | Bin 0 -> 1709 bytes
 .../web/api/dto/provenance/AttributeDTO.class   | Bin 0 -> 2088 bytes
 .../web/api/dto/provenance/ProvenanceDTO.class  | Bin 0 -> 5450 bytes
 .../api/dto/provenance/ProvenanceEventDTO.class | Bin 0 -> 14164 bytes
 .../dto/provenance/ProvenanceOptionsDTO.class   | Bin 0 -> 1740 bytes
 .../dto/provenance/ProvenanceRequestDTO.class   | Bin 0 -> 4846 bytes
 .../dto/provenance/ProvenanceResultsDTO.class   | Bin 0 -> 5826 bytes
 .../ProvenanceSearchableFieldDTO.class          | Bin 0 -> 2458 bytes
 .../api/dto/provenance/lineage/LineageDTO.class | Bin 0 -> 5504 bytes
 .../LineageRequestDTO$LineageRequestType.class  | Bin 0 -> 1834 bytes
 .../provenance/lineage/LineageRequestDTO.class  | Bin 0 -> 2853 bytes
 .../provenance/lineage/LineageResultsDTO.class  | Bin 0 -> 3296 bytes
 .../provenance/lineage/ProvenanceLinkDTO.class  | Bin 0 -> 3581 bytes
 .../provenance/lineage/ProvenanceNodeDTO.class  | Bin 0 -> 5835 bytes
 .../dto/search/ComponentSearchResultDTO.class   | Bin 0 -> 3014 bytes
 .../api/dto/search/NodeSearchResultDTO.class    | Bin 0 -> 1714 bytes
 .../web/api/dto/search/SearchResultsDTO.class   | Bin 0 -> 4454 bytes
 .../dto/search/UserGroupSearchResultDTO.class   | Bin 0 -> 1377 bytes
 .../api/dto/search/UserSearchResultDTO.class    | Bin 0 -> 1729 bytes
 .../dto/status/ClusterConnectionStatusDTO.class | Bin 0 -> 3694 bytes
 .../api/dto/status/ClusterPortStatusDTO.class   | Bin 0 -> 3604 bytes
 .../dto/status/ClusterProcessorStatusDTO.class  | Bin 0 -> 4452 bytes
 .../ClusterRemoteProcessGroupStatusDTO.class    | Bin 0 -> 3814 bytes
 .../web/api/dto/status/ClusterStatusDTO.class   | Bin 0 -> 1721 bytes
 .../dto/status/ClusterStatusHistoryDTO.class    | Bin 0 -> 3169 bytes
 .../api/dto/status/ConnectionStatusDTO.class    | Bin 0 -> 5378 bytes
 .../api/dto/status/ControllerStatusDTO.class    | Bin 0 -> 6248 bytes
 .../dto/status/NodeConnectionStatusDTO.class    | Bin 0 -> 1867 bytes
 .../web/api/dto/status/NodePortStatusDTO.class  | Bin 0 -> 1813 bytes
 .../api/dto/status/NodeProcessorStatusDTO.class | Bin 0 -> 1858 bytes
 .../NodeRemoteProcessGroupStatusDTO.class       | Bin 0 -> 1939 bytes
 .../nifi/web/api/dto/status/NodeStatusDTO.class | Bin 0 -> 1843 bytes
 .../api/dto/status/NodeStatusHistoryDTO.class   | Bin 0 -> 1840 bytes
 .../nifi/web/api/dto/status/PortStatusDTO.class | Bin 0 -> 4179 bytes
 .../api/dto/status/ProcessGroupStatusDTO.class  | Bin 0 -> 9135 bytes
 .../web/api/dto/status/ProcessorStatusDTO.class | Bin 0 -> 5441 bytes
 .../api/dto/status/RemotePortStatusDTO.class    | Bin 0 -> 2982 bytes
 .../status/RemoteProcessGroupStatusDTO.class    | Bin 0 -> 5035 bytes
 .../nifi/web/api/dto/status/StatusDTO.class     | Bin 0 -> 1808 bytes
 .../status/StatusDescriptorDTO$Formatter.class  | Bin 0 -> 2523 bytes
 .../api/dto/status/StatusDescriptorDTO.class    | Bin 0 -> 3423 bytes
 .../web/api/dto/status/StatusHistoryDTO.class   | Bin 0 -> 4503 bytes
 .../api/dto/status/StatusHistoryDetailDTO.class | Bin 0 -> 1726 bytes
 .../web/api/dto/status/StatusSnapshotDTO.class  | Bin 0 -> 2607 bytes
 .../nifi/web/api/dto/util/DateTimeAdapter.class | Bin 0 -> 3043 bytes
 .../nifi/web/api/dto/util/TimeAdapter.class     | Bin 0 -> 3015 bytes
 .../web/api/dto/util/TimestampAdapter.class     | Bin 0 -> 3050 bytes
 .../nifi/web/api/entity/AboutEntity.class       | Bin 0 -> 1294 bytes
 .../nifi/web/api/entity/ActionEntity.class      | Bin 0 -> 1311 bytes
 .../nifi/web/api/entity/AuthorityEntity.class   | Bin 0 -> 2199 bytes
 .../nifi/web/api/entity/BannerEntity.class      | Bin 0 -> 1314 bytes
 .../web/api/entity/BulletinBoardEntity.class    | Bin 0 -> 1425 bytes
 .../entity/ClusterConnectionStatusEntity.class  | Bin 0 -> 1600 bytes
 .../nifi/web/api/entity/ClusterEntity.class     | Bin 0 -> 1328 bytes
 .../api/entity/ClusterPortStatusEntity.class    | Bin 0 -> 1498 bytes
 .../entity/ClusterProcessorStatusEntity.class   | Bin 0 -> 1583 bytes
 .../ClusterRemoteProcessGroupStatusEntity.class | Bin 0 -> 1736 bytes
 .../api/entity/ClusterSearchResultsEntity.class | Bin 0 -> 1944 bytes
 .../web/api/entity/ClusterStatusEntity.class    | Bin 0 -> 1430 bytes
 .../api/entity/ClusterStatusHistoryEntity.class | Bin 0 -> 1549 bytes
 .../nifi/web/api/entity/ConnectionEntity.class  | Bin 0 -> 1379 bytes
 .../nifi/web/api/entity/ConnectionsEntity.class | Bin 0 -> 1761 bytes
 .../entity/ControllerConfigurationEntity.class  | Bin 0 -> 1549 bytes
 .../nifi/web/api/entity/ControllerEntity.class  | Bin 0 -> 1379 bytes
 .../web/api/entity/ControllerStatusEntity.class | Bin 0 -> 1481 bytes
 .../nifi/web/api/entity/CounterEntity.class     | Bin 0 -> 1328 bytes
 .../nifi/web/api/entity/CountersEntity.class    | Bin 0 -> 1345 bytes
 .../org/apache/nifi/web/api/entity/Entity.class | Bin 0 -> 1397 bytes
 .../nifi/web/api/entity/FlowSnippetEntity.class | Bin 0 -> 1387 bytes
 .../nifi/web/api/entity/FunnelEntity.class      | Bin 0 -> 1311 bytes
 .../nifi/web/api/entity/FunnelsEntity.class     | Bin 0 -> 1693 bytes
 .../nifi/web/api/entity/HistoryEntity.class     | Bin 0 -> 1328 bytes
 .../nifi/web/api/entity/InputPortEntity.class   | Bin 0 -> 1307 bytes
 .../nifi/web/api/entity/InputPortsEntity.class  | Bin 0 -> 1689 bytes
 .../nifi/web/api/entity/LabelEntity.class       | Bin 0 -> 1294 bytes
 .../nifi/web/api/entity/LabelsEntity.class      | Bin 0 -> 1676 bytes
 .../nifi/web/api/entity/LineageEntity.class     | Bin 0 -> 1328 bytes
 .../apache/nifi/web/api/entity/NodeEntity.class | Bin 0 -> 1277 bytes
 .../nifi/web/api/entity/NodeStatusEntity.class  | Bin 0 -> 1379 bytes
 .../entity/NodeSystemDiagnosticsEntity.class    | Bin 0 -> 1566 bytes
 .../nifi/web/api/entity/OutputPortEntity.class  | Bin 0 -> 1313 bytes
 .../nifi/web/api/entity/OutputPortsEntity.class | Bin 0 -> 1695 bytes
 .../web/api/entity/PrioritizerTypesEntity.class | Bin 0 -> 1835 bytes
 .../web/api/entity/ProcessGroupEntity.class     | Bin 0 -> 1413 bytes
 .../api/entity/ProcessGroupStatusEntity.class   | Bin 0 -> 1515 bytes
 .../web/api/entity/ProcessGroupsEntity.class    | Bin 0 -> 1795 bytes
 .../nifi/web/api/entity/ProcessorEntity.class   | Bin 0 -> 1362 bytes
 .../web/api/entity/ProcessorHistoryEntity.class | Bin 0 -> 1480 bytes
 .../web/api/entity/ProcessorTypesEntity.class   | Bin 0 -> 1823 bytes
 .../nifi/web/api/entity/ProcessorsEntity.class  | Bin 0 -> 1744 bytes
 .../nifi/web/api/entity/ProvenanceEntity.class  | Bin 0 -> 1379 bytes
 .../web/api/entity/ProvenanceEventEntity.class  | Bin 0 -> 1464 bytes
 .../api/entity/ProvenanceOptionsEntity.class    | Bin 0 -> 1498 bytes
 .../api/entity/RemoteProcessGroupEntity.class   | Bin 0 -> 1515 bytes
 .../entity/RemoteProcessGroupPortEntity.class   | Bin 0 -> 1583 bytes
 .../api/entity/RemoteProcessGroupsEntity.class  | Bin 0 -> 1897 bytes
 .../web/api/entity/SearchResultsEntity.class    | Bin 0 -> 1515 bytes
 .../nifi/web/api/entity/SnippetEntity.class     | Bin 0 -> 1328 bytes
 .../web/api/entity/StatusHistoryEntity.class    | Bin 0 -> 1430 bytes
 .../api/entity/SystemDiagnosticsEntity.class    | Bin 0 -> 1498 bytes
 .../nifi/web/api/entity/TemplateEntity.class    | Bin 0 -> 1345 bytes
 .../nifi/web/api/entity/TemplatesEntity.class   | Bin 0 -> 2643 bytes
 .../apache/nifi/web/api/entity/UserEntity.class | Bin 0 -> 1277 bytes
 .../nifi/web/api/entity/UserGroupEntity.class   | Bin 0 -> 1362 bytes
 .../api/entity/UserSearchResultsEntity.class    | Bin 0 -> 3015 bytes
 .../nifi/web/api/entity/UsersEntity.class       | Bin 0 -> 2876 bytes
 .../apache/nifi/controller/FlowController.java  |   5 +-
 .../src/main/resources/conf/nifi.properties     |  35 ++
 .../journaling/JournaledStorageLocation.java    |  11 +-
 .../JournalingProvenanceRepository.java         | 210 ++++++++--
 .../config/JournalingRepositoryConfig.java      |  73 +++-
 .../journaling/index/EventIndexSearcher.java    |  47 +++
 .../journaling/index/EventIndexWriter.java      |  20 +-
 .../journaling/index/IndexAction.java           |  23 ++
 .../journaling/index/IndexManager.java          | 109 +++++
 .../journaling/index/LuceneIndexManager.java    | 211 +++++++++-
 .../journaling/index/LuceneIndexSearcher.java   |  73 +++-
 .../journaling/index/LuceneIndexWriter.java     |  62 ++-
 .../journaling/index/MultiIndexSearcher.java    |  51 +++
 .../provenance/journaling/index/QueryUtils.java |   9 +
 .../journaling/index/VoidIndexAction.java       |  23 ++
 .../journaling/journals/JournalReader.java      |   8 +
 .../journals/StandardJournalReader.java         |  67 +++-
 .../journals/StandardJournalWriter.java         |  38 +-
 .../partition/JournalingPartition.java          | 397 ++++++++++++++-----
 .../journaling/partition/Partition.java         |  41 ++
 .../journaling/partition/PartitionManager.java  |  27 +-
 .../partition/QueuingPartitionManager.java      | 365 +++++++++++++----
 .../journaling/query/QueryManager.java          |  53 ++-
 .../journaling/query/StandardQueryManager.java  | 356 ++++++++++++++---
 .../journaling/tasks/CompressionTask.java       |  49 ++-
 .../journaling/toc/StandardTocReader.java       |   8 +
 .../journaling/toc/StandardTocWriter.java       |  24 +-
 .../journaling/toc/TocJournalReader.java        |   4 +-
 .../provenance/journaling/toc/TocReader.java    |   5 +
 .../TestJournalingProvenanceRepository.java     | 287 ++++++++++++++
 .../journaling/index/TestEventIndexWriter.java  |   9 +-
 .../journals/TestStandardJournalWriter.java     |  38 ++
 .../journaling/toc/TestStandardTocWriter.java   |  64 +++
 191 files changed, 2460 insertions(+), 342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.class
new file mode 100644
index 0000000..4055ba4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.class
new file mode 100644
index 0000000..c9122f2
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.class
new file mode 100644
index 0000000..120c886
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.class
new file mode 100644
index 0000000..5065ae3
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.class
new file mode 100644
index 0000000..b44967b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.class
new file mode 100644
index 0000000..f9b0aa2
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.class
new file mode 100644
index 0000000..dfe4ac7
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.class
new file mode 100644
index 0000000..dd8e71a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.class
new file mode 100644
index 0000000..d776194
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.class
new file mode 100644
index 0000000..ce50308
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.class
new file mode 100644
index 0000000..d5609d0
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.class
new file mode 100644
index 0000000..3138f4c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.class
new file mode 100644
index 0000000..b960bb9
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.class
new file mode 100644
index 0000000..3c432e4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.class
new file mode 100644
index 0000000..101617e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.class
new file mode 100644
index 0000000..bcba8b5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.class
new file mode 100644
index 0000000..69f1a4e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.class
new file mode 100644
index 0000000..e78e015
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.class
new file mode 100644
index 0000000..fc3e322
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.class
new file mode 100644
index 0000000..c4210a7
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PortDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PortDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PortDTO.class
new file mode 100644
index 0000000..7cda829
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PortDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.class
new file mode 100644
index 0000000..7cbbea7
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.class
new file mode 100644
index 0000000..6e84fb9
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.class
new file mode 100644
index 0000000..a0b5c0b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$AllowableValueDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$AllowableValueDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$AllowableValueDTO.class
new file mode 100644
index 0000000..713edf6
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$AllowableValueDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$PropertyDescriptorDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$PropertyDescriptorDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$PropertyDescriptorDTO.class
new file mode 100644
index 0000000..fd8b48b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO$PropertyDescriptorDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.class
new file mode 100644
index 0000000..edbe592
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.class
new file mode 100644
index 0000000..5792819
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.class
new file mode 100644
index 0000000..5bc4934
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.class
new file mode 100644
index 0000000..1ef3b27
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.class
new file mode 100644
index 0000000..37d249f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.class
new file mode 100644
index 0000000..8115ad2
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.class
new file mode 100644
index 0000000..0f09381
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.class
new file mode 100644
index 0000000..079c53c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.class
new file mode 100644
index 0000000..5ecd23b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.class
new file mode 100644
index 0000000..78129c4
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$GarbageCollectionDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$GarbageCollectionDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$GarbageCollectionDTO.class
new file mode 100644
index 0000000..e0ec9f6
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$GarbageCollectionDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$StorageUsageDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$StorageUsageDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$StorageUsageDTO.class
new file mode 100644
index 0000000..f9c9b44
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO$StorageUsageDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.class
new file mode 100644
index 0000000..89c3a62
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.class
new file mode 100644
index 0000000..b099679
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserDTO.class
new file mode 100644
index 0000000..80c8817
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.class
new file mode 100644
index 0000000..d6d671c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.class
new file mode 100644
index 0000000..60d6e9e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.class
new file mode 100644
index 0000000..b247015
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.class
new file mode 100644
index 0000000..830f7e6
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.class
new file mode 100644
index 0000000..a45a51b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.class
new file mode 100644
index 0000000..9aa6881
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.class
new file mode 100644
index 0000000..45732d6
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.class
new file mode 100644
index 0000000..0a94a2a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.class
new file mode 100644
index 0000000..d61b48d
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.class
new file mode 100644
index 0000000..6c61912
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.class
new file mode 100644
index 0000000..fb47ac1
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.class
new file mode 100644
index 0000000..0b4005f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.class
new file mode 100644
index 0000000..28df4d5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.class
new file mode 100644
index 0000000..4a17300
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.class
new file mode 100644
index 0000000..b86182f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.class
new file mode 100644
index 0000000..432cc92
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.class
new file mode 100644
index 0000000..f37f69a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.class
new file mode 100644
index 0000000..fc32034
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.class
new file mode 100644
index 0000000..b52c24b
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.class
new file mode 100644
index 0000000..96f1a91
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO$LineageRequestType.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO$LineageRequestType.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO$LineageRequestType.class
new file mode 100644
index 0000000..c14bc33
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO$LineageRequestType.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.class
new file mode 100644
index 0000000..b577d57
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.class
new file mode 100644
index 0000000..7dd4096
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.class
new file mode 100644
index 0000000..5e6002e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.class
new file mode 100644
index 0000000..c09560c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.class
new file mode 100644
index 0000000..e6ac2e2
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.class
new file mode 100644
index 0000000..6203ed1
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.class
new file mode 100644
index 0000000..6889c35
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.class
new file mode 100644
index 0000000..07a4e62
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.class
new file mode 100644
index 0000000..1ca441c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.class
new file mode 100644
index 0000000..c1e90c5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.class
new file mode 100644
index 0000000..f749350
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.class
new file mode 100644
index 0000000..ec72a99
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.class
new file mode 100644
index 0000000..3432ca6
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.class
new file mode 100644
index 0000000..e5f56cc
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.class
new file mode 100644
index 0000000..5cd42c7
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.class
new file mode 100644
index 0000000..bb725e8
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.class
new file mode 100644
index 0000000..1b7f80c
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.class
new file mode 100644
index 0000000..67677a9
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.class
new file mode 100644
index 0000000..6d8fd3a
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.class
new file mode 100644
index 0000000..4a3167f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.class
new file mode 100644
index 0000000..e9a6001
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.class
new file mode 100644
index 0000000..97f115e
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.class
new file mode 100644
index 0000000..0bbaf51
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.class
new file mode 100644
index 0000000..01c90aa
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.class
new file mode 100644
index 0000000..31d4c68
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.class
new file mode 100644
index 0000000..f11659f
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.class
new file mode 100644
index 0000000..56351e9
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.class differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/f23f36d7/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.class
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.class b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.class
new file mode 100644
index 0000000..f97c7d0
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/bin/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.class differ