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>
+ * <header>
+ * <begin block 1>
+ * <record 1>
+ * <record 2>
+ * <record 3>
+ * <end block 1>
+ * <begin block 2>
+ * <record 4>
+ * <record 5>
+ * <end block 2>
+ * ...
+ * <begin block N>
+ * <record N>
+ * <end block N>
+ * </pre>
+ *
+ * Where <header> is defined as:
+ * <pre>
+ * 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 <record> is defined as:
+ * <pre>
+ * bytes 0-3: int: record length
+ * bytes 4-11: long: record id
+ * bytes 12-N: serialized event according to the applied {@link Serializer}
+ * </pre>
+ * </p>
+ *
+ * <p>
+ * The structure of the <begin block> and <end block> element depend on whether or not
+ * compression is enabled. If the journal is not compressed, these elements are 0 bytes.
+ * If the journal is compressed, these are the compression header and compression footer, respectively.
+ * </p>
+ *
+ */
+public class StandardJournalWriter implements JournalWriter {
+ private 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