diff --git a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java index 7ba1622b7b..b05bd85932 100644 --- a/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java +++ b/nifi-api/src/main/java/org/apache/nifi/provenance/ProvenanceEventRecord.java @@ -79,6 +79,16 @@ public interface ProvenanceEventRecord { */ Map getAttributes(); + /** + * Returns the attribute with the given name + * + * @param attributeName the name of the attribute to get + * @return the attribute with the given name or null if no attribute exists with the given name + */ + default String getAttribute(String attributeName) { + return getAttributes().get(attributeName); + } + /** * @return all FlowFile attributes that existed on the FlowFile before this * event occurred diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java index 8d92c44292..10d5cde9d3 100644 --- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java +++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java @@ -1067,11 +1067,14 @@ public class RunNiFi { Process process = builder.start(); handleLogging(process); Long pid = getPid(process, cmdLogger); - if (pid != null) { + if (pid == null) { + cmdLogger.info("Launched Apache NiFi but could not determined the Process ID"); + } else { nifiPid = pid; final Properties pidProperties = new Properties(); pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid)); savePidProperties(pidProperties, cmdLogger); + cmdLogger.info("Launched Apache NiFi with Process ID " + pid); } shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor); @@ -1129,11 +1132,14 @@ public class RunNiFi { handleLogging(process); pid = getPid(process, defaultLogger); - if (pid != null) { + if (pid == null) { + cmdLogger.info("Launched Apache NiFi but could not obtain the Process ID"); + } else { nifiPid = pid; final Properties pidProperties = new Properties(); pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid)); savePidProperties(pidProperties, defaultLogger); + cmdLogger.info("Launched Apache NiFi with Process ID " + pid); } shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor); diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProgressiveResult.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProgressiveResult.java new file mode 100644 index 0000000000..e2dd33b16c --- /dev/null +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/ProgressiveResult.java @@ -0,0 +1,34 @@ +/* + * 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.Collection; + +/** + * A Provenance query result that is capable of being updated + */ +public interface ProgressiveResult { + + void update(Collection records, long totalHitCount); + + void setError(String error); + + long getTotalHitCount(); + + boolean isFinished(); +} diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java index 78b31889a1..2b7b34a864 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java @@ -44,7 +44,7 @@ import org.apache.nifi.provenance.lineage.LineageNode; /** * */ -public class StandardLineageResult implements ComputeLineageResult { +public class StandardLineageResult implements ComputeLineageResult, ProgressiveResult { public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES); private static final Logger logger = LoggerFactory.getLogger(StandardLineageResult.class); @@ -66,6 +66,7 @@ public class StandardLineageResult implements ComputeLineageResult { private int numCompletedSteps = 0; private volatile boolean canceled = false; + private final Object completionMonitor = new Object(); public StandardLineageResult(final int numSteps, final Collection flowFileUuids) { this.numSteps = numSteps; @@ -162,6 +163,7 @@ public class StandardLineageResult implements ComputeLineageResult { } } + @Override public void setError(final String error) { writeLock.lock(); try { @@ -178,7 +180,10 @@ public class StandardLineageResult implements ComputeLineageResult { } } - public void update(final Collection records) { + @Override + public void update(final Collection records, final long totalHitCount) { + boolean computationComplete = false; + writeLock.lock(); try { relevantRecords.addAll(records); @@ -187,12 +192,22 @@ public class StandardLineageResult implements ComputeLineageResult { updateExpiration(); if (numCompletedSteps >= numSteps && error == null) { + computationComplete = true; computeLineage(); computationNanos = System.nanoTime() - creationNanos; } } finally { writeLock.unlock(); } + + if (computationComplete) { + final long computationMillis = TimeUnit.NANOSECONDS.toMillis(computationNanos); + logger.info("Completed computation of lineage for FlowFile UUID(s) {} comprised of {} steps in {} millis", flowFileUuids, numSteps, computationMillis); + + synchronized (completionMonitor) { + completionMonitor.notifyAll(); + } + } } /** @@ -201,6 +216,7 @@ public class StandardLineageResult implements ComputeLineageResult { * useful after all of the records have been successfully obtained */ private void computeLineage() { + logger.debug("Computing lineage with the following events: {}", relevantRecords); final long startNanos = System.nanoTime(); nodes.clear(); @@ -324,4 +340,31 @@ public class StandardLineageResult implements ComputeLineageResult { private void updateExpiration() { expirationDate = new Date(System.currentTimeMillis() + TTL); } + + @Override + public boolean awaitCompletion(final long time, final TimeUnit unit) throws InterruptedException { + final long finishTime = System.currentTimeMillis() + unit.toMillis(time); + synchronized (completionMonitor) { + while (!isFinished()) { + final long millisToWait = finishTime - System.currentTimeMillis(); + if (millisToWait > 0) { + completionMonitor.wait(millisToWait); + } else { + return isFinished(); + } + } + } + + return isFinished(); + } + + @Override + public long getTotalHitCount() { + readLock.lock(); + try { + return relevantRecords.size(); + } finally { + readLock.unlock(); + } + } } diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java index f8f4055463..ac60d4f680 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java @@ -67,7 +67,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor private final Map previousAttributes; private final Map updatedAttributes; - private volatile long eventId; + private volatile long eventId = -1L; private StandardProvenanceEventRecord(final Builder builder) { this.eventTime = builder.eventTime; @@ -369,14 +369,22 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor return false; } - if (a == null && b != null) { + if (a == null && b != null && !b.isEmpty()) { return true; } - if (a != null && b == null) { + if (a == null && b.isEmpty()) { + return false; + } + + if (a != null && !a.isEmpty() && b == null) { return true; } + if (a.isEmpty() && b == null) { + return false; + } + if (a.size() != b.size()) { return true; } diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java index 5c09e8eeb4..277733970a 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java @@ -22,7 +22,7 @@ import java.util.Comparator; import java.util.Date; import java.util.Iterator; import java.util.List; -import java.util.Set; +import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; @@ -31,8 +31,11 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.nifi.provenance.search.Query; import org.apache.nifi.provenance.search.QueryResult; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class StandardQueryResult implements QueryResult { +public class StandardQueryResult implements QueryResult, ProgressiveResult { + private static final Logger logger = LoggerFactory.getLogger(StandardQueryResult.class); public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES); private final Query query; @@ -44,12 +47,12 @@ public class StandardQueryResult implements QueryResult { private final Lock writeLock = rwLock.writeLock(); // guarded by writeLock - private final Set matchingRecords = new TreeSet<>(new EventIdComparator()); - private long totalHitCount; + private final SortedSet matchingRecords = new TreeSet<>(new EventIdComparator()); private int numCompletedSteps = 0; private Date expirationDate; private String error; private long queryTime; + private final Object completionMonitor = new Object(); private volatile boolean canceled = false; @@ -65,22 +68,7 @@ public class StandardQueryResult implements QueryResult { public List getMatchingEvents() { readLock.lock(); try { - if (matchingRecords.size() <= query.getMaxResults()) { - return new ArrayList<>(matchingRecords); - } - - final List copy = new ArrayList<>(query.getMaxResults()); - - int i = 0; - final Iterator itr = matchingRecords.iterator(); - while (itr.hasNext()) { - copy.add(itr.next()); - if (++i >= query.getMaxResults()) { - break; - } - } - - return copy; + return new ArrayList<>(matchingRecords); } finally { readLock.unlock(); } @@ -137,7 +125,7 @@ public class StandardQueryResult implements QueryResult { public boolean isFinished() { readLock.lock(); try { - return numCompletedSteps >= numSteps || canceled; + return numCompletedSteps >= numSteps || canceled || matchingRecords.size() >= query.getMaxResults(); } finally { readLock.unlock(); } @@ -147,6 +135,7 @@ public class StandardQueryResult implements QueryResult { this.canceled = true; } + @Override public void setError(final String error) { writeLock.lock(); try { @@ -163,22 +152,74 @@ public class StandardQueryResult implements QueryResult { } } - public void update(final Collection matchingRecords, final long totalHits) { + @Override + public void update(final Collection newEvents, final long totalHits) { + boolean queryComplete = false; + writeLock.lock(); try { - this.matchingRecords.addAll(matchingRecords); - this.totalHitCount += totalHits; + if (isFinished()) { + return; + } + + this.matchingRecords.addAll(newEvents); + + // If we've added more records than the query's max, then remove the trailing elements. + // We do this, rather than avoiding the addition of the elements because we want to choose + // the events with the largest ID. + if (matchingRecords.size() > query.getMaxResults()) { + final Iterator itr = matchingRecords.iterator(); + for (int i = 0; i < query.getMaxResults(); i++) { + itr.next(); + } + + while (itr.hasNext()) { + itr.next(); + itr.remove(); + } + } numCompletedSteps++; updateExpiration(); - if (numCompletedSteps >= numSteps) { + if (numCompletedSteps >= numSteps || this.matchingRecords.size() >= query.getMaxResults()) { final long searchNanos = System.nanoTime() - creationNanos; queryTime = TimeUnit.MILLISECONDS.convert(searchNanos, TimeUnit.NANOSECONDS); + queryComplete = true; + + if (numCompletedSteps >= numSteps) { + logger.info("Completed {} comprised of {} steps in {} millis", query, numSteps, queryTime); + } else { + logger.info("Completed {} comprised of {} steps in {} millis (only completed {} steps because the maximum number of results was reached)", + query, numSteps, queryTime, numCompletedSteps); + } } } finally { writeLock.unlock(); } + + if (queryComplete) { + synchronized (completionMonitor) { + completionMonitor.notifyAll(); + } + } + } + + @Override + public boolean awaitCompletion(final long time, final TimeUnit unit) throws InterruptedException { + final long finishTime = System.currentTimeMillis() + unit.toMillis(time); + synchronized (completionMonitor) { + while (!isFinished()) { + final long millisToWait = finishTime - System.currentTimeMillis(); + if (millisToWait > 0) { + completionMonitor.wait(millisToWait); + } else { + return isFinished(); + } + } + } + + return isFinished(); } /** diff --git a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java index de516cb9fd..4906ea39ca 100644 --- a/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java +++ b/nifi-commons/nifi-data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java @@ -18,7 +18,6 @@ package org.apache.nifi.provenance.lineage; import java.util.List; -import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventType; @@ -58,7 +57,7 @@ public class EventNode implements ProvenanceEventLineageNode { @Override public String getFlowFileUuid() { - return record.getAttributes().get(CoreAttributes.UUID.key()); + return record.getFlowFileUuid(); } @Override diff --git a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/FieldMapRecord.java b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/FieldMapRecord.java index c9368a7a74..acebcb9b48 100644 --- a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/FieldMapRecord.java +++ b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/FieldMapRecord.java @@ -29,6 +29,11 @@ public class FieldMapRecord implements Record { this.values = convertFieldToName(values); } + public FieldMapRecord(final RecordSchema schema, final Map values) { + this.schema = schema; + this.values = new HashMap<>(values); + } + private static Map convertFieldToName(final Map map) { final Map nameMap = new HashMap<>(map.size()); for (final Map.Entry entry : map.entrySet()) { diff --git a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/RecordSchema.java b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/RecordSchema.java index 965254bc54..fe187652a0 100644 --- a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/RecordSchema.java +++ b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/RecordSchema.java @@ -120,8 +120,8 @@ public class RecordSchema { @SuppressWarnings("unchecked") private static RecordField readField(final DataInputStream dis) throws IOException { - final Map schemaFieldMap = new HashMap<>(); final int numElementsToRead = dis.readInt(); + final Map schemaFieldMap = new HashMap<>(numElementsToRead); for (int i = 0; i < numElementsToRead; i++) { final String fieldName = dis.readUTF(); final String typeName = dis.readUTF(); diff --git a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordReader.java b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordReader.java index b296b13bc2..84f353231a 100644 --- a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordReader.java +++ b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordReader.java @@ -109,7 +109,15 @@ public class SchemaRecordReader { } } - return readFieldValue(in, field.getFieldType(), field.getFieldName(), field.getSubFields()); + try { + return readFieldValue(in, field.getFieldType(), field.getFieldName(), field.getSubFields()); + } catch (final EOFException eof) { + final EOFException exception = new EOFException("Failed to read field '" + field.getFieldName() + "'"); + exception.addSuppressed(eof); + throw exception; + } catch (final IOException ioe) { + throw new IOException("Failed to read field '" + field.getFieldName() + "'", ioe); + } } diff --git a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordWriter.java b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordWriter.java index 3e4a059322..5305e5bd01 100644 --- a/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordWriter.java +++ b/nifi-commons/nifi-schema-utils/src/main/java/org/apache/nifi/repository/schema/SchemaRecordWriter.java @@ -44,8 +44,12 @@ public class SchemaRecordWriter { } private void writeRecordFields(final Record record, final OutputStream out) throws IOException { + writeRecordFields(record, record.getSchema(), out); + } + + private void writeRecordFields(final Record record, final RecordSchema schema, final OutputStream out) throws IOException { final DataOutputStream dos = out instanceof DataOutputStream ? (DataOutputStream) out : new DataOutputStream(out); - for (final RecordField field : record.getSchema().getFields()) { + for (final RecordField field : schema.getFields()) { final Object value = record.getFieldValue(field); try { diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java index 12e2b10acb..1a9e219d66 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java @@ -18,6 +18,9 @@ package org.wali; import static java.util.Objects.requireNonNull; +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; @@ -55,12 +58,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.regex.Pattern; -import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.stream.io.BufferedOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -230,7 +230,7 @@ public final class MinimalLockingWriteAheadLog implements WriteAheadRepositor final long transactionId = transactionIdGenerator.getAndIncrement(); if (logger.isTraceEnabled()) { for (final T record : records) { - logger.trace("Partition {} performing Transaction {}: {}", new Object[]{partition, transactionId, record}); + logger.trace("Partition {} performing Transaction {}: {}", new Object[] {partition, transactionId, record}); } } @@ -670,11 +670,10 @@ public final class MinimalLockingWriteAheadLog implements WriteAheadRepositor private final Path editDirectory; private final int writeAheadLogVersion; - private final Lock lock = new ReentrantLock(); private DataOutputStream dataOut = null; private FileOutputStream fileOut = null; - private boolean blackListed = false; - private boolean closed = false; + private volatile boolean blackListed = false; + private volatile boolean closed = false; private DataInputStream recoveryIn; private int recoveryVersion; private String currentJournalFilename = ""; @@ -707,26 +706,15 @@ public final class MinimalLockingWriteAheadLog implements WriteAheadRepositor } public boolean tryClaim() { - final boolean obtainedLock = lock.tryLock(); - if (!obtainedLock) { - return false; - } - - // Check if the partition is blacklisted. If so, unlock it and return false. Otherwise, - // leave it locked and return true, so that the caller will need to unlock. - if (blackListed) { - lock.unlock(); - return false; - } - - return true; + return !blackListed; } public void releaseClaim() { - lock.unlock(); } public void close() { + this.closed = true; + // Note that here we are closing fileOut and NOT dataOut. // This is very much intentional, not an oversight. This is done because of // the way that the OutputStreams are structured. dataOut wraps a BufferedOutputStream, @@ -761,18 +749,12 @@ public final class MinimalLockingWriteAheadLog implements WriteAheadRepositor } } - this.closed = true; this.dataOut = null; this.fileOut = null; } public void blackList() { - lock.lock(); - try { - blackListed = true; - } finally { - lock.unlock(); - } + blackListed = true; logger.debug("Blacklisted {}", this); } @@ -783,55 +765,50 @@ public final class MinimalLockingWriteAheadLog implements WriteAheadRepositor * @throws IOException if failure to rollover */ public OutputStream rollover() throws IOException { - lock.lock(); + // Note that here we are closing fileOut and NOT dataOut. See the note in the close() + // method to understand the logic behind this. + final OutputStream oldOutputStream = fileOut; + dataOut = null; + fileOut = null; + + this.serde = serdeFactory.createSerDe(null); + final Path editPath = getNewEditPath(); + final FileOutputStream fos = new FileOutputStream(editPath.toFile()); try { - // Note that here we are closing fileOut and NOT dataOut. See the note in the close() - // method to understand the logic behind this. - final OutputStream oldOutputStream = fileOut; - dataOut = null; - fileOut = null; + final DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(fos)); + outStream.writeUTF(MinimalLockingWriteAheadLog.class.getName()); + outStream.writeInt(writeAheadLogVersion); + outStream.writeUTF(serde.getClass().getName()); + outStream.writeInt(serde.getVersion()); + serde.writeHeader(outStream); - this.serde = serdeFactory.createSerDe(null); - final Path editPath = getNewEditPath(); - final FileOutputStream fos = new FileOutputStream(editPath.toFile()); + outStream.flush(); + dataOut = outStream; + fileOut = fos; + } catch (final IOException ioe) { try { - final DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(fos)); - outStream.writeUTF(MinimalLockingWriteAheadLog.class.getName()); - outStream.writeInt(writeAheadLogVersion); - outStream.writeUTF(serde.getClass().getName()); - outStream.writeInt(serde.getVersion()); - serde.writeHeader(outStream); - - outStream.flush(); - dataOut = outStream; - fileOut = fos; - } catch (final IOException ioe) { - try { - oldOutputStream.close(); - } catch (final IOException ioe2) { - ioe.addSuppressed(ioe2); - } - - logger.error("Failed to create new journal for {} due to {}", new Object[] {this, ioe.toString()}, ioe); - try { - fos.close(); - } catch (final IOException innerIOE) { - } - - dataOut = null; - fileOut = null; - blackList(); - - throw ioe; + oldOutputStream.close(); + } catch (final IOException ioe2) { + ioe.addSuppressed(ioe2); } - currentJournalFilename = editPath.toFile().getName(); + logger.error("Failed to create new journal for {} due to {}", new Object[] {this, ioe.toString()}, ioe); + try { + fos.close(); + } catch (final IOException innerIOE) { + } - blackListed = false; - return oldOutputStream; - } finally { - lock.unlock(); + dataOut = null; + fileOut = null; + blackList(); + + throw ioe; } + + currentJournalFilename = editPath.toFile().getName(); + + blackListed = false; + return oldOutputStream; } private long getJournalIndex(final File file) { @@ -939,33 +916,39 @@ public final class MinimalLockingWriteAheadLog implements WriteAheadRepositor return true; } - public void update(final Collection records, final long transactionId, final Map recordMap, final boolean forceSync) - throws IOException { - if (this.closed) { - throw new IllegalStateException("Partition is closed"); - } + public void update(final Collection records, final long transactionId, final Map recordMap, final boolean forceSync) throws IOException { + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(256); + final DataOutputStream out = new DataOutputStream(baos)) { - final DataOutputStream out = dataOut; - out.writeLong(transactionId); + out.writeLong(transactionId); + final int numEditsToSerialize = records.size(); + int editsSerialized = 0; + for (final S record : records) { + final Object recordId = serde.getRecordIdentifier(record); + final S previousVersion = recordMap.get(recordId); - final int numEditsToSerialize = records.size(); - int editsSerialized = 0; - for (final S record : records) { - final Object recordId = serde.getRecordIdentifier(record); - final S previousVersion = recordMap.get(recordId); - - serde.serializeEdit(previousVersion, record, out); - if (++editsSerialized < numEditsToSerialize) { - out.write(TRANSACTION_CONTINUE); - } else { - out.write(TRANSACTION_COMMIT); + serde.serializeEdit(previousVersion, record, out); + if (++editsSerialized < numEditsToSerialize) { + out.write(TRANSACTION_CONTINUE); + } else { + out.write(TRANSACTION_COMMIT); + } } - } - out.flush(); + out.flush(); - if (forceSync) { - fileOut.getFD().sync(); + if (this.closed) { + throw new IllegalStateException("Partition is closed"); + } + + baos.writeTo(dataOut); + dataOut.flush(); + + if (forceSync) { + synchronized (fileOut) { + fileOut.getFD().sync(); + } + } } } diff --git a/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java b/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java index 7b7d2cacbe..5cdad8225e 100644 --- a/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java +++ b/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java @@ -53,6 +53,64 @@ public class TestMinimalLockingWriteAheadLog { private static final Logger logger = LoggerFactory.getLogger(TestMinimalLockingWriteAheadLog.class); + @Test + @Ignore("for local testing only") + public void testUpdatePerformance() throws IOException, InterruptedException { + final int numPartitions = 4; + + final Path path = Paths.get("target/minimal-locking-repo"); + deleteRecursively(path.toFile()); + assertTrue(path.toFile().mkdirs()); + + final DummyRecordSerde serde = new DummyRecordSerde(); + final WriteAheadRepository repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null); + final Collection initialRecs = repo.recoverRecords(); + assertTrue(initialRecs.isEmpty()); + + final int updateCountPerThread = 1_000_000; + final int numThreads = 16; + + final Thread[] threads = new Thread[numThreads]; + + for (int j = 0; j < 2; j++) { + for (int i = 0; i < numThreads; i++) { + final Thread t = new Thread(new Runnable() { + @Override + public void run() { + for (int i = 0; i < updateCountPerThread; i++) { + final DummyRecord record = new DummyRecord(String.valueOf(i), UpdateType.CREATE); + try { + repo.update(Collections.singleton(record), false); + } catch (IOException e) { + e.printStackTrace(); + Assert.fail(e.toString()); + } + } + } + }); + + threads[i] = t; + } + + final long start = System.nanoTime(); + for (final Thread t : threads) { + t.start(); + } + for (final Thread t : threads) { + t.join(); + } + + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + if (j == 0) { + System.out.println(millis + " ms to insert " + updateCountPerThread * numThreads + " updates using " + numPartitions + " partitions and " + numThreads + " threads, *as a warmup!*"); + } else { + System.out.println(millis + " ms to insert " + updateCountPerThread * numThreads + " updates using " + numPartitions + " partitions and " + numThreads + " threads"); + } + } + } + + + @Test public void testRepoDoesntContinuallyGrowOnOutOfMemoryError() throws IOException, InterruptedException { final int numPartitions = 8; @@ -557,21 +615,10 @@ public class TestMinimalLockingWriteAheadLog { assertEquals(2, transactionIndicator); } - long transactionId = in.readLong(); - assertEquals(2L, transactionId); - - long thirdSize = in.readLong(); - assertEquals(8194, thirdSize); - - // should be 8176 A's because we threw an Exception after writing 8194 of them, - // but the BufferedOutputStream's buffer already had 8 bytes on it for the - // transaction id and the size. - for (int i = 0; i < 8176; i++) { - final int c = in.read(); - assertEquals("i = " + i, 'A', c); - } - - // Stream should now be out of data, because we threw an Exception! + // In previous implementations, we would still have a partial record written out. + // In the current version, however, the serde above would result in the data serialization + // failing and as a result no data would be written to the stream, so the stream should + // now be out of data final int nextByte = in.read(); assertEquals(-1, nextByte); } diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc index f7f992042b..fff0bdd990 100644 --- a/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -2047,8 +2047,8 @@ FlowFile Repository, if also on that disk, could become corrupt. To avoid this s + For example, to provide two additional locations to act as part of the content repository, a user could also specify additional properties with keys of: + + -nifi.provenance.repository.directory.content1=/repos/provenance1 + -nifi.provenance.repository.directory.content2=/repos/provenance2 + +nifi.content.repository.directory.content1=/repos/content1 + +nifi.content.repository.directory.content2=/repos/content2 + + Providing three total locations, including _nifi.content.repository.directory.default_. |nifi.content.repository.archive.max.retention.period|If archiving is enabled (see nifi.content.repository.archive.enabled below), then @@ -2073,7 +2073,25 @@ The Provenance Repository contains the information related to Data Provenance. T |==== |*Property*|*Description* -|nifi.provenance.repository.implementation|The Provenance Repository implementation. The default value is org.apache.nifi.provenance.PersistentProvenanceRepository and should only be changed with caution. To store provenance events in memory instead of on disk (at the risk of data loss in the event of power/machine failure), set this property to org.apache.nifi.provenance.VolatileProvenanceRepository. +|nifi.provenance.repository.implementation|The Provenance Repository implementation. The default value is org.apache.nifi.provenance.PersistentProvenanceRepository. +Two additional repositories are available as well. +To store provenance events in memory instead of on disk (in which case all events will be lost on restart, and events will be evicted in a first-in-first-out order), +set this property to org.apache.nifi.provenance.VolatileProvenanceRepository. This leaves a configurable number of Provenance Events in the Java heap, so the number +of events that can be retained is very limited. + +As of Apache NiFi 1.2.0, a third option is available: org.apache.nifi.provenance.WriteAheadProvenanceRepository. +This implementation was created to replace the PersistentProvenanceRepository. The PersistentProvenanceRepository was originally written with the simple goal of persisting +Provenance Events as they are generated and providing the ability to iterate over those events sequentially. Later, it was desired to be able to compress the data so that +more data could be stored. After that, the ability to index and query the data was added. As requirements evolved over time, the repository kept changing without any major +redesigns. When used in a NiFi instance that is responsible for processing large volumes of small FlowFiles, the PersistentProvenanceRepository can quickly become a bottleneck. +The WriteAheadProvenanceRepository was then written to provide the same capabilities as the PersistentProvenanceRepository while providing far better performance. +Changing to the WriteAheadProvenanceRepository is easy to accomplish, as the two repositories support most of the same properties. +*Note Well*, however, the follow caveat: The WriteAheadProvenanceRepository will make use of the Provenance data stored by the PersistentProvenanceRepository. However, the +PersistentProvenanceRepository may not be able to read the data written by the WriteAheadProvenanceRepository. Therefore, once the Provenance Repository is changed to use +the WriteAheadProvenanceRepository, it cannot be changed back to the PersistentProvenanceRepository without deleting the data in the Provenance Repository. It is therefore +recommended that before changing the implementation, users ensure that their version of NiFi is stable, in case any issue arises that causes the user to need to roll back to +a previous version of NiFi that did not support the WriteAheadProvenanceRepository. It is for this reason that the default is still set to the PersistentProvenanceRepository +at this time. |==== === Persistent Provenance Repository Properties @@ -2115,6 +2133,70 @@ Providing three total locations, including _nifi.provenance.repository.director |nifi.provenance.repository.buffer.size|The Provenance Repository buffer size. The default value is 100000. |==== +=== Write Ahead Provenance Repository Properties + +|==== +|*Property*|*Description* +|nifi.provenance.repository.directory.default*|The location of the Provenance Repository. The default value is ./provenance_repository. + + + + *NOTE*: Multiple provenance repositories can be specified by using the *_nifi.provenance.repository.directory._* prefix with unique suffixes and separate paths as values. + + + + For example, to provide two additional locations to act as part of the provenance repository, a user could also specify additional properties with keys of: + + + + nifi.provenance.repository.directory.provenance1=/repos/provenance1 + + nifi.provenance.repository.directory.provenance2=/repos/provenance2 + + + + Providing three total locations, including _nifi.provenance.repository.directory.default_. +|nifi.provenance.repository.max.storage.time|The maximum amount of time to keep data provenance information. The default value is 24 hours. +|nifi.provenance.repository.max.storage.size|The maximum amount of data provenance information to store at a time. + The default is 1 GB. The Data Provenance capability can consume a great deal of storage space because so much data is kept. + For production environments, values of 1-2 TB or more is not uncommon. The repository will write to a single "event file" (or set of + "event files" if multiple storage locations are defined, as described above) for some period of time (defined by the + nifi.provenance.repository.rollover.time and nifi.provenance.repository.rollover.size properties). Data is always aged off one file at a time, + so it is not advisable to write to a single "event file" for a tremendous amount of time, as it will prevent old data from aging off as smoothly. +|nifi.provenance.repository.rollover.time|The amount of time to wait before rolling over the "event file" that the repository is writing to. +|nifi.provenance.repository.rollover.size|The amount of data to write to a single "event file." The default value is 100 MB. For production + environments where a very large amount of Data Provenance is generated, a value of 1 GB is also very reasonable. +|nifi.provenance.repository.query.threads|The number of threads to use for Provenance Repository queries. The default value is 2. +|nifi.provenance.repository.index.threads|The number of threads to use for indexing Provenance events so that they are searchable. The default value is 1. + For flows that operate on a very high number of FlowFiles, the indexing of Provenance events could become a bottleneck. If this happens, increasing the + value of this property may increase the rate at which the Provenance Repository is able to process these records, resulting in better overall throughput. + It is advisable to use at least 1 thread per storage location (i.e., if there are 3 storage locations, at least 3 threads should be used). For high + throughput environments, where more CPU and disk I/O is available, it may make sense to increase this value significantly. Typically going beyond + 2-4 threads per storage location is not valuable. However, this can be tuned depending on the CPU resources available compared to the I/O resources. +|nifi.provenance.repository.compress.on.rollover|Indicates whether to compress the provenance information when an "event file" is rolled over. The default value is _true_. +|nifi.provenance.repository.always.sync|If set to _true_, any change to the repository will be synchronized to the disk, meaning that NiFi will ask the operating system + not to cache the information. This is very expensive and can significantly reduce NiFi performance. However, if it is _false_, there could be the potential for data + loss if either there is a sudden power loss or the operating system crashes. The default value is _false_. +|nifi.provenance.repository.indexed.fields|This is a comma-separated list of the fields that should be indexed and made searchable. + Fields that are not indexed will not be searchable. Valid fields are: EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, + AlternateIdentifierURI, Relationship, Details. The default value is: EventType, FlowFileUUID, Filename, ProcessorID. +|nifi.provenance.repository.indexed.attributes|This is a comma-separated list of FlowFile Attributes that should be indexed and made searchable. It is blank by default. + But some good examples to consider are 'filename' and 'mime.type' as well as any custom attritubes you might use which are valuable for your use case. +|nifi.provenance.repository.index.shard.size|The repository uses Apache Lucene to performing indexing and searching capabilities. This value indicates how large a Lucene Index should + become before the Repository starts writing to a new Index. Large values for the shard size will result in more Java heap usage when searching the Provenance Repository but should + provide better performance. The default value is 500 MB. However, this is due to the fact that defaults are tuned for very small environments where most users begin to use NiFi. + For production environments, it is advisable to change this value to *4 to 8 GB*. Once all Provenance Events in the index have been aged off from the "event files," the index + will be destroyed as well. +|nifi.provenance.repository.max.attribute.length|Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from the repository. + If the length of any attribute exceeds this value, it will be truncated when the event is retrieved. The default is 65536. +|nifi.provenance.repository.concurrent.merge.threads|Apache Lucene creates several "segments" in an Index. These segments are periodically merged together in order to provide faster + querying. This property specifies the maximum number of threads that are allowed to be used for *each* of the storage directories. The default value is 2. For high throughput + environments, it is advisable to set the number of index threads larger than the number of merge threads * the number of storage locations. For example, if there are 2 storage + locations and the number of index threads is set to 8, then the number of merge threads should likely be less than 4. While it is not critical that this be done, setting the + number of merge threads larger than this can result in all index threads being used to merge, which would cause the NiFi flow to periodically pause while indexing is happening, + resulting in some data being processed with much higher latency than other data. +|nifi.provenance.repository.warm.cache.frequency|Each time that a Provenance query is run, the query must first search the Apache Lucene indices (at least, in most cases - there are + some queries that are run often and the results are cached to avoid searching the Lucene indices). When a Lucene index is opened for the first time, it can be very expensive and take + several seconds. This is compounded by having many different indices, and can result in a Provenance query taking much longer. After the index has been opened, the Operating System's + disk cache will typically hold onto enough data to make re-opening the index much faster - at least for a period of time, until the disk cache evicts this data. If this value is set, + NiFi will periodically open each Lucene index and then close it, in order to "warm" the cache. This will result in far faster queries when the Provenance Repository is large. As with + all great things, though, it comes with a cost. Warming the cache does take some CPU resources, but more importantly it will evict other data from the Operating System disk cache and + will result in reading (potentially a great deal of) data from the disk. This can result in lower NiFi performance. However, if NiFi is running in an environment where CPU and disk + are not fully utilized, this feature can result in far faster Provenance queries. +|==== + + === Component Status Repository The Component Status Repository contains the information for the Component Status History tool in the User Interface. These diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/Triggerable.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/Triggerable.java index 4b3149b213..5255c0504e 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/Triggerable.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/Triggerable.java @@ -25,7 +25,7 @@ import org.apache.nifi.processor.exception.ProcessException; public interface Triggerable { - public static final long MINIMUM_SCHEDULING_NANOS = 30000L; + public static final long MINIMUM_SCHEDULING_NANOS = 1L; /** *

diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/IdentifierLookup.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/IdentifierLookup.java new file mode 100644 index 0000000000..6d548d2f4a --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/IdentifierLookup.java @@ -0,0 +1,88 @@ +/* + * 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.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Provides a mechanism for obtaining the identifiers of components, queues, etc. + */ +public interface IdentifierLookup { + + /** + * @return the identifiers of components that may generate Provenance Events + */ + List getComponentIdentifiers(); + + /** + * @return a list of component types that may generate Provenance Events + */ + List getComponentTypes(); + + /** + * + * @return the identifiers of FlowFile Queues that are in the flow + */ + List getQueueIdentifiers(); + + default Map invertQueueIdentifiers() { + return invertList(getQueueIdentifiers()); + } + + default Map invertComponentTypes() { + return invertList(getComponentTypes()); + } + + default Map invertComponentIdentifiers() { + return invertList(getComponentIdentifiers()); + } + + default Map invertList(final List values) { + final Map inverted = new HashMap<>(values.size()); + for (int i = 0; i < values.size(); i++) { + inverted.put(values.get(i), i); + } + return inverted; + } + + + public static final IdentifierLookup EMPTY = new IdentifierLookup() { + @Override + public List getComponentIdentifiers() { + return Collections.emptyList(); + } + + @Override + public List getComponentTypes() { + return Collections.emptyList(); + } + + @Override + public List getQueueIdentifiers() { + return Collections.emptyList(); + } + + @Override + public Map invertList(List values) { + return Collections.emptyMap(); + } + }; +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java index 6a5954a16f..516a36defd 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java @@ -34,12 +34,13 @@ public interface ProvenanceRepository extends ProvenanceEventRepository { * Performs any initialization needed. This should be called only by the * framework. * - * @param eventReporter to report to - * @param authorizer the authorizer to use for authorizing individual events + * @param eventReporter to report to + * @param authorizer the authorizer to use for authorizing individual events * @param resourceFactory the resource factory to use for generating Provenance Resource objects for authorization purposes + * @param identifierLookup a mechanism for looking up identifiers in the flow * @throws java.io.IOException if unable to initialize */ - void initialize(EventReporter eventReporter, Authorizer authorizer, ProvenanceAuthorizableFactory resourceFactory) throws IOException; + void initialize(EventReporter eventReporter, Authorizer authorizer, ProvenanceAuthorizableFactory resourceFactory, IdentifierLookup identifierLookup) throws IOException; ProvenanceEventRecord getEvent(long id, NiFiUser user) throws IOException; diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java index 4d0f9919d4..ad480be0a3 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/lineage/ComputeLineageResult.java @@ -18,6 +18,7 @@ package org.apache.nifi.provenance.lineage; import java.util.Date; import java.util.List; +import java.util.concurrent.TimeUnit; /** * @@ -55,4 +56,6 @@ public interface ComputeLineageResult { * @return Indicates whether or not the lineage has finished running */ boolean isFinished(); + + boolean awaitCompletion(long time, TimeUnit unit) throws InterruptedException; } diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.java index 3519c1468a..4db8e0ff2c 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/Query.java @@ -93,4 +93,8 @@ public class Query { public String toString() { return "Query[ " + searchTerms + " ]"; } + + public boolean isEmpty() { + return searchTerms.isEmpty() && maxFileSize == null && minFileSize == null && startDate == null && endDate == null; + } } diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java index 0079433ec5..cc84ea17a8 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/search/QueryResult.java @@ -18,6 +18,7 @@ package org.apache.nifi.provenance.search; import java.util.Date; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.nifi.provenance.ProvenanceEventRecord; @@ -60,4 +61,6 @@ public interface QueryResult { * @return Indicates whether or not the query has finished running */ boolean isFinished(); + + boolean awaitCompletion(long time, TimeUnit unit) throws InterruptedException; } diff --git a/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java b/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java index 9bc5f0ecd7..53c3c2e62f 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java +++ b/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java @@ -56,7 +56,7 @@ public class MockProvenanceRepository implements ProvenanceRepository { } @Override - public void initialize(EventReporter eventReporter, Authorizer authorizer, ProvenanceAuthorizableFactory resourceFactory) throws IOException { + public void initialize(EventReporter eventReporter, Authorizer authorizer, ProvenanceAuthorizableFactory resourceFactory, IdentifierLookup idLookup) throws IOException { } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java index 1ff1a2ffc8..df87de59c5 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockFlowFile.java @@ -63,7 +63,29 @@ public class MockFlowFile implements FlowFileRecord { } public MockFlowFile(final long id, final FlowFile toCopy) { - this(id); + this.creationTime = System.nanoTime(); + this.id = id; + entryDate = System.currentTimeMillis(); + + final Map attributesToCopy = toCopy.getAttributes(); + String filename = attributesToCopy.get(CoreAttributes.FILENAME.key()); + if (filename == null) { + filename = String.valueOf(System.nanoTime()) + ".mockFlowFile"; + } + attributes.put(CoreAttributes.FILENAME.key(), filename); + + String path = attributesToCopy.get(CoreAttributes.PATH.key()); + if (path == null) { + path = "target"; + } + attributes.put(CoreAttributes.PATH.key(), path); + + String uuid = attributesToCopy.get(CoreAttributes.UUID.key()); + if (uuid == null) { + uuid = UUID.randomUUID().toString(); + } + attributes.put(CoreAttributes.UUID.key(), uuid); + attributes.putAll(toCopy.getAttributes()); final byte[] dataToCopy = ((MockFlowFile) toCopy).data; this.data = new byte[dataToCopy.length]; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java index ff42f47748..1177dad1af 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java @@ -111,7 +111,7 @@ public abstract class AbstractPort implements Port { yieldPeriod = new AtomicReference<>("1 sec"); yieldExpiration = new AtomicLong(0L); schedulingPeriod = new AtomicReference<>("0 millis"); - schedulingNanos = new AtomicLong(30000); + schedulingNanos = new AtomicLong(MINIMUM_SCHEDULING_NANOS); scheduledState = new AtomicReference<>(ScheduledState.STOPPED); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java index 18bcc3cb4d..34ffbac62e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java @@ -100,7 +100,7 @@ public class StandardFunnel implements Funnel { yieldPeriod = new AtomicReference<>("250 millis"); yieldExpiration = new AtomicLong(0L); schedulingPeriod = new AtomicReference<>("0 millis"); - schedulingNanos = new AtomicLong(30000); + schedulingNanos = new AtomicLong(MINIMUM_SCHEDULING_NANOS); name = new AtomicReference<>("Funnel"); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 7fd85b9640..191fc65fa7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -152,6 +152,7 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardProcessorInitializationContext; import org.apache.nifi.processor.StandardValidationContextFactory; +import org.apache.nifi.provenance.IdentifierLookup; import org.apache.nifi.provenance.ProvenanceAuthorizableFactory; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventType; @@ -233,10 +234,12 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import static java.util.Objects.requireNonNull; -public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider { +public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, + QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider, IdentifierLookup { // default repository implementations public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository"; @@ -454,7 +457,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R try { this.provenanceRepository = createProvenanceRepository(nifiProperties); - this.provenanceRepository.initialize(createEventReporter(bulletinRepository), authorizer, this); + this.provenanceRepository.initialize(createEventReporter(bulletinRepository), authorizer, this, this); } catch (final Exception e) { throw new RuntimeException("Unable to create Provenance Repository", e); } @@ -3886,6 +3889,39 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R return replayEvent; } + @Override + public List getComponentIdentifiers() { + final List componentIds = new ArrayList<>(); + getGroup(getRootGroupId()).findAllProcessors().stream() + .forEach(proc -> componentIds.add(proc.getIdentifier())); + getGroup(getRootGroupId()).getInputPorts().stream() + .forEach(port -> componentIds.add(port.getIdentifier())); + getGroup(getRootGroupId()).getOutputPorts().stream() + .forEach(port -> componentIds.add(port.getIdentifier())); + + return componentIds; + } + + @Override + @SuppressWarnings("rawtypes") + public List getComponentTypes() { + final Set procClasses = ExtensionManager.getExtensions(Processor.class); + final List componentTypes = new ArrayList<>(procClasses.size() + 2); + componentTypes.add(ProvenanceEventRecord.REMOTE_INPUT_PORT_TYPE); + componentTypes.add(ProvenanceEventRecord.REMOTE_OUTPUT_PORT_TYPE); + procClasses.stream() + .map(procClass -> procClass.getSimpleName()) + .forEach(componentType -> componentTypes.add(componentType)); + return componentTypes; + } + + @Override + public List getQueueIdentifiers() { + return getAllQueues().stream() + .map(q -> q.getIdentifier()) + .collect(Collectors.toList()); + } + public boolean isConnected() { rwLock.readLock().lock(); try { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index 7108cfe81d..67df53952c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java @@ -911,6 +911,7 @@ public class FileSystemRepository implements ContentRepository { } bytesWritten += len; + scc.setLength(bytesWritten + initialLength); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 54987b9a8e..16a6534410 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -16,6 +16,32 @@ */ package org.apache.nifi.controller.repository; +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +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.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + import org.apache.commons.io.IOUtils; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; @@ -23,6 +49,7 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.repository.claim.ContentClaim; +import org.apache.nifi.controller.repository.claim.ContentClaimWriteCache; import org.apache.nifi.controller.repository.claim.ResourceClaim; import org.apache.nifi.controller.repository.io.DisableOnCloseInputStream; import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream; @@ -53,32 +80,6 @@ import org.apache.nifi.stream.io.StreamUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedOutputStream; -import java.io.ByteArrayInputStream; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -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.Objects; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - /** *

* Provides a ProcessSession that ensures all accesses, changes and transfers @@ -143,6 +144,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE private final Map forkEventBuilders = new HashMap<>(); private Checkpoint checkpoint = new Checkpoint(); + private final ContentClaimWriteCache claimCache; public StandardProcessSession(final ProcessContext context) { this.context = context; @@ -180,7 +182,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE context.getProvenanceRepository(), this); this.sessionId = idGenerator.getAndIncrement(); this.connectableDescription = description; - + this.claimCache = new ContentClaimWriteCache(context.getContentRepository()); LOG.trace("Session {} created for {}", this, connectableDescription); processingStartTime = System.nanoTime(); } @@ -312,6 +314,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE final long commitStartNanos = System.nanoTime(); resetReadClaim(); + try { + claimCache.flush(); + } finally { + claimCache.reset(); + } final long updateProvenanceStart = System.nanoTime(); updateProvenanceRepo(checkpoint); @@ -375,7 +382,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE updateEventRepository(checkpoint); final long updateEventRepositoryFinishNanos = System.nanoTime(); - final long updateEventRepositoryNanos = updateEventRepositoryFinishNanos - claimRemovalFinishNanos; + final long updateEventRepositoryNanos = updateEventRepositoryFinishNanos - flowFileRepoUpdateFinishNanos; // transfer the flowfiles to the connections' queues. final Map> recordMap = new HashMap<>(); @@ -454,7 +461,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE } catch (final Exception e1) { e.addSuppressed(e1); } - throw e; + + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } else { + throw new ProcessException(e); + } } } @@ -904,6 +916,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE } } + try { + claimCache.reset(); + } catch (IOException e1) { + LOG.warn("{} Attempted to close Output Stream for {} due to session rollback but close failed", this, this.connectableDescription, e1); + } + final Set recordsToHandle = new HashSet<>(); recordsToHandle.addAll(records.values()); if (rollbackCheckpoint) { @@ -2033,6 +2051,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE } } + claimCache.flush(claim); final InputStream rawInStream = context.getContentRepository().read(claim); if (currentReadClaimStream != null) { @@ -2047,6 +2066,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE // reuse the same InputStream for the next FlowFile return new DisableOnCloseInputStream(currentReadClaimStream); } else { + claimCache.flush(claim); final InputStream rawInStream = context.getContentRepository().read(claim); try { StreamUtils.skip(rawInStream, offset); @@ -2077,6 +2097,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE try { ensureNotAppending(record.getCurrentClaim()); + claimCache.flush(record.getCurrentClaim()); } catch (final IOException e) { throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e); } @@ -2241,6 +2262,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE try { ensureNotAppending(record.getCurrentClaim()); + claimCache.flush(record.getCurrentClaim()); } catch (final IOException e) { throw new FlowFileAccessException("Unable to read from source " + source + " due to " + e.toString(), e); } @@ -2334,11 +2356,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE long writtenToFlowFile = 0L; ContentClaim newClaim = null; try { - newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); + newClaim = claimCache.getContentClaim(); claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source); ensureNotAppending(newClaim); - try (final OutputStream stream = context.getContentRepository().write(newClaim); + try (final OutputStream stream = claimCache.write(newClaim); final OutputStream disableOnClose = new DisableOnCloseOutputStream(stream); final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnClose)) { try { @@ -2373,7 +2395,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() .fromFlowFile(record.getCurrent()) .contentClaim(newClaim) - .contentClaimOffset(0) + .contentClaimOffset(Math.max(0, newClaim.getLength() - writtenToFlowFile)) .size(writtenToFlowFile) .build(); @@ -2396,6 +2418,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE ContentClaim newClaim = null; try { if (outStream == null) { + claimCache.flush(oldClaim); + try (final InputStream oldClaimIn = context.getContentRepository().read(oldClaim)) { newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); claimLog.debug("Creating ContentClaim {} for 'append' for {}", newClaim, source); @@ -2568,16 +2592,20 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE long writtenToFlowFile = 0L; ContentClaim newClaim = null; try { - newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); + newClaim = claimCache.getContentClaim(); claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source); ensureNotAppending(newClaim); + if (currClaim != null) { + claimCache.flush(currClaim.getResourceClaim()); + } + try (final InputStream is = getInputStream(source, currClaim, record.getCurrentClaimOffset(), true); final InputStream limitedIn = new LimitedInputStream(is, source.getSize()); final InputStream disableOnCloseIn = new DisableOnCloseInputStream(limitedIn); final ByteCountingInputStream countingIn = new ByteCountingInputStream(disableOnCloseIn, bytesRead); - final OutputStream os = context.getContentRepository().write(newClaim); + final OutputStream os = claimCache.write(newClaim); final OutputStream disableOnCloseOut = new DisableOnCloseOutputStream(os); final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnCloseOut)) { @@ -2626,7 +2654,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() .fromFlowFile(record.getCurrent()) .contentClaim(newClaim) - .contentClaimOffset(0L) + .contentClaimOffset(Math.max(0L, newClaim.getLength() - writtenToFlowFile)) .size(writtenToFlowFile) .build(); @@ -2668,8 +2696,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE removeTemporaryClaim(record); - final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()) - .contentClaim(newClaim).contentClaimOffset(claimOffset).size(newSize) + final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() + .fromFlowFile(record.getCurrent()) + .contentClaim(newClaim) + .contentClaimOffset(claimOffset) + .size(newSize) .addAttribute(CoreAttributes.FILENAME.key(), source.toFile().getName()) .build(); record.setWorking(newFile, CoreAttributes.FILENAME.key(), source.toFile().getName()); @@ -2708,7 +2739,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE } removeTemporaryClaim(record); - final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).contentClaim(newClaim).contentClaimOffset(claimOffset).size(newSize).build(); + final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() + .fromFlowFile(record.getCurrent()) + .contentClaim(newClaim) + .contentClaimOffset(claimOffset) + .size(newSize) + .build(); record.setWorking(newFile); return newFile; } @@ -2720,6 +2756,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE try { ensureNotAppending(record.getCurrentClaim()); + claimCache.flush(record.getCurrentClaim()); final long copyCount = context.getContentRepository().exportTo(record.getCurrentClaim(), destination, append, record.getCurrentClaimOffset(), source.getSize()); bytesRead += copyCount; bytesWritten += copyCount; @@ -2741,6 +2778,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE try { ensureNotAppending(record.getCurrentClaim()); + claimCache.flush(record.getCurrentClaim()); } catch (final IOException e) { throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java index b5807ca213..9f8ff989ea 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java @@ -19,6 +19,7 @@ package org.apache.nifi.controller.repository; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -27,6 +28,8 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -74,6 +77,7 @@ import org.wali.WriteAheadRepository; *

*/ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncListener { + private static final String FLOWFILE_REPOSITORY_DIRECTORY_PREFIX = "nifi.flowfile.repository.directory"; private final AtomicLong flowFileSequenceGenerator = new AtomicLong(0L); private final boolean alwaysSync; @@ -82,7 +86,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis private volatile ScheduledFuture checkpointFuture; private final long checkpointDelayMillis; - private final Path flowFileRepositoryPath; + private final SortedSet flowFileRepositoryPaths = new TreeSet<>(); private final int numPartitions; private final ScheduledExecutorService checkpointExecutor; @@ -120,7 +124,6 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis public WriteAheadFlowFileRepository() { alwaysSync = false; checkpointDelayMillis = 0l; - flowFileRepositoryPath = null; numPartitions = 0; checkpointExecutor = null; } @@ -129,7 +132,13 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis alwaysSync = Boolean.parseBoolean(nifiProperties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_ALWAYS_SYNC, "false")); // determine the database file path and ensure it exists - flowFileRepositoryPath = nifiProperties.getFlowFileRepositoryPath(); + for (final String propertyName : nifiProperties.getPropertyKeys()) { + if (propertyName.startsWith(FLOWFILE_REPOSITORY_DIRECTORY_PREFIX)) { + final String directoryName = nifiProperties.getProperty(propertyName); + flowFileRepositoryPaths.add(Paths.get(directoryName)); + } + } + numPartitions = nifiProperties.getFlowFileRepositoryPartitions(); checkpointDelayMillis = FormatUtils.getTimeDuration(nifiProperties.getFlowFileRepositoryCheckpointInterval(), TimeUnit.MILLISECONDS); @@ -140,14 +149,17 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis public void initialize(final ResourceClaimManager claimManager) throws IOException { this.claimManager = claimManager; - Files.createDirectories(flowFileRepositoryPath); + for (final Path path : flowFileRepositoryPaths) { + Files.createDirectories(path); + } // TODO: Should ensure that only 1 instance running and pointing at a particular path // TODO: Allow for backup path that can be used if disk out of space?? Would allow a snapshot to be stored on // backup and then the data deleted from the normal location; then can move backup to normal location and // delete backup. On restore, if no files exist in partition's directory, would have to check backup directory serdeFactory = new RepositoryRecordSerdeFactory(claimManager); - wal = new MinimalLockingWriteAheadLog<>(flowFileRepositoryPath, numPartitions, serdeFactory, this); + wal = new MinimalLockingWriteAheadLog<>(flowFileRepositoryPaths, numPartitions, serdeFactory, this); + logger.info("Initialized FlowFile Repository using {} partitions", numPartitions); } @Override @@ -167,12 +179,22 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis @Override public long getStorageCapacity() throws IOException { - return Files.getFileStore(flowFileRepositoryPath).getTotalSpace(); + long capacity = 0L; + for (final Path path : flowFileRepositoryPaths) { + capacity += Files.getFileStore(path).getTotalSpace(); + } + + return capacity; } @Override public long getUsableStorageSpace() throws IOException { - return Files.getFileStore(flowFileRepositoryPath).getUsableSpace(); + long usableSpace = 0L; + for (final Path path : flowFileRepositoryPaths) { + usableSpace += Files.getFileStore(path).getUsableSpace(); + } + + return usableSpace; } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimWriteCache.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimWriteCache.java new file mode 100644 index 0000000000..6b608acc21 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimWriteCache.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.controller.repository.claim; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.Map; +import java.util.Queue; + +import org.apache.nifi.controller.repository.ContentRepository; + +public class ContentClaimWriteCache { + private final ContentRepository contentRepo; + private final Map streamMap = new HashMap<>(); + private final Queue queue = new LinkedList<>(); + private final int bufferSize; + + public ContentClaimWriteCache(final ContentRepository contentRepo) { + this(contentRepo, 8192); + } + + public ContentClaimWriteCache(final ContentRepository contentRepo, final int bufferSize) { + this.contentRepo = contentRepo; + this.bufferSize = bufferSize; + } + + public void reset() throws IOException { + try { + forEachStream(OutputStream::close); + } finally { + streamMap.clear(); + queue.clear(); + } + } + + public ContentClaim getContentClaim() throws IOException { + final ContentClaim contentClaim = queue.poll(); + if (contentClaim != null) { + contentRepo.incrementClaimaintCount(contentClaim); + return contentClaim; + } + + final ContentClaim claim = contentRepo.create(false); + registerStream(claim); + return claim; + } + + private OutputStream registerStream(final ContentClaim contentClaim) throws IOException { + final OutputStream out = contentRepo.write(contentClaim); + final OutputStream buffered = new BufferedOutputStream(out, bufferSize); + streamMap.put(contentClaim.getResourceClaim(), buffered); + return buffered; + } + + public OutputStream write(final ContentClaim claim) throws IOException { + OutputStream out = streamMap.get(claim.getResourceClaim()); + if (out == null) { + out = registerStream(claim); + } + + if (!(claim instanceof StandardContentClaim)) { + // we know that we will only create Content Claims that are of type StandardContentClaim, so if we get anything + // else, just throw an Exception because it is not valid for this Repository + throw new IllegalArgumentException("Cannot write to " + claim + " because that Content Claim does belong to this Claim Cache"); + } + + final StandardContentClaim scc = (StandardContentClaim) claim; + final long initialLength = Math.max(0L, scc.getLength()); + + final OutputStream bcos = out; + return new OutputStream() { + private long bytesWritten = 0L; + + @Override + public void write(final int b) throws IOException { + bcos.write(b); + bytesWritten++; + scc.setLength(initialLength + bytesWritten); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + bcos.write(b, off, len); + bytesWritten += len; + scc.setLength(initialLength + bytesWritten); + } + + @Override + public void write(final byte[] b) throws IOException { + write(b, 0, b.length); + } + + @Override + public void flush() throws IOException { + // do nothing - do not flush underlying stream. + } + + @Override + public void close() throws IOException { + queue.offer(claim); + } + }; + } + + public void flush(final ContentClaim contentClaim) throws IOException { + if (contentClaim == null) { + return; + } + + flush(contentClaim.getResourceClaim()); + } + + public void flush(final ResourceClaim claim) throws IOException { + final OutputStream out = streamMap.get(claim); + if (out != null) { + out.flush(); + } + } + + public void flush() throws IOException { + forEachStream(OutputStream::flush); + } + + private void forEachStream(final StreamProcessor proc) throws IOException { + IOException exception = null; + + for (final OutputStream out : streamMap.values()) { + try { + proc.process(out); + } catch (final IOException ioe) { + if (exception == null) { + exception = ioe; + } else { + ioe.addSuppressed(exception); + exception = ioe; + } + } + } + + if (exception != null) { + throw exception; + } + } + + private interface StreamProcessor { + void process(final OutputStream out) throws IOException; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java index 76c208dd9a..b218ee60fe 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java @@ -64,6 +64,39 @@ public class ContentClaimFieldMap implements Record { return schema; } + @Override + public int hashCode() { + return (int) (31 + contentClaimOffset + 21 * resourceClaimFieldMap.hashCode()); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + + ContentClaimFieldMap other = (ContentClaimFieldMap) obj; + if (contentClaimOffset != other.contentClaimOffset) { + return false; + } + + if (resourceClaimFieldMap == null) { + if (other.resourceClaimFieldMap != null) { + return false; + } + } else if (!resourceClaimFieldMap.equals(other.resourceClaimFieldMap)) { + return false; + } + + return true; + } + @Override public String toString() { return "ContentClaimFieldMap[" + contentClaim + "]"; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java index c11353b552..93fa4e4133 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller.repository.schema; +import org.apache.nifi.controller.repository.RepositoryRecordType; import org.apache.nifi.repository.schema.NamedValue; import org.apache.nifi.repository.schema.Record; import org.apache.nifi.repository.schema.RecordSchema; @@ -39,7 +40,10 @@ public class RepositoryRecordUpdate implements Record { @Override public Object getFieldValue(final String fieldName) { if (RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V2.equals(fieldName)) { - final String actionType = (String) fieldMap.getFieldValue(RepositoryRecordSchema.ACTION_TYPE); + String actionType = (String) fieldMap.getFieldValue(RepositoryRecordSchema.ACTION_TYPE); + if (RepositoryRecordType.CONTENTMISSING.name().equals(actionType)) { + actionType = RepositoryRecordType.DELETE.name(); + } final UpdateType updateType = UpdateType.valueOf(actionType); final String actionName; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java index 0c4972ba35..22684a6d88 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java @@ -83,7 +83,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { for (int i = 0; i < maxThreadCount; i++) { final Runnable eventDrivenTask = new EventDrivenTask(workerQueue); - flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 30000, TimeUnit.NANOSECONDS); + flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 1L, TimeUnit.NANOSECONDS); } } @@ -132,7 +132,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { final int tasksToAdd = maxThreadCount - oldMax; for (int i = 0; i < tasksToAdd; i++) { final Runnable eventDrivenTask = new EventDrivenTask(workerQueue); - flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 30000, TimeUnit.NANOSECONDS); + flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 1L, TimeUnit.NANOSECONDS); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/ProcessorStatusDescriptor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/ProcessorStatusDescriptor.java index 05c32e1ee5..59a4b1b377 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/ProcessorStatusDescriptor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/ProcessorStatusDescriptor.java @@ -87,6 +87,13 @@ public enum ProcessorStatusDescriptor { Formatter.DURATION, s -> TimeUnit.MILLISECONDS.convert(s.getProcessingNanos(), TimeUnit.NANOSECONDS))), + TASK_NANOS(new StandardMetricDescriptor( + "taskNanos", + "Total Task Time (nanos)", + "The total number of thread-nanoseconds that the Processor has used to complete its tasks in the past 5 minutes", + Formatter.COUNT, + ProcessorStatus::getProcessingNanos), false), + FLOWFILES_REMOVED(new StandardMetricDescriptor( "flowFilesRemoved", "FlowFiles Removed (5 mins)", @@ -122,35 +129,50 @@ public enum ProcessorStatusDescriptor { } )), - AVERAGE_TASK_MILLIS(new StandardMetricDescriptor( - "averageTaskMillis", - "Average Task Duration", - "The average duration it took this Processor to complete a task, as averaged over the past 5 minutes", - Formatter.DURATION, - s -> s.getInvocations() == 0 ? 0 : TimeUnit.MILLISECONDS.convert(s.getProcessingNanos(), TimeUnit.NANOSECONDS) / s.getInvocations(), + AVERAGE_TASK_NANOS(new StandardMetricDescriptor( + "averageTaskNanos", + "Average Task Duration (nanoseconds)", + "The average number of nanoseconds it took this Processor to complete a task, over the past 5 minutes", + Formatter.COUNT, + s -> s.getInvocations() == 0 ? 0 : s.getProcessingNanos() / s.getInvocations(), new ValueReducer() { @Override public Long reduce(final List values) { - long procMillis = 0L; + long procNanos = 0L; int invocations = 0; for (final StatusSnapshot snapshot : values) { - procMillis += snapshot.getStatusMetrics().get(TASK_MILLIS.getDescriptor()).longValue(); - invocations += snapshot.getStatusMetrics().get(TASK_COUNT.getDescriptor()).intValue(); + final Long taskNanos = snapshot.getStatusMetrics().get(TASK_NANOS.getDescriptor()); + if (taskNanos != null) { + procNanos += taskNanos.longValue(); + } + + final Long taskInvocations = snapshot.getStatusMetrics().get(TASK_COUNT.getDescriptor()); + if (taskInvocations != null) { + invocations += taskInvocations.intValue(); + } } if (invocations == 0) { return 0L; } - return procMillis / invocations; + return procNanos / invocations; } })); - private MetricDescriptor descriptor; + + + private final MetricDescriptor descriptor; + private final boolean visible; private ProcessorStatusDescriptor(final MetricDescriptor descriptor) { + this(descriptor, true); + } + + private ProcessorStatusDescriptor(final MetricDescriptor descriptor, final boolean visible) { this.descriptor = descriptor; + this.visible = visible; } public String getField() { @@ -160,4 +182,8 @@ public enum ProcessorStatusDescriptor { public MetricDescriptor getDescriptor() { return descriptor; } + + public boolean isVisible() { + return visible; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java index 4c620d1c7f..320397267b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java @@ -93,7 +93,9 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit snapshot.setTimestamp(capture.getCaptureDate()); for (final ProcessorStatusDescriptor descriptor : ProcessorStatusDescriptor.values()) { - snapshot.addStatusMetric(descriptor.getDescriptor(), descriptor.getDescriptor().getValueFunction().getValue(status)); + if (descriptor.isVisible()) { + snapshot.addStatusMetric(descriptor.getDescriptor(), descriptor.getDescriptor().getValueFunction().getValue(status)); + } } history.addStatusSnapshot(snapshot); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java index 3bc235643e..01f3c8cb0a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java @@ -82,6 +82,13 @@ public class ContinuallyRunProcessorTask implements Callable { return procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || !Connectables.hasNonLoopConnection(procNode) || Connectables.flowFilesQueued(procNode); } + private boolean isBackPressureEngaged() { + return procNode.getIncomingConnections().stream() + .filter(con -> con.getSource() == procNode) + .map(con -> con.getFlowFileQueue()) + .anyMatch(queue -> queue.isFull()); + } + @Override public Boolean call() { // make sure processor is not yielded @@ -127,6 +134,7 @@ public class ContinuallyRunProcessorTask implements Callable { scheduleState.incrementActiveThreadCount(); final long startNanos = System.nanoTime(); + final long finishIfBackpressureEngaged = startNanos + (batchNanos / 25L); final long finishNanos = startNanos + batchNanos; int invocationCount = 0; try { @@ -140,10 +148,16 @@ public class ContinuallyRunProcessorTask implements Callable { return false; } - if (System.nanoTime() > finishNanos) { + final long nanoTime = System.nanoTime(); + if (nanoTime > finishNanos) { return false; } + if (nanoTime > finishIfBackpressureEngaged && isBackPressureEngaged()) { + return false; + } + + if (!isWorkToDo(procNode)) { break; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java index 7edb0e7592..845ca56007 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java @@ -34,10 +34,15 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; import java.nio.file.StandardOpenOption; +import java.text.NumberFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Random; import java.util.concurrent.TimeUnit; import org.apache.nifi.controller.repository.claim.ContentClaim; @@ -56,8 +61,6 @@ import ch.qos.logback.classic.Level; import ch.qos.logback.classic.Logger; import ch.qos.logback.classic.spi.ILoggingEvent; import ch.qos.logback.core.read.ListAppender; -import java.util.HashMap; -import java.util.Map; public class TestFileSystemRepository { @@ -88,6 +91,38 @@ public class TestFileSystemRepository { repository.shutdown(); } + @Test + public void testWritePerformance() throws IOException { + final long bytesToWrite = 1_000_000_000L; + final int contentSize = 100; + + final int iterations = (int) (bytesToWrite / contentSize); + final byte[] content = new byte[contentSize]; + final Random random = new Random(); + random.nextBytes(content); + + // final ContentClaimWriteCache cache = new ContentClaimWriteCache(repository); + + final long start = System.nanoTime(); + for (int i = 0; i < iterations; i++) { + final ContentClaim claim = repository.create(false); + try (final OutputStream out = repository.write(claim)) { + out.write(content); + } + // final ContentClaim claim = cache.getContentClaim(); + // try (final OutputStream out = cache.write(claim)) { + // out.write(content); + // } + } + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + + final long mb = bytesToWrite / (1024 * 1024); + final long seconds = millis / 1000L; + final double mbps = (double) mb / (double) seconds; + System.out.println("Took " + millis + " millis to write " + contentSize + " bytes " + iterations + " times (total of " + + NumberFormat.getNumberInstance(Locale.US).format(bytesToWrite) + " bytes) for a write rate of " + mbps + " MB/s"); + } + @Test public void testMinimalArchiveCleanupIntervalHonoredAndLogged() throws Exception { // We are going to construct our own repository using different properties, so diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java index 9070d0c408..e0c9ffee5e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java @@ -86,6 +86,7 @@ import org.apache.nifi.util.NiFiProperties; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; @@ -987,6 +988,7 @@ public class TestStandardProcessSession { } @Test + @Ignore public void testManyFilesOpened() throws IOException { StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000]; @@ -1672,9 +1674,9 @@ public class TestStandardProcessSession { @Override public int incrementClaimaintCount(ContentClaim claim) { - final AtomicInteger count = claimantCounts.get(claim); + AtomicInteger count = claimantCounts.get(claim); if (count == null) { - throw new IllegalArgumentException("Unknown Claim: " + claim); + count = new AtomicInteger(0); } return count.incrementAndGet(); } @@ -1835,6 +1837,11 @@ public class TestStandardProcessSession { fos.write(b); ((StandardContentClaim) claim).setLength(claim.getLength() + b.length); } + + @Override + public void close() throws IOException { + super.close(); + } }; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java index 65258223bc..329b26819e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java @@ -18,6 +18,7 @@ package org.apache.nifi.controller.repository; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.when; @@ -33,12 +34,17 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Queue; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.StandardFlowFileQueue; +import org.apache.nifi.controller.queue.DropFlowFileStatus; import org.apache.nifi.controller.queue.FlowFileQueue; +import org.apache.nifi.controller.queue.ListFlowFileStatus; import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.repository.claim.ContentClaim; import org.apache.nifi.controller.repository.claim.ResourceClaim; @@ -47,15 +53,22 @@ import org.apache.nifi.controller.repository.claim.StandardContentClaim; import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager; import org.apache.nifi.controller.swap.StandardSwapContents; import org.apache.nifi.controller.swap.StandardSwapSummary; +import org.apache.nifi.flowfile.FlowFilePrioritizer; +import org.apache.nifi.processor.FlowFileFilter; +import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.file.FileUtils; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.wali.MinimalLockingWriteAheadLog; +import org.wali.WriteAheadRepository; public class TestWriteAheadFlowFileRepository { @@ -74,6 +87,254 @@ public class TestWriteAheadFlowFileRepository { } } + + @Test + @Ignore("Intended only for local performance testing before/after making changes") + public void testUpdatePerformance() throws IOException, InterruptedException { + final FlowFileQueue queue = new FlowFileQueue() { + + @Override + public String getIdentifier() { + return "4444"; + } + + @Override + public List getPriorities() { + return null; + } + + @Override + public SwapSummary recoverSwappedFlowFiles() { + return null; + } + + @Override + public void purgeSwapFiles() { + } + + @Override + public void setPriorities(List newPriorities) { + } + + @Override + public void setBackPressureObjectThreshold(long maxQueueSize) { + } + + @Override + public long getBackPressureObjectThreshold() { + return 0; + } + + @Override + public void setBackPressureDataSizeThreshold(String maxDataSize) { + } + + @Override + public String getBackPressureDataSizeThreshold() { + return null; + } + + @Override + public QueueSize size() { + return null; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public boolean isActiveQueueEmpty() { + return false; + } + + @Override + public QueueSize getUnacknowledgedQueueSize() { + return null; + } + + @Override + public void acknowledge(FlowFileRecord flowFile) { + } + + @Override + public void acknowledge(Collection flowFiles) { + } + + @Override + public boolean isFull() { + return false; + } + + @Override + public void put(FlowFileRecord file) { + } + + @Override + public void putAll(Collection files) { + } + + @Override + public FlowFileRecord poll(Set expiredRecords) { + return null; + } + + @Override + public List poll(int maxResults, Set expiredRecords) { + return null; + } + + @Override + public long drainQueue(Queue sourceQueue, List destination, int maxResults, Set expiredRecords) { + return 0; + } + + @Override + public List poll(FlowFileFilter filter, Set expiredRecords) { + return null; + } + + @Override + public String getFlowFileExpiration() { + return null; + } + + @Override + public int getFlowFileExpiration(TimeUnit timeUnit) { + return 0; + } + + @Override + public void setFlowFileExpiration(String flowExpirationPeriod) { + } + + @Override + public DropFlowFileStatus dropFlowFiles(String requestIdentifier, String requestor) { + return null; + } + + @Override + public DropFlowFileStatus getDropFlowFileStatus(String requestIdentifier) { + return null; + } + + @Override + public DropFlowFileStatus cancelDropFlowFileRequest(String requestIdentifier) { + return null; + } + + @Override + public ListFlowFileStatus listFlowFiles(String requestIdentifier, int maxResults) { + return null; + } + + @Override + public ListFlowFileStatus getListFlowFileStatus(String requestIdentifier) { + return null; + } + + @Override + public ListFlowFileStatus cancelListFlowFileRequest(String requestIdentifier) { + return null; + } + + @Override + public FlowFileRecord getFlowFile(String flowFileUuid) throws IOException { + return null; + } + + @Override + public void verifyCanList() throws IllegalStateException { + } + }; + + + final int numPartitions = 16; + final int numThreads = 8; + final int totalUpdates = 160_000_000; + final int batchSize = 10; + + final Path path = Paths.get("target/minimal-locking-repo"); + deleteRecursively(path.toFile()); + assertTrue(path.toFile().mkdirs()); + + final ResourceClaimManager claimManager = new StandardResourceClaimManager(); + final RepositoryRecordSerdeFactory serdeFactory = new RepositoryRecordSerdeFactory(claimManager); + final WriteAheadRepository repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serdeFactory, null); + final Collection initialRecs = repo.recoverRecords(); + assertTrue(initialRecs.isEmpty()); + + final int updateCountPerThread = totalUpdates / numThreads; + + final Thread[] threads = new Thread[numThreads]; + for (int j = 0; j < 2; j++) { + for (int i = 0; i < numThreads; i++) { + final Thread t = new Thread(new Runnable() { + @Override + public void run() { + final List records = new ArrayList<>(); + final int numBatches = updateCountPerThread / batchSize; + final MockFlowFile baseFlowFile = new MockFlowFile(0L); + + for (int i = 0; i < numBatches; i++) { + records.clear(); + for (int k = 0; k < batchSize; k++) { + final FlowFileRecord flowFile = new MockFlowFile(i % 100_000, baseFlowFile); + final String uuid = flowFile.getAttribute("uuid"); + + final StandardRepositoryRecord record = new StandardRepositoryRecord(null, flowFile); + record.setDestination(queue); + final Map updatedAttrs = Collections.singletonMap("uuid", uuid); + record.setWorking(flowFile, updatedAttrs); + + records.add(record); + } + + try { + repo.update(records, false); + } catch (IOException e) { + e.printStackTrace(); + Assert.fail(e.toString()); + } + } + } + }); + + t.setDaemon(true); + threads[i] = t; + } + + final long start = System.nanoTime(); + for (final Thread t : threads) { + t.start(); + } + for (final Thread t : threads) { + t.join(); + } + + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + if (j == 0) { + System.out.println(millis + " ms to insert " + updateCountPerThread * numThreads + " updates using " + numPartitions + " partitions and " + numThreads + " threads, *as a warmup!*"); + } else { + System.out.println(millis + " ms to insert " + updateCountPerThread * numThreads + " updates using " + numPartitions + " partitions and " + numThreads + " threads"); + } + } + } + + private void deleteRecursively(final File file) { + final File[] children = file.listFiles(); + if (children != null) { + for (final File child : children) { + deleteRecursively(child); + } + } + + file.delete(); + } + + + @Test public void testResourceClaimsIncremented() throws IOException { final ResourceClaimManager claimManager = new StandardResourceClaimManager(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/claim/TestContentClaimWriteCache.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/claim/TestContentClaimWriteCache.java new file mode 100644 index 0000000000..fc08f55c02 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/claim/TestContentClaimWriteCache.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.controller.repository.claim; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.nifi.controller.repository.FileSystemRepository; +import org.apache.nifi.controller.repository.TestFileSystemRepository; +import org.apache.nifi.controller.repository.util.DiskUtils; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.util.NiFiProperties; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestContentClaimWriteCache { + + private FileSystemRepository repository = null; + private StandardResourceClaimManager claimManager = null; + private final File rootFile = new File("target/testContentClaimWriteCache"); + private NiFiProperties nifiProperties; + + @Before + public void setup() throws IOException { + System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile()); + nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null); + if (rootFile.exists()) { + DiskUtils.deleteRecursively(rootFile); + } + repository = new FileSystemRepository(nifiProperties); + claimManager = new StandardResourceClaimManager(); + repository.initialize(claimManager); + repository.purge(); + } + + @After + public void shutdown() throws IOException { + repository.shutdown(); + } + + @Test + public void testFlushWriteCorrectData() throws IOException { + final ContentClaimWriteCache cache = new ContentClaimWriteCache(repository, 4); + + final ContentClaim claim1 = cache.getContentClaim(); + assertNotNull(claim1); + + final OutputStream out = cache.write(claim1); + assertNotNull(out); + out.write("hello".getBytes()); + out.write("good-bye".getBytes()); + + cache.flush(); + + assertEquals(13L, claim1.getLength()); + final InputStream in = repository.read(claim1); + final byte[] buff = new byte[(int) claim1.getLength()]; + StreamUtils.fillBuffer(in, buff); + Assert.assertArrayEquals("hellogood-bye".getBytes(), buff); + + final ContentClaim claim2 = cache.getContentClaim(); + final OutputStream out2 = cache.write(claim2); + assertNotNull(out2); + out2.write("good-day".getBytes()); + out2.write("hello".getBytes()); + + cache.flush(); + + assertEquals(13L, claim2.getLength()); + final InputStream in2 = repository.read(claim2); + final byte[] buff2 = new byte[(int) claim2.getLength()]; + StreamUtils.fillBuffer(in2, buff2); + Assert.assertArrayEquals("good-dayhello".getBytes(), buff2); + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml index 662f87ef5d..cecfabf4ae 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml @@ -98,7 +98,7 @@ 30 secs 100 MB 2 - 1 + 2 true EventType, FlowFileUUID, Filename, ProcessorID, Relationship @@ -106,6 +106,8 @@ false 16 65536 + 2 + 1 hour 100000 diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/AbstractRecordWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/AbstractRecordWriter.java index 2c84861dad..fcc481b9ad 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/AbstractRecordWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/AbstractRecordWriter.java @@ -32,18 +32,26 @@ public abstract class AbstractRecordWriter implements RecordWriter { private static final Logger logger = LoggerFactory.getLogger(AbstractRecordWriter.class); private final File file; + private final String storageLocation; private final TocWriter tocWriter; private final Lock lock = new ReentrantLock(); private volatile boolean dirty = false; private volatile boolean closed = false; - private int recordsWritten = 0; - public AbstractRecordWriter(final File file, final TocWriter writer) throws IOException { logger.trace("Creating Record Writer for {}", file); this.file = file; + this.storageLocation = file.getName(); + this.tocWriter = writer; + } + + public AbstractRecordWriter(final String storageLocation, final TocWriter writer) throws IOException { + logger.trace("Creating Record Writer for {}", storageLocation); + + this.file = null; + this.storageLocation = storageLocation; this.tocWriter = writer; } @@ -51,7 +59,7 @@ public abstract class AbstractRecordWriter implements RecordWriter { public synchronized void close() throws IOException { closed = true; - logger.trace("Closing Record Writer for {}", file == null ? null : file.getName()); + logger.trace("Closing Record Writer for {}", getStorageLocation()); lock(); try { @@ -94,9 +102,8 @@ public abstract class AbstractRecordWriter implements RecordWriter { } } - @Override - public int getRecordsWritten() { - return recordsWritten; + protected String getStorageLocation() { + return storageLocation; } @Override @@ -133,6 +140,7 @@ public abstract class AbstractRecordWriter implements RecordWriter { this.dirty = true; } + @Override public boolean isDirty() { return dirty; } @@ -142,7 +150,7 @@ public abstract class AbstractRecordWriter implements RecordWriter { } @Override - public void sync() throws IOException { + public synchronized void sync() throws IOException { try { if (tocWriter != null) { tocWriter.sync(); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/ByteArraySchemaRecordWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/ByteArraySchemaRecordWriter.java index cae2f40d86..2a42501530 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/ByteArraySchemaRecordWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/ByteArraySchemaRecordWriter.java @@ -18,34 +18,37 @@ package org.apache.nifi.provenance; import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.File; import java.io.IOException; import java.io.OutputStream; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.provenance.schema.EventFieldNames; import org.apache.nifi.provenance.schema.EventRecord; -import org.apache.nifi.provenance.schema.EventRecordFields; import org.apache.nifi.provenance.schema.ProvenanceEventSchema; import org.apache.nifi.provenance.serialization.CompressableRecordWriter; import org.apache.nifi.provenance.toc.TocWriter; import org.apache.nifi.repository.schema.Record; import org.apache.nifi.repository.schema.RecordSchema; import org.apache.nifi.repository.schema.SchemaRecordWriter; -import org.apache.nifi.stream.io.DataOutputStream; public class ByteArraySchemaRecordWriter extends CompressableRecordWriter { private static final RecordSchema eventSchema = ProvenanceEventSchema.PROVENANCE_EVENT_SCHEMA_V1; - private static final RecordSchema contentClaimSchema = new RecordSchema(eventSchema.getField(EventRecordFields.Names.CONTENT_CLAIM).getSubFields()); + private static final RecordSchema contentClaimSchema = new RecordSchema(eventSchema.getField(EventFieldNames.CONTENT_CLAIM).getSubFields()); public static final int SERIALIZATION_VERSION = 1; public static final String SERIALIZATION_NAME = "ByteArraySchemaRecordWriter"; private final SchemaRecordWriter recordWriter = new SchemaRecordWriter(); - public ByteArraySchemaRecordWriter(final File file, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { - super(file, tocWriter, compressed, uncompressedBlockSize); + public ByteArraySchemaRecordWriter(final File file, final AtomicLong idGenerator, final TocWriter tocWriter, final boolean compressed, + final int uncompressedBlockSize) throws IOException { + super(file, idGenerator, tocWriter, compressed, uncompressedBlockSize); } - public ByteArraySchemaRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { - super(out, tocWriter, compressed, uncompressedBlockSize); + public ByteArraySchemaRecordWriter(final OutputStream out, final String storageLocation, final AtomicLong idGenerator, final TocWriter tocWriter, final boolean compressed, + final int uncompressedBlockSize) throws IOException { + super(out, storageLocation, idGenerator, tocWriter, compressed, uncompressedBlockSize); } @Override diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordReader.java new file mode 100644 index 0000000000..612b6c8c37 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordReader.java @@ -0,0 +1,145 @@ +/* + * 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.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.Optional; + +import org.apache.nifi.provenance.schema.EventIdFirstHeaderSchema; +import org.apache.nifi.provenance.schema.LookupTableEventRecord; +import org.apache.nifi.provenance.serialization.CompressableRecordReader; +import org.apache.nifi.provenance.toc.TocReader; +import org.apache.nifi.repository.schema.Record; +import org.apache.nifi.repository.schema.RecordSchema; +import org.apache.nifi.repository.schema.SchemaRecordReader; +import org.apache.nifi.stream.io.LimitingInputStream; +import org.apache.nifi.stream.io.StreamUtils; + +public class EventIdFirstSchemaRecordReader extends CompressableRecordReader { + private RecordSchema schema; // effectively final + private SchemaRecordReader recordReader; // effectively final + + private List componentIds; + private List componentTypes; + private List queueIds; + private List eventTypes; + private long firstEventId; + private long systemTimeOffset; + + public EventIdFirstSchemaRecordReader(final InputStream in, final String filename, final TocReader tocReader, final int maxAttributeChars) throws IOException { + super(in, filename, tocReader, maxAttributeChars); + } + + private void verifySerializationVersion(final int serializationVersion) { + if (serializationVersion > EventIdFirstSchemaRecordWriter.SERIALIZATION_VERSION) { + throw new IllegalArgumentException("Unable to deserialize record because the version is " + serializationVersion + + " and supported versions are 1-" + EventIdFirstSchemaRecordWriter.SERIALIZATION_VERSION); + } + } + + @Override + @SuppressWarnings("unchecked") + protected synchronized void readHeader(final DataInputStream in, final int serializationVersion) throws IOException { + verifySerializationVersion(serializationVersion); + final int eventSchemaLength = in.readInt(); + final byte[] buffer = new byte[eventSchemaLength]; + StreamUtils.fillBuffer(in, buffer); + + try (final ByteArrayInputStream bais = new ByteArrayInputStream(buffer)) { + schema = RecordSchema.readFrom(bais); + } + + recordReader = SchemaRecordReader.fromSchema(schema); + + final int headerSchemaLength = in.readInt(); + final byte[] headerSchemaBuffer = new byte[headerSchemaLength]; + StreamUtils.fillBuffer(in, headerSchemaBuffer); + + final RecordSchema headerSchema; + try (final ByteArrayInputStream bais = new ByteArrayInputStream(headerSchemaBuffer)) { + headerSchema = RecordSchema.readFrom(bais); + } + + final SchemaRecordReader headerReader = SchemaRecordReader.fromSchema(headerSchema); + final Record headerRecord = headerReader.readRecord(in); + componentIds = (List) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.COMPONENT_IDS); + componentTypes = (List) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.COMPONENT_TYPES); + queueIds = (List) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.QUEUE_IDS); + eventTypes = (List) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.EVENT_TYPES); + firstEventId = (Long) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.FIRST_EVENT_ID); + systemTimeOffset = (Long) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.TIMESTAMP_OFFSET); + } + + @Override + protected StandardProvenanceEventRecord nextRecord(final DataInputStream in, final int serializationVersion) throws IOException { + verifySerializationVersion(serializationVersion); + + final long byteOffset = getBytesConsumed(); + final long eventId = in.readInt() + firstEventId; + final int recordLength = in.readInt(); + + return readRecord(in, eventId, byteOffset, recordLength); + } + + private StandardProvenanceEventRecord readRecord(final DataInputStream in, final long eventId, final long startOffset, final int recordLength) throws IOException { + final InputStream limitedIn = new LimitingInputStream(in, recordLength); + + final Record eventRecord = recordReader.readRecord(limitedIn); + if (eventRecord == null) { + return null; + } + + final StandardProvenanceEventRecord deserializedEvent = LookupTableEventRecord.getEvent(eventRecord, getFilename(), startOffset, getMaxAttributeLength(), + firstEventId, systemTimeOffset, componentIds, componentTypes, queueIds, eventTypes); + deserializedEvent.setEventId(eventId); + return deserializedEvent; + } + + private boolean isData(final InputStream in) throws IOException { + in.mark(1); + final int nextByte = in.read(); + in.reset(); + + return nextByte > -1; + } + + @Override + protected Optional readToEvent(final long eventId, final DataInputStream dis, final int serializationVersion) throws IOException { + verifySerializationVersion(serializationVersion); + + while (isData(dis)) { + final long startOffset = getBytesConsumed(); + final long id = dis.readInt() + firstEventId; + final int recordLength = dis.readInt(); + + if (id >= eventId) { + final StandardProvenanceEventRecord event = readRecord(dis, id, startOffset, recordLength); + return Optional.ofNullable(event); + } else { + // This is not the record we want. Skip over it instead of deserializing it. + StreamUtils.skip(dis, recordLength); + } + } + + return Optional.empty(); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordWriter.java new file mode 100644 index 0000000000..bb8d52fef3 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/EventIdFirstSchemaRecordWriter.java @@ -0,0 +1,241 @@ +/* + * 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.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.provenance.schema.EventFieldNames; +import org.apache.nifi.provenance.schema.EventIdFirstHeaderSchema; +import org.apache.nifi.provenance.schema.LookupTableEventRecord; +import org.apache.nifi.provenance.schema.LookupTableEventSchema; +import org.apache.nifi.provenance.serialization.CompressableRecordWriter; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.provenance.toc.TocWriter; +import org.apache.nifi.repository.schema.FieldMapRecord; +import org.apache.nifi.repository.schema.Record; +import org.apache.nifi.repository.schema.RecordSchema; +import org.apache.nifi.repository.schema.SchemaRecordWriter; +import org.apache.nifi.util.timebuffer.LongEntityAccess; +import org.apache.nifi.util.timebuffer.TimedBuffer; +import org.apache.nifi.util.timebuffer.TimestampedLong; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class EventIdFirstSchemaRecordWriter extends CompressableRecordWriter { + private static final Logger logger = LoggerFactory.getLogger(EventIdFirstSchemaRecordWriter.class); + + private static final RecordSchema eventSchema = LookupTableEventSchema.EVENT_SCHEMA; + private static final RecordSchema contentClaimSchema = new RecordSchema(eventSchema.getField(EventFieldNames.CONTENT_CLAIM).getSubFields()); + private static final RecordSchema previousContentClaimSchema = new RecordSchema(eventSchema.getField(EventFieldNames.PREVIOUS_CONTENT_CLAIM).getSubFields()); + private static final RecordSchema headerSchema = EventIdFirstHeaderSchema.SCHEMA; + + public static final int SERIALIZATION_VERSION = 1; + public static final String SERIALIZATION_NAME = "EventIdFirstSchemaRecordWriter"; + private final IdentifierLookup idLookup; + + private final SchemaRecordWriter schemaRecordWriter = new SchemaRecordWriter(); + private final AtomicInteger recordCount = new AtomicInteger(0); + + private final Map componentIdMap; + private final Map componentTypeMap; + private final Map queueIdMap; + private static final Map eventTypeMap; + private static final List eventTypeNames; + + private static final TimedBuffer serializeTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess()); + private static final TimedBuffer lockTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess()); + private static final TimedBuffer writeTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess()); + private static final TimedBuffer bytesWritten = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess()); + private static final AtomicLong totalRecordCount = new AtomicLong(0L); + + private long firstEventId; + private long systemTimeOffset; + + static { + eventTypeMap = new HashMap<>(); + eventTypeNames = new ArrayList<>(); + + int count = 0; + for (final ProvenanceEventType eventType : ProvenanceEventType.values()) { + eventTypeMap.put(eventType.name(), count++); + eventTypeNames.add(eventType.name()); + } + } + + public EventIdFirstSchemaRecordWriter(final File file, final AtomicLong idGenerator, final TocWriter writer, final boolean compressed, + final int uncompressedBlockSize, final IdentifierLookup idLookup) throws IOException { + super(file, idGenerator, writer, compressed, uncompressedBlockSize); + + this.idLookup = idLookup; + componentIdMap = idLookup.invertComponentIdentifiers(); + componentTypeMap = idLookup.invertComponentTypes(); + queueIdMap = idLookup.invertQueueIdentifiers(); + } + + public EventIdFirstSchemaRecordWriter(final OutputStream out, final String storageLocation, final AtomicLong idGenerator, final TocWriter tocWriter, final boolean compressed, + final int uncompressedBlockSize, final IdentifierLookup idLookup) throws IOException { + super(out, storageLocation, idGenerator, tocWriter, compressed, uncompressedBlockSize); + + this.idLookup = idLookup; + componentIdMap = idLookup.invertComponentIdentifiers(); + componentTypeMap = idLookup.invertComponentTypes(); + queueIdMap = idLookup.invertQueueIdentifiers(); + } + + @Override + public StorageSummary writeRecord(final ProvenanceEventRecord record) throws IOException { + if (isDirty()) { + throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository"); + } + + final long serializeStart = System.nanoTime(); + final byte[] serialized; + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(256); + final DataOutputStream dos = new DataOutputStream(baos)) { + writeRecord(record, 0L, dos); + serialized = baos.toByteArray(); + } + + final long lockStart = System.nanoTime(); + final long writeStart; + final long startBytes; + final long endBytes; + final long recordIdentifier; + synchronized (this) { + writeStart = System.nanoTime(); + try { + recordIdentifier = record.getEventId() == -1L ? getIdGenerator().getAndIncrement() : record.getEventId(); + startBytes = getBytesWritten(); + + ensureStreamState(recordIdentifier, startBytes); + + final DataOutputStream out = getBufferedOutputStream(); + final int recordIdOffset = (int) (recordIdentifier - firstEventId); + out.writeInt(recordIdOffset); + out.writeInt(serialized.length); + out.write(serialized); + + recordCount.incrementAndGet(); + endBytes = getBytesWritten(); + } catch (final IOException ioe) { + markDirty(); + throw ioe; + } + } + + if (logger.isDebugEnabled()) { + // Collect stats and periodically dump them if log level is set to at least info. + final long writeNanos = System.nanoTime() - writeStart; + writeTimes.add(new TimestampedLong(writeNanos)); + + final long serializeNanos = lockStart - serializeStart; + serializeTimes.add(new TimestampedLong(serializeNanos)); + + final long lockNanos = writeStart - lockStart; + lockTimes.add(new TimestampedLong(lockNanos)); + bytesWritten.add(new TimestampedLong(endBytes - startBytes)); + + final long recordCount = totalRecordCount.incrementAndGet(); + if (recordCount % 1_000_000 == 0) { + final long sixtySecondsAgo = System.currentTimeMillis() - 60000L; + final Long writeNanosLast60 = writeTimes.getAggregateValue(sixtySecondsAgo).getValue(); + final Long lockNanosLast60 = lockTimes.getAggregateValue(sixtySecondsAgo).getValue(); + final Long serializeNanosLast60 = serializeTimes.getAggregateValue(sixtySecondsAgo).getValue(); + final Long bytesWrittenLast60 = bytesWritten.getAggregateValue(sixtySecondsAgo).getValue(); + logger.debug("In the last 60 seconds, have spent {} millis writing to file ({} MB), {} millis waiting on synchronize block, {} millis serializing events", + TimeUnit.NANOSECONDS.toMillis(writeNanosLast60), + bytesWrittenLast60 / 1024 / 1024, + TimeUnit.NANOSECONDS.toMillis(lockNanosLast60), + TimeUnit.NANOSECONDS.toMillis(serializeNanosLast60)); + } + } + + final long serializedLength = endBytes - startBytes; + final TocWriter tocWriter = getTocWriter(); + final Integer blockIndex = tocWriter == null ? null : tocWriter.getCurrentBlockIndex(); + final File file = getFile(); + final String storageLocation = file.getParentFile().getName() + "/" + file.getName(); + return new StorageSummary(recordIdentifier, storageLocation, blockIndex, serializedLength, endBytes); + } + + @Override + public int getRecordsWritten() { + return recordCount.get(); + } + + protected Record createRecord(final ProvenanceEventRecord event, final long eventId) { + return new LookupTableEventRecord(event, eventId, eventSchema, contentClaimSchema, previousContentClaimSchema, firstEventId, systemTimeOffset, + componentIdMap, componentTypeMap, queueIdMap, eventTypeMap); + } + + @Override + protected void writeRecord(final ProvenanceEventRecord event, final long eventId, final DataOutputStream out) throws IOException { + final Record eventRecord = createRecord(event, eventId); + schemaRecordWriter.writeRecord(eventRecord, out); + } + + @Override + protected synchronized void writeHeader(final long firstEventId, final DataOutputStream out) throws IOException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + eventSchema.writeTo(baos); + + out.writeInt(baos.size()); + baos.writeTo(out); + + baos.reset(); + headerSchema.writeTo(baos); + out.writeInt(baos.size()); + baos.writeTo(out); + + this.firstEventId = firstEventId; + this.systemTimeOffset = System.currentTimeMillis(); + + final Map headerValues = new HashMap<>(); + headerValues.put(EventIdFirstHeaderSchema.FieldNames.FIRST_EVENT_ID, firstEventId); + headerValues.put(EventIdFirstHeaderSchema.FieldNames.TIMESTAMP_OFFSET, systemTimeOffset); + headerValues.put(EventIdFirstHeaderSchema.FieldNames.COMPONENT_IDS, idLookup.getComponentIdentifiers()); + headerValues.put(EventIdFirstHeaderSchema.FieldNames.COMPONENT_TYPES, idLookup.getComponentTypes()); + headerValues.put(EventIdFirstHeaderSchema.FieldNames.QUEUE_IDS, idLookup.getQueueIdentifiers()); + headerValues.put(EventIdFirstHeaderSchema.FieldNames.EVENT_TYPES, eventTypeNames); + final FieldMapRecord headerInfo = new FieldMapRecord(headerSchema, headerValues); + + schemaRecordWriter.writeRecord(headerInfo, out); + } + + @Override + protected int getSerializationVersion() { + return SERIALIZATION_VERSION; + } + + @Override + protected String getSerializationName() { + return SERIALIZATION_NAME; + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java index af7bff5421..a28d15018f 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/IndexConfiguration.java @@ -63,7 +63,7 @@ public class IndexConfiguration { private Map> recoverIndexDirectories() { final Map> indexDirectoryMap = new HashMap<>(); - for (final File storageDirectory : repoConfig.getStorageDirectories()) { + for (final File storageDirectory : repoConfig.getStorageDirectories().values()) { final List indexDirectories = new ArrayList<>(); final File[] matching = storageDirectory.listFiles(new FileFilter() { @Override @@ -85,6 +85,10 @@ public class IndexConfiguration { } private Long getFirstEntryTime(final File provenanceLogFile) { + if (provenanceLogFile == null) { + return null; + } + try (final RecordReader reader = RecordReaders.newRecordReader(provenanceLogFile, null, Integer.MAX_VALUE)) { final StandardProvenanceEventRecord firstRecord = reader.nextRecord(); if (firstRecord == null) { @@ -121,10 +125,14 @@ public class IndexConfiguration { } } + public File getWritableIndexDirectory(final File provenanceLogFile, final long newIndexTimestamp) { + return getWritableIndexDirectoryForStorageDirectory(provenanceLogFile.getParentFile(), provenanceLogFile, newIndexTimestamp); + } + + public File getWritableIndexDirectoryForStorageDirectory(final File storageDirectory, final File provenanceLogFile, final long newIndexTimestamp) { lock.lock(); try { - final File storageDirectory = provenanceLogFile.getParentFile(); List indexDirectories = this.indexDirectoryMap.get(storageDirectory); if (indexDirectories == null) { final File newDir = addNewIndex(storageDirectory, provenanceLogFile, newIndexTimestamp); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java index 3037e66fcf..ed183f9c5b 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java @@ -16,64 +16,6 @@ */ package org.apache.nifi.provenance; -import org.apache.lucene.document.Document; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexNotFoundException; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.store.FSDirectory; -import org.apache.nifi.authorization.AccessDeniedException; -import org.apache.nifi.authorization.AuthorizationResult; -import org.apache.nifi.authorization.AuthorizationResult.Result; -import org.apache.nifi.authorization.Authorizer; -import org.apache.nifi.authorization.RequestAction; -import org.apache.nifi.authorization.resource.Authorizable; -import org.apache.nifi.authorization.user.NiFiUser; -import org.apache.nifi.events.EventReporter; -import org.apache.nifi.processor.DataUnit; -import org.apache.nifi.provenance.expiration.ExpirationAction; -import org.apache.nifi.provenance.expiration.FileRemovalAction; -import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; -import org.apache.nifi.provenance.lineage.FlowFileLineage; -import org.apache.nifi.provenance.lineage.Lineage; -import org.apache.nifi.provenance.lineage.LineageComputationType; -import org.apache.nifi.provenance.lucene.DeleteIndexAction; -import org.apache.nifi.provenance.lucene.FieldNames; -import org.apache.nifi.provenance.lucene.IndexManager; -import org.apache.nifi.provenance.lucene.IndexSearch; -import org.apache.nifi.provenance.lucene.IndexingAction; -import org.apache.nifi.provenance.lucene.LineageQuery; -import org.apache.nifi.provenance.lucene.LuceneUtil; -import org.apache.nifi.provenance.lucene.SimpleIndexManager; -import org.apache.nifi.provenance.lucene.UpdateMinimumEventId; -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.SearchableField; -import org.apache.nifi.provenance.serialization.RecordReader; -import org.apache.nifi.provenance.serialization.RecordReaders; -import org.apache.nifi.provenance.serialization.RecordWriter; -import org.apache.nifi.provenance.serialization.RecordWriters; -import org.apache.nifi.provenance.toc.TocReader; -import org.apache.nifi.provenance.toc.TocUtil; -import org.apache.nifi.reporting.Severity; -import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.NiFiProperties; -import org.apache.nifi.util.RingBuffer; -import org.apache.nifi.util.RingBuffer.ForEachEvaluator; -import org.apache.nifi.util.StopWatch; -import org.apache.nifi.util.Tuple; -import org.apache.nifi.util.timebuffer.CountSizeEntityAccess; -import org.apache.nifi.util.timebuffer.LongEntityAccess; -import org.apache.nifi.util.timebuffer.TimedBuffer; -import org.apache.nifi.util.timebuffer.TimedCountSize; -import org.apache.nifi.util.timebuffer.TimestampedLong; -import org.apache.nifi.web.ResourceNotFoundException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.EOFException; import java.io.File; import java.io.FileFilter; @@ -82,7 +24,6 @@ import java.io.FilenameFilter; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -119,6 +60,69 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.regex.Pattern; import java.util.stream.Collectors; +import org.apache.lucene.document.Document; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexNotFoundException; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.store.FSDirectory; +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.AuthorizationResult.Result; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.expiration.ExpirationAction; +import org.apache.nifi.provenance.expiration.FileRemovalAction; +import org.apache.nifi.provenance.index.EventIndexWriter; +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; +import org.apache.nifi.provenance.lineage.FlowFileLineage; +import org.apache.nifi.provenance.lineage.Lineage; +import org.apache.nifi.provenance.lineage.LineageComputationType; +import org.apache.nifi.provenance.lucene.DeleteIndexAction; +import org.apache.nifi.provenance.lucene.DocsReader; +import org.apache.nifi.provenance.lucene.DocumentToEventConverter; +import org.apache.nifi.provenance.lucene.FieldNames; +import org.apache.nifi.provenance.lucene.IndexManager; +import org.apache.nifi.provenance.lucene.IndexSearch; +import org.apache.nifi.provenance.lucene.IndexingAction; +import org.apache.nifi.provenance.lucene.LineageQuery; +import org.apache.nifi.provenance.lucene.LuceneUtil; +import org.apache.nifi.provenance.lucene.SimpleIndexManager; +import org.apache.nifi.provenance.lucene.UpdateMinimumEventId; +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.SearchableField; +import org.apache.nifi.provenance.serialization.RecordReader; +import org.apache.nifi.provenance.serialization.RecordReaders; +import org.apache.nifi.provenance.serialization.RecordWriter; +import org.apache.nifi.provenance.serialization.RecordWriters; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.provenance.toc.TocReader; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.util.NamedThreadFactory; +import org.apache.nifi.reporting.Severity; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.util.RingBuffer; +import org.apache.nifi.util.RingBuffer.ForEachEvaluator; +import org.apache.nifi.util.StopWatch; +import org.apache.nifi.util.Tuple; +import org.apache.nifi.util.timebuffer.CountSizeEntityAccess; +import org.apache.nifi.util.timebuffer.LongEntityAccess; +import org.apache.nifi.util.timebuffer.TimedBuffer; +import org.apache.nifi.util.timebuffer.TimedCountSize; +import org.apache.nifi.util.timebuffer.TimestampedLong; +import org.apache.nifi.web.ResourceNotFoundException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + public class PersistentProvenanceRepository implements ProvenanceRepository { public static final String EVENT_CATEGORY = "Provenance Repository"; @@ -209,7 +213,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { } public PersistentProvenanceRepository(final NiFiProperties nifiProperties) throws IOException { - this(createRepositoryConfiguration(nifiProperties), 10000); + this(RepositoryConfiguration.create(nifiProperties), 10000); } public PersistentProvenanceRepository(final RepositoryConfiguration configuration, final int rolloverCheckMillis) throws IOException { @@ -220,7 +224,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { this.configuration = configuration; this.maxAttributeChars = configuration.getMaxAttributeChars(); - for (final File file : configuration.getStorageDirectories()) { + for (final File file : configuration.getStorageDirectories().values()) { final Path storageDirectory = file.toPath(); final Path journalDirectory = storageDirectory.resolve("journals"); @@ -234,7 +238,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { this.maxPartitionMillis = configuration.getMaxEventFileLife(TimeUnit.MILLISECONDS); this.maxPartitionBytes = configuration.getMaxEventFileCapacity(); this.indexConfig = new IndexConfiguration(configuration); - this.indexManager = new SimpleIndexManager(); + this.indexManager = new SimpleIndexManager(configuration); this.alwaysSync = configuration.isAlwaysSync(); this.rolloverCheckMillis = rolloverCheckMillis; @@ -254,7 +258,8 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { } @Override - public void initialize(final EventReporter eventReporter, final Authorizer authorizer, final ProvenanceAuthorizableFactory resourceFactory) throws IOException { + public void initialize(final EventReporter eventReporter, final Authorizer authorizer, final ProvenanceAuthorizableFactory resourceFactory, + final IdentifierLookup idLookup) throws IOException { writeLock.lock(); try { if (initialized.getAndSet(true)) { @@ -325,82 +330,10 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { } } - private static RepositoryConfiguration createRepositoryConfiguration(final NiFiProperties nifiProperties) throws IOException { - final Map storageDirectories = nifiProperties.getProvenanceRepositoryPaths(); - if (storageDirectories.isEmpty()) { - storageDirectories.put("provenance_repository", Paths.get("provenance_repository")); - } - final String storageTime = nifiProperties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_TIME, "24 hours"); - final String storageSize = nifiProperties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_SIZE, "1 GB"); - final String rolloverTime = nifiProperties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_TIME, "5 mins"); - final String rolloverSize = nifiProperties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_SIZE, "100 MB"); - final String shardSize = nifiProperties.getProperty(NiFiProperties.PROVENANCE_INDEX_SHARD_SIZE, "500 MB"); - final int queryThreads = nifiProperties.getIntegerProperty(NiFiProperties.PROVENANCE_QUERY_THREAD_POOL_SIZE, 2); - final int indexThreads = nifiProperties.getIntegerProperty(NiFiProperties.PROVENANCE_INDEX_THREAD_POOL_SIZE, 1); - final int journalCount = nifiProperties.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(nifiProperties.getProperty(NiFiProperties.PROVENANCE_COMPRESS_ON_ROLLOVER)); - final String indexedFieldString = nifiProperties.getProperty(NiFiProperties.PROVENANCE_INDEXED_FIELDS); - final String indexedAttrString = nifiProperties.getProperty(NiFiProperties.PROVENANCE_INDEXED_ATTRIBUTES); - - final Boolean alwaysSync = Boolean.parseBoolean(nifiProperties.getProperty("nifi.provenance.repository.always.sync", "false")); - - final int defaultMaxAttrChars = 65536; - final String maxAttrLength = nifiProperties.getProperty("nifi.provenance.repository.max.attribute.length", String.valueOf(defaultMaxAttrChars)); - int maxAttrChars; - try { - maxAttrChars = Integer.parseInt(maxAttrLength); - // must be at least 36 characters because that's the length of the uuid attribute, - // which must be kept intact - if (maxAttrChars < 36) { - maxAttrChars = 36; - logger.warn("Found max attribute length property set to " + maxAttrLength + " but minimum length is 36; using 36 instead"); - } - } catch (final Exception e) { - maxAttrChars = defaultMaxAttrChars; - } - - final List searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true); - final List searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false); - - // We always want to index the Event Time. - if (!searchableFields.contains(SearchableFields.EventTime)) { - searchableFields.add(SearchableFields.EventTime); - } - - final RepositoryConfiguration config = new RepositoryConfiguration(); - for (final Path path : storageDirectories.values()) { - config.addStorageDirectory(path.toFile()); - } - config.setCompressOnRollover(compressOnRollover); - config.setSearchableFields(searchableFields); - config.setSearchableAttributes(searchableAttributes); - config.setMaxEventFileCapacity(rolloverBytes); - config.setMaxEventFileLife(rolloverMillis, TimeUnit.MILLISECONDS); - config.setMaxRecordLife(storageMillis, TimeUnit.MILLISECONDS); - config.setMaxStorageCapacity(maxStorageBytes); - config.setQueryThreadPoolSize(queryThreads); - config.setIndexThreadPoolSize(indexThreads); - config.setJournalCount(journalCount); - config.setMaxAttributeChars(maxAttrChars); - - if (shardSize != null) { - config.setDesiredIndexSize(DataUnit.parseDataSize(shardSize, DataUnit.B).longValue()); - } - - config.setAlwaysSync(alwaysSync); - - return config; - } // protected in order to override for unit tests protected RecordWriter[] createWriters(final RepositoryConfiguration config, final long initialRecordId) throws IOException { - final List storageDirectories = config.getStorageDirectories(); + final List storageDirectories = new ArrayList<>(config.getStorageDirectories().values()); final RecordWriter[] writers = new RecordWriter[config.getJournalCount()]; for (int i = 0; i < config.getJournalCount(); i++) { @@ -408,7 +341,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { final File journalDirectory = new File(storageDirectory, "journals"); final File journalFile = new File(journalDirectory, String.valueOf(initialRecordId) + ".journal." + i); - writers[i] = RecordWriters.newSchemaRecordWriter(journalFile, false, false); + writers[i] = RecordWriters.newSchemaRecordWriter(journalFile, idGenerator, false, false); writers[i].writeHeader(initialRecordId); } @@ -460,7 +393,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { return Result.Approved.equals(result.getResult()); } - protected void authorize(final ProvenanceEventRecord event, final NiFiUser user) { + public void authorize(final ProvenanceEventRecord event, final NiFiUser user) { if (authorizer == null) { return; } @@ -474,11 +407,11 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { eventAuthorizable.authorize(authorizer, RequestAction.READ, user, event.getAttributes()); } - private List filterUnauthorizedEvents(final List events, final NiFiUser user) { + public List filterUnauthorizedEvents(final List events, final NiFiUser user) { return events.stream().filter(event -> isAuthorized(event, user)).collect(Collectors.toList()); } - private Set replaceUnauthorizedWithPlaceholders(final Set events, final NiFiUser user) { + public Set replaceUnauthorizedWithPlaceholders(final Set events, final NiFiUser user) { return events.stream().map(event -> isAuthorized(event, user) ? event : new PlaceholderProvenanceEvent(event)).collect(Collectors.toSet()); } @@ -594,7 +527,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { long minIndexedId = Long.MAX_VALUE; final List filesToRecover = new ArrayList<>(); - for (final File file : configuration.getStorageDirectories()) { + for (final File file : configuration.getStorageDirectories().values()) { final File[] matchingFiles = file.listFiles(new FileFilter() { @Override public boolean accept(final File pathname) { @@ -780,10 +713,10 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { try { long recordsWritten = 0L; for (final ProvenanceEventRecord nextRecord : records) { - final long eventId = idGenerator.getAndIncrement(); - bytesWritten += writer.writeRecord(nextRecord, eventId); + final StorageSummary persistedEvent = writer.writeRecord(nextRecord); + bytesWritten += persistedEvent.getSerializedLength(); recordsWritten++; - logger.trace("Wrote record with ID {} to {}", eventId, writer); + logger.trace("Wrote record with ID {} to {}", persistedEvent.getEventId(), writer); } writer.flush(); @@ -1175,7 +1108,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { */ private List getAllIndexDirectories() { final List allIndexDirs = new ArrayList<>(); - for (final File storageDir : configuration.getStorageDirectories()) { + for (final File storageDir : configuration.getStorageDirectories().values()) { final File[] indexDirs = storageDir.listFiles(new FilenameFilter() { @Override public boolean accept(final File dir, final String name) { @@ -1237,7 +1170,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { protected int getJournalCount() { // determine how many 'journals' we have in the journals directories int journalFileCount = 0; - for (final File storageDir : configuration.getStorageDirectories()) { + for (final File storageDir : configuration.getStorageDirectories().values()) { final File journalsDir = new File(storageDir, "journals"); final File[] journalFiles = journalsDir.listFiles(); if (journalFiles != null) { @@ -1313,7 +1246,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { // Choose a storage directory to store the merged file in. final long storageDirIdx = storageDirectoryIndex.getAndIncrement(); - final List storageDirs = configuration.getStorageDirectories(); + final List storageDirs = new ArrayList<>(configuration.getStorageDirectories().values()); final File storageDir = storageDirs.get((int) (storageDirIdx % storageDirs.size())); Future future = null; @@ -1479,8 +1412,8 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { final Map> journalMap = new HashMap<>(); // Map journals' basenames to the files with that basename. - final List storageDirs = configuration.getStorageDirectories(); - for (final File storageDir : configuration.getStorageDirectories()) { + final List storageDirs = new ArrayList<>(configuration.getStorageDirectories().values()); + for (final File storageDir : storageDirs) { final File journalDir = new File(storageDir, "journals"); if (!journalDir.exists()) { continue; @@ -1729,7 +1662,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { // loop over each entry in the map, persisting the records to the merged file in order, and populating the map // with the next entry from the journal file from which the previous record was written. - try (final RecordWriter writer = RecordWriters.newSchemaRecordWriter(writerFile, configuration.isCompressOnRollover(), true)) { + try (final RecordWriter writer = RecordWriters.newSchemaRecordWriter(writerFile, idGenerator, configuration.isCompressOnRollover(), true)) { writer.writeHeader(minEventId); final IndexingAction indexingAction = createIndexingAction(); @@ -1741,7 +1674,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { final AtomicBoolean finishedAdding = new AtomicBoolean(false); final List> futures = new ArrayList<>(); - final IndexWriter indexWriter = getIndexManager().borrowIndexWriter(indexingDirectory); + final EventIndexWriter indexWriter = getIndexManager().borrowIndexWriter(indexingDirectory); try { final ExecutorService exec = Executors.newFixedThreadPool(configuration.getIndexThreadPoolSize(), new ThreadFactory() { @Override @@ -1772,7 +1705,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { continue; } - indexingAction.index(tuple.getKey(), indexWriter, tuple.getValue()); + indexingAction.index(tuple.getKey(), indexWriter.getIndexWriter(), tuple.getValue()); } catch (final Throwable t) { logger.error("Failed to index Provenance Event for " + writerFile + " to " + indexingDirectory, t); if (indexingFailureCount.incrementAndGet() >= MAX_INDEXING_FAILURE_COUNT) { @@ -1795,7 +1728,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { final StandardProvenanceEventRecord record = entry.getKey(); final RecordReader reader = entry.getValue(); - writer.writeRecord(record, record.getEventId()); + writer.writeRecord(record); final int blockIndex = writer.getTocWriter().getCurrentBlockIndex(); boolean accepted = false; @@ -1879,7 +1812,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { } } } finally { - getIndexManager().returnIndexWriter(indexingDirectory, indexWriter); + getIndexManager().returnIndexWriter(indexWriter); } indexConfig.setMaxIdIndexed(maxId); @@ -1945,7 +1878,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { * events indexed, etc. */ protected IndexingAction createIndexingAction() { - return new IndexingAction(this); + return new IndexingAction(configuration.getSearchableFields(), configuration.getSearchableAttributes()); } private StandardProvenanceEventRecord truncateAttributes(final StandardProvenanceEventRecord original) { @@ -2322,7 +2255,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { if (event == null) { final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); - submission.getResult().update(Collections.emptyList()); + submission.getResult().update(Collections.emptyList(), 0L); return submission; } @@ -2359,9 +2292,9 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { try { final ProvenanceEventRecord event = getEvent(eventId); if (event == null) { - final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.emptyList(), 1, userId); + final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); - submission.getResult().update(Collections.emptyList()); + submission.getResult().update(Collections.emptyList(), 0L); return submission; } @@ -2642,11 +2575,21 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { } try { - final Set matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, - getIndexManager(), indexDir, null, flowFileUuids, maxAttributeChars); + final DocumentToEventConverter converter = new DocumentToEventConverter() { + @Override + public Set convert(TopDocs topDocs, IndexReader indexReader) throws IOException { + // Always authorized. We do this because we need to pull back the event, regardless of whether or not + // the user is truly authorized, because instead of ignoring unauthorized events, we want to replace them. + final EventAuthorizer authorizer = EventAuthorizer.GRANT_ALL; + final DocsReader docsReader = new DocsReader(); + return docsReader.read(topDocs, authorizer, indexReader, getAllLogFiles(), new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars); + } + }; + + final Set matchingRecords = LineageQuery.computeLineageForFlowFiles(getIndexManager(), indexDir, null, flowFileUuids, converter); final StandardLineageResult result = submission.getResult(); - result.update(replaceUnauthorizedWithPlaceholders(matchingRecords, user)); + result.update(replaceUnauthorizedWithPlaceholders(matchingRecords, user), matchingRecords.size()); logger.info("Successfully created Lineage for FlowFiles with UUIDs {} in {} milliseconds; Lineage contains {} nodes and {} edges", flowFileUuids, result.getComputationTime(TimeUnit.MILLISECONDS), result.getNodes().size(), result.getEdges().size()); @@ -2666,7 +2609,6 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { } private class RemoveExpiredQueryResults implements Runnable { - @Override public void run() { try { @@ -2697,22 +2639,4 @@ public class PersistentProvenanceRepository implements ProvenanceRepository { } } } - - private static class NamedThreadFactory implements ThreadFactory { - - private final AtomicInteger counter = new AtomicInteger(0); - private final ThreadFactory defaultThreadFactory = Executors.defaultThreadFactory(); - private final String namePrefix; - - public NamedThreadFactory(final String namePrefix) { - this.namePrefix = namePrefix; - } - - @Override - public Thread newThread(final Runnable r) { - final Thread thread = defaultThreadFactory.newThread(r); - thread.setName(namePrefix + "-" + counter.incrementAndGet()); - return thread; - } - } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java index e63133afd3..7a2f57e37a 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/RepositoryConfiguration.java @@ -17,20 +17,35 @@ package org.apache.nifi.provenance; import java.io.File; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; +import org.apache.nifi.processor.DataUnit; 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 RepositoryConfiguration { + private static final Logger logger = LoggerFactory.getLogger(RepositoryConfiguration.class); - private final List storageDirectories = new ArrayList<>(); + public static final String CONCURRENT_MERGE_THREADS = "nifi.provenance.repository.concurrent.merge.threads"; + public static final String WARM_CACHE_FREQUENCY = "nifi.provenance.repository.warm.cache.frequency"; + + private final Map storageDirectories = new LinkedHashMap<>(); private long recordLifeMillis = TimeUnit.MILLISECONDS.convert(24, TimeUnit.HOURS); private long storageCapacity = 1024L * 1024L * 1024L; // 1 GB private long eventFileMillis = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES); private long eventFileBytes = 1024L * 1024L * 5L; // 5 MB + private int maxFileEvents = Integer.MAX_VALUE; private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB private int journalCount = 16; private int compressionBlockBytes = 1024 * 1024; @@ -43,6 +58,8 @@ public class RepositoryConfiguration { private int queryThreadPoolSize = 2; private int indexThreadPoolSize = 1; private boolean allowRollover = true; + private int concurrentMergeThreads = 4; + private Integer warmCacheFrequencyMinutes = null; public void setAllowRollover(final boolean allow) { this.allowRollover = allow; @@ -52,7 +69,6 @@ public class RepositoryConfiguration { return allowRollover; } - public int getCompressionBlockBytes() { return compressionBlockBytes; } @@ -66,8 +82,8 @@ public class RepositoryConfiguration { * * @return the directories where provenance files will be stored */ - public List getStorageDirectories() { - return Collections.unmodifiableList(storageDirectories); + public Map getStorageDirectories() { + return Collections.unmodifiableMap(storageDirectories); } /** @@ -75,8 +91,12 @@ public class RepositoryConfiguration { * * @param storageDirectory the directory to store provenance files */ - public void addStorageDirectory(final File storageDirectory) { - this.storageDirectories.add(storageDirectory); + public void addStorageDirectory(final String partitionName, final File storageDirectory) { + this.storageDirectories.put(partitionName, storageDirectory); + } + + public void addStorageDirectories(final Map storageDirectories) { + this.storageDirectories.putAll(storageDirectories); } /** @@ -147,6 +167,20 @@ public class RepositoryConfiguration { this.eventFileBytes = maxEventFileBytes; } + /** + * @return the maximum number of events that should be written to a single event file before the file is rolled over + */ + public int getMaxEventFileCount() { + return maxFileEvents; + } + + /** + * @param maxCount the maximum number of events that should be written to a single event file before the file is rolled over + */ + public void setMaxEventFileCount(final int maxCount) { + this.maxFileEvents = maxCount; + } + /** * @return the fields that should be indexed */ @@ -218,6 +252,14 @@ public class RepositoryConfiguration { this.indexThreadPoolSize = indexThreadPoolSize; } + public void setConcurrentMergeThreads(final int mergeThreads) { + this.concurrentMergeThreads = mergeThreads; + } + + public int getConcurrentMergeThreads() { + return concurrentMergeThreads; + } + /** *

* Specifies the desired size of each Provenance Event index shard, in @@ -310,4 +352,90 @@ public class RepositoryConfiguration { this.maxAttributeChars = maxAttributeChars; } + public void setWarmCacheFrequencyMinutes(Integer frequencyMinutes) { + this.warmCacheFrequencyMinutes = frequencyMinutes; + } + + public Optional getWarmCacheFrequencyMinutes() { + return Optional.ofNullable(warmCacheFrequencyMinutes); + } + + public static RepositoryConfiguration create(final NiFiProperties nifiProperties) { + final Map storageDirectories = nifiProperties.getProvenanceRepositoryPaths(); + if (storageDirectories.isEmpty()) { + storageDirectories.put("provenance_repository", Paths.get("provenance_repository")); + } + final String storageTime = nifiProperties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_TIME, "24 hours"); + final String storageSize = nifiProperties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_SIZE, "1 GB"); + final String rolloverTime = nifiProperties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_TIME, "5 mins"); + final String rolloverSize = nifiProperties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_SIZE, "100 MB"); + final String shardSize = nifiProperties.getProperty(NiFiProperties.PROVENANCE_INDEX_SHARD_SIZE, "500 MB"); + final int queryThreads = nifiProperties.getIntegerProperty(NiFiProperties.PROVENANCE_QUERY_THREAD_POOL_SIZE, 2); + final int indexThreads = nifiProperties.getIntegerProperty(NiFiProperties.PROVENANCE_INDEX_THREAD_POOL_SIZE, 2); + final int journalCount = nifiProperties.getIntegerProperty(NiFiProperties.PROVENANCE_JOURNAL_COUNT, 16); + final int concurrentMergeThreads = nifiProperties.getIntegerProperty(CONCURRENT_MERGE_THREADS, 2); + final String warmCacheFrequency = nifiProperties.getProperty(WARM_CACHE_FREQUENCY); + + 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(nifiProperties.getProperty(NiFiProperties.PROVENANCE_COMPRESS_ON_ROLLOVER)); + final String indexedFieldString = nifiProperties.getProperty(NiFiProperties.PROVENANCE_INDEXED_FIELDS); + final String indexedAttrString = nifiProperties.getProperty(NiFiProperties.PROVENANCE_INDEXED_ATTRIBUTES); + + final Boolean alwaysSync = Boolean.parseBoolean(nifiProperties.getProperty("nifi.provenance.repository.always.sync", "false")); + + final int defaultMaxAttrChars = 65536; + final String maxAttrLength = nifiProperties.getProperty("nifi.provenance.repository.max.attribute.length", String.valueOf(defaultMaxAttrChars)); + int maxAttrChars; + try { + maxAttrChars = Integer.parseInt(maxAttrLength); + // must be at least 36 characters because that's the length of the uuid attribute, + // which must be kept intact + if (maxAttrChars < 36) { + maxAttrChars = 36; + logger.warn("Found max attribute length property set to " + maxAttrLength + " but minimum length is 36; using 36 instead"); + } + } catch (final Exception e) { + maxAttrChars = defaultMaxAttrChars; + } + + final List searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true); + final List searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false); + + // We always want to index the Event Time. + if (!searchableFields.contains(SearchableFields.EventTime)) { + searchableFields.add(SearchableFields.EventTime); + } + + final RepositoryConfiguration config = new RepositoryConfiguration(); + for (final Map.Entry entry : storageDirectories.entrySet()) { + config.addStorageDirectory(entry.getKey(), entry.getValue().toFile()); + } + config.setCompressOnRollover(compressOnRollover); + config.setSearchableFields(searchableFields); + config.setSearchableAttributes(searchableAttributes); + config.setMaxEventFileCapacity(rolloverBytes); + config.setMaxEventFileLife(rolloverMillis, TimeUnit.MILLISECONDS); + config.setMaxRecordLife(storageMillis, TimeUnit.MILLISECONDS); + config.setMaxStorageCapacity(maxStorageBytes); + config.setQueryThreadPoolSize(queryThreads); + config.setIndexThreadPoolSize(indexThreads); + config.setJournalCount(journalCount); + config.setMaxAttributeChars(maxAttrChars); + config.setConcurrentMergeThreads(concurrentMergeThreads); + + if (warmCacheFrequency != null && !warmCacheFrequency.trim().equals("")) { + config.setWarmCacheFrequencyMinutes((int) FormatUtils.getTimeDuration(warmCacheFrequency, TimeUnit.MINUTES)); + } + if (shardSize != null) { + config.setDesiredIndexSize(DataUnit.parseDataSize(shardSize, DataUnit.B).longValue()); + } + + config.setAlwaysSync(alwaysSync); + + return config; + } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java index f018685e4c..ce875d6196 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordReader.java @@ -32,6 +32,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class StandardRecordReader extends CompressableRecordReader { + public static final int SERIALIZATION_VERISON = 9; + public static final String SERIALIZATION_NAME = "org.apache.nifi.provenance.PersistentProvenanceRepository"; + private static final Logger logger = LoggerFactory.getLogger(StandardRecordReader.class); private static final Pattern UUID_PATTERN = Pattern.compile("[a-fA-F0-9]{8}\\-([a-fA-F0-9]{4}\\-){3}[a-fA-F0-9]{12}"); @@ -121,9 +124,9 @@ public class StandardRecordReader extends CompressableRecordReader { @Override public StandardProvenanceEventRecord nextRecord(final DataInputStream dis, final int serializationVersion) throws IOException { - if (serializationVersion > StandardRecordWriter.SERIALIZATION_VERISON) { + if (serializationVersion > SERIALIZATION_VERISON) { throw new IllegalArgumentException("Unable to deserialize record because the version is " - + serializationVersion + " and supported versions are 1-" + StandardRecordWriter.SERIALIZATION_VERISON); + + serializationVersion + " and supported versions are 1-" + SERIALIZATION_VERISON); } // Schema changed drastically in version 6 so we created a new method to handle old records diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java index 46967672c9..0a749adb2e 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/StandardRecordWriter.java @@ -16,17 +16,18 @@ */ package org.apache.nifi.provenance; +import java.io.DataOutputStream; import java.io.File; import java.io.IOException; import java.io.OutputStream; import java.io.UTFDataFormatException; import java.util.Collection; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import org.apache.nifi.provenance.serialization.CompressableRecordWriter; import org.apache.nifi.provenance.serialization.RecordWriter; import org.apache.nifi.provenance.toc.TocWriter; -import org.apache.nifi.stream.io.DataOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,15 +46,16 @@ public class StandardRecordWriter extends CompressableRecordWriter implements Re private final File file; - public StandardRecordWriter(final File file, final TocWriter writer, final boolean compressed, final int uncompressedBlockSize) throws IOException { - super(file, writer, compressed, uncompressedBlockSize); + public StandardRecordWriter(final File file, final AtomicLong idGenerator, final TocWriter writer, final boolean compressed, final int uncompressedBlockSize) throws IOException { + super(file, idGenerator, writer, compressed, uncompressedBlockSize); logger.trace("Creating Record Writer for {}", file.getName()); this.file = file; } - public StandardRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { - super(out, tocWriter, compressed, uncompressedBlockSize); + public StandardRecordWriter(final OutputStream out, final String storageLocation, final AtomicLong idGenerator, final TocWriter tocWriter, + final boolean compressed, final int uncompressedBlockSize) throws IOException { + super(out, storageLocation, idGenerator, tocWriter, compressed, uncompressedBlockSize); this.file = null; } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java new file mode 100644 index 0000000000..229a96d5d9 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java @@ -0,0 +1,280 @@ +/* + * 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.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.UserEventAuthorizer; +import org.apache.nifi.provenance.index.EventIndex; +import org.apache.nifi.provenance.index.lucene.LuceneEventIndex; +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; +import org.apache.nifi.provenance.lucene.IndexManager; +import org.apache.nifi.provenance.lucene.SimpleIndexManager; +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.provenance.serialization.RecordReaders; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.provenance.store.EventFileManager; +import org.apache.nifi.provenance.store.EventStore; +import org.apache.nifi.provenance.store.PartitionedWriteAheadEventStore; +import org.apache.nifi.provenance.store.RecordReaderFactory; +import org.apache.nifi.provenance.store.RecordWriterFactory; +import org.apache.nifi.provenance.store.StorageResult; +import org.apache.nifi.provenance.toc.StandardTocWriter; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.toc.TocWriter; +import org.apache.nifi.provenance.util.CloseableUtil; +import org.apache.nifi.reporting.Severity; +import org.apache.nifi.util.NiFiProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + *

+ * A Provenance Repository that is made up of two distinct concepts: An {@link EventStore Event Store} that is responsible + * for storing and accessing the events (this repository makes use of an Event Store that uses a backing Write-Ahead Log, hence the name + * WriteAheadProvenanceRepository) and an {@link EventIndex Event Index} that is responsible for indexing and searching those + * events. + *

+ * + *

+ * When a Provenance Event is added to the repository, it is first stored in the Event Store. The Event Store reports the location (namely, the + * Event Identifier) that it used to store the event. The stored event is then given to the Event Index along with its storage location. The index + * is then responsible for indexing the event in real-time. Once this has completed, the method returns. + *

+ * + *

+ * The Event Index that is used by this implementation currently is the {@link LuceneEventIndex}, which is powered by Apache Lucene. This index provides + * very high throughput. However, this high throughput is gained by avoiding continual 'commits' of the Index Writer. As a result, on restart, this Repository + * may take a minute or two to re-index some of the Provenance Events, as some of the Events may have been added to the index without committing the Index Writer. + * Given the substantial performance improvement gained by committing the Index Writer only periodically, this trade-off is generally well accepted. + *

+ * + *

+ * This Repositories supports the notion of 'partitions'. The repository can be configured to store data to one or more partitions. Each partition is typically + * stored on a separate physical partition on disk. As a result, this allows striping of data across multiple partitions in order to achieve linear scalability + * across disks for far greater performance. + *

+ */ +public class WriteAheadProvenanceRepository implements ProvenanceRepository { + private static final Logger logger = LoggerFactory.getLogger(WriteAheadProvenanceRepository.class); + private static final int BLOCK_SIZE = 1024 * 32; + public static final String EVENT_CATEGORY = "Provenance Repository"; + + private final RepositoryConfiguration config; + + // effectively final + private EventStore eventStore; + private EventIndex eventIndex; + private EventReporter eventReporter; + private Authorizer authorizer; + private ProvenanceAuthorizableFactory resourceFactory; + + /** + * This constructor exists solely for the use of the Java Service Loader mechanism and should not be used. + */ + public WriteAheadProvenanceRepository() { + config = null; + } + + public WriteAheadProvenanceRepository(final NiFiProperties nifiProperties) { + this(RepositoryConfiguration.create(nifiProperties)); + } + + public WriteAheadProvenanceRepository(final RepositoryConfiguration config) { + this.config = config; + } + + @Override + public synchronized void initialize(final EventReporter eventReporter, final Authorizer authorizer, final ProvenanceAuthorizableFactory resourceFactory, + final IdentifierLookup idLookup) throws IOException { + final RecordWriterFactory recordWriterFactory = (file, idGenerator, compressed, createToc) -> { + final TocWriter tocWriter = createToc ? new StandardTocWriter(TocUtil.getTocFile(file), false, false) : null; + return new EventIdFirstSchemaRecordWriter(file, idGenerator, tocWriter, compressed, BLOCK_SIZE, idLookup); + }; + + final EventFileManager fileManager = new EventFileManager(); + final RecordReaderFactory recordReaderFactory = (file, logs, maxChars) -> { + fileManager.obtainReadLock(file); + try { + return RecordReaders.newRecordReader(file, logs, maxChars); + } finally { + fileManager.releaseReadLock(file); + } + }; + + eventStore = new PartitionedWriteAheadEventStore(config, recordWriterFactory, recordReaderFactory, eventReporter, fileManager); + + final IndexManager indexManager = new SimpleIndexManager(config); + eventIndex = new LuceneEventIndex(config, indexManager, eventReporter); + + this.eventReporter = eventReporter; + this.authorizer = authorizer; + this.resourceFactory = resourceFactory; + + eventStore.initialize(); + eventIndex.initialize(eventStore); + + eventStore.reindexLatestEvents(eventIndex); + } + + @Override + public ProvenanceEventBuilder eventBuilder() { + return new StandardProvenanceEventRecord.Builder(); + } + + @Override + public void registerEvent(final ProvenanceEventRecord event) { + registerEvents(Collections.singleton(event)); + } + + @Override + public void registerEvents(final Iterable events) { + final StorageResult storageResult; + + try { + storageResult = eventStore.addEvents(events); + } catch (final IOException e) { + logger.error("Failed to write events to the Event Store", e); + eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to write Provenance Events to the repository. See logs for more details."); + return; + } + + final Map locationMap = storageResult.getStorageLocations(); + if (!locationMap.isEmpty()) { + eventIndex.addEvents(locationMap); + } + } + + @Override + public List getEvents(final long firstRecordId, final int maxRecords) throws IOException { + return eventStore.getEvents(firstRecordId, maxRecords); + } + + @Override + public ProvenanceEventRecord getEvent(final long id) throws IOException { + return eventStore.getEvent(id).orElse(null); + } + + @Override + public Long getMaxEventId() { + return eventStore.getMaxEventId(); + } + + @Override + public void close() { + CloseableUtil.closeQuietly(eventStore, eventIndex); + } + + @Override + public ProvenanceEventRecord getEvent(final long id, final NiFiUser user) throws IOException { + final ProvenanceEventRecord event = getEvent(id); + if (event == null) { + return null; + } + + authorize(event, user); + return event; + } + + private void authorize(final ProvenanceEventRecord event, final NiFiUser user) { + if (authorizer == null) { + return; + } + + final Authorizable eventAuthorizable; + if (event.isRemotePortType()) { + eventAuthorizable = resourceFactory.createRemoteDataAuthorizable(event.getComponentId()); + } else { + eventAuthorizable = resourceFactory.createLocalDataAuthorizable(event.getComponentId()); + } + eventAuthorizable.authorize(authorizer, RequestAction.READ, user, event.getAttributes()); + } + + + @Override + public List getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) throws IOException { + final List events = getEvents(firstRecordId, maxRecords); + return createEventAuthorizer(user).filterUnauthorizedEvents(events); + } + + private EventAuthorizer createEventAuthorizer(final NiFiUser user) { + return new UserEventAuthorizer(authorizer, resourceFactory, user); + } + + @Override + public ProvenanceEventRepository getProvenanceEventRepository() { + return this; + } + + @Override + public QuerySubmission submitQuery(final Query query, final NiFiUser user) { + return eventIndex.submitQuery(query, createEventAuthorizer(user), user.getIdentity()); + } + + @Override + public QuerySubmission retrieveQuerySubmission(final String queryIdentifier, final NiFiUser user) { + return eventIndex.retrieveQuerySubmission(queryIdentifier, user); + } + + @Override + public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid, final NiFiUser user) { + return eventIndex.submitLineageComputation(flowFileUuid, user, createEventAuthorizer(user)); + } + + @Override + public ComputeLineageSubmission submitLineageComputation(final long eventId, final NiFiUser user) { + return eventIndex.submitLineageComputation(eventId, user, createEventAuthorizer(user)); + } + + @Override + public ComputeLineageSubmission retrieveLineageSubmission(final String lineageIdentifier, final NiFiUser user) { + return eventIndex.retrieveLineageSubmission(lineageIdentifier, user); + } + + @Override + public ComputeLineageSubmission submitExpandParents(final long eventId, final NiFiUser user) { + return eventIndex.submitExpandParents(eventId, user, createEventAuthorizer(user)); + } + + @Override + public ComputeLineageSubmission submitExpandChildren(final long eventId, final NiFiUser user) { + return eventIndex.submitExpandChildren(eventId, user, createEventAuthorizer(user)); + } + + @Override + public List getSearchableFields() { + return Collections.unmodifiableList(config.getSearchableFields()); + } + + @Override + public List getSearchableAttributes() { + return Collections.unmodifiableList(config.getSearchableAttributes()); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/EventAuthorizer.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/EventAuthorizer.java new file mode 100644 index 0000000000..ab193e489c --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/EventAuthorizer.java @@ -0,0 +1,119 @@ +/* + * 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.authorization; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.provenance.PlaceholderProvenanceEvent; +import org.apache.nifi.provenance.ProvenanceEventRecord; + +public interface EventAuthorizer { + + /** + * Determines whether or not the has access to the given Provenance Event. + * This method does not imply the user is directly attempting to access the specified resource. If the user is + * attempting a direct access use Authorizable.authorize(). + * + * @param event the event to authorize + * @return is authorized + */ + boolean isAuthorized(ProvenanceEventRecord event); + + /** + * Authorizes the current user for the specified action on the specified resource. This method does + * imply the user is directly accessing the specified resource. + * + * @param event the event to authorize + * @throws AccessDeniedException if the user is not authorized + */ + void authorize(ProvenanceEventRecord event) throws AccessDeniedException; + + /** + * Filters out any events that the user is not authorized to access + * + * @param events the events to filtered + * @return a List that contains only events from the original, for which the user has access + */ + default List filterUnauthorizedEvents(List events) { + return events.stream() + .filter(event -> isAuthorized(event)) + .collect(Collectors.toList()); + } + + /** + * Returns a Set of provenance events for which any of the given events that the user does not + * have access to has been replaced by a placeholder event + * + * @param events the events to filter + * @return a Set containing only provenance events that the user has access to + */ + default Set replaceUnauthorizedWithPlaceholders(Set events) { + return events.stream() + .map(event -> isAuthorized(event) ? event : new PlaceholderProvenanceEvent(event)) + .collect(Collectors.toSet()); + } + + public static final EventAuthorizer GRANT_ALL = new EventAuthorizer() { + @Override + public boolean isAuthorized(ProvenanceEventRecord event) { + return true; + } + + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + } + + @Override + public List filterUnauthorizedEvents(List events) { + return events; + } + + @Override + public Set replaceUnauthorizedWithPlaceholders(Set events) { + return events; + } + }; + + public static final EventAuthorizer DENY_ALL = new EventAuthorizer() { + @Override + public boolean isAuthorized(ProvenanceEventRecord event) { + return false; + } + + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + throw new AccessDeniedException(); + } + + @Override + public List filterUnauthorizedEvents(List events) { + return Collections.emptyList(); + } + + @Override + public Set replaceUnauthorizedWithPlaceholders(Set events) { + return events.stream() + .map(event -> new PlaceholderProvenanceEvent(event)) + .collect(Collectors.toSet()); + } + }; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/EventTransformer.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/EventTransformer.java new file mode 100644 index 0000000000..1c48aad367 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/EventTransformer.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.authorization; + +import java.util.Optional; + +import org.apache.nifi.provenance.PlaceholderProvenanceEvent; +import org.apache.nifi.provenance.ProvenanceEventRecord; + +/** + * An interface for transforming a Provenance Event for which the user is not authorized to access + */ +public interface EventTransformer { + + Optional transform(ProvenanceEventRecord unauthorizedEvent); + + /** + * An EventTransformer that transforms any event into an Empty Optional + */ + public static final EventTransformer EMPTY_TRANSFORMER = event -> Optional.empty(); + + /** + * An EventTransformer that transforms any event into a Placeholder event + */ + public static final EventTransformer PLACEHOLDER_TRANSFORMER = event -> Optional.of(new PlaceholderProvenanceEvent(event)); + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/UserEventAuthorizer.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/UserEventAuthorizer.java new file mode 100644 index 0000000000..5126b7e678 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/authorization/UserEventAuthorizer.java @@ -0,0 +1,76 @@ +/* + * 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.authorization; + +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.AuthorizationResult.Result; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.provenance.ProvenanceAuthorizableFactory; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.web.ResourceNotFoundException; + +public class UserEventAuthorizer implements EventAuthorizer { + private final Authorizer authorizer; + private final ProvenanceAuthorizableFactory resourceFactory; + private final NiFiUser user; + + public UserEventAuthorizer(final Authorizer authorizer, final ProvenanceAuthorizableFactory authorizableFactory, final NiFiUser user) { + this.authorizer = authorizer; + this.resourceFactory = authorizableFactory; + this.user = user; + } + + @Override + public boolean isAuthorized(final ProvenanceEventRecord event) { + if (authorizer == null || user == null) { + return true; + } + + final Authorizable eventAuthorizable; + try { + if (event.isRemotePortType()) { + eventAuthorizable = resourceFactory.createRemoteDataAuthorizable(event.getComponentId()); + } else { + eventAuthorizable = resourceFactory.createLocalDataAuthorizable(event.getComponentId()); + } + } catch (final ResourceNotFoundException rnfe) { + return false; + } + + final AuthorizationResult result = eventAuthorizable.checkAuthorization(authorizer, RequestAction.READ, user, event.getAttributes()); + return Result.Approved.equals(result.getResult()); + } + + @Override + public void authorize(final ProvenanceEventRecord event) { + if (authorizer == null) { + return; + } + + final Authorizable eventAuthorizable; + if (event.isRemotePortType()) { + eventAuthorizable = resourceFactory.createRemoteDataAuthorizable(event.getComponentId()); + } else { + eventAuthorizable = resourceFactory.createLocalDataAuthorizable(event.getComponentId()); + } + eventAuthorizable.authorize(authorizer, RequestAction.READ, user, event.getAttributes()); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndex.java new file mode 100644 index 0000000000..051cd1f930 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndex.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.index; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Map; + +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +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.serialization.StorageSummary; +import org.apache.nifi.provenance.store.EventStore; + +/** + * An Event Index is responsible for indexing Provenance Events in such a way that the index can be quickly + * searched to in order to retrieve events of interest. + */ +public interface EventIndex extends Closeable { + + /** + * Initializes the Event Index, providing it access to the Event Store, in case it is necessary for performing + * initialization tasks + * + * @param eventStore the EventStore that holds the events that have been given to the repository. + */ + void initialize(EventStore eventStore); + + /** + * Adds the given events to the index so that they can be queried later. + * + * @param events the events to index along with their associated Storage Summaries + */ + void addEvents(Map events); + + /** + * Replaces the entries in the appropriate index with the given events + * + * @param events the events to add or replace along with their associated Storage Summaries + */ + void reindexEvents(Map events); + + /** + * @return the number of bytes that are utilized by the Event Index + */ + long getSize(); + + /** + * Submits a Query asynchronously and returns a QuerySubmission that can be used to obtain the results + * + * @param query the query to perform + * @param authorizer the authorizer to use in order to determine whether or not a particular event should be included in the result + * @param userId the ID of the user on whose behalf the query is being submitted + * + * @return a QuerySubmission that can be used to retrieve the results later + */ + QuerySubmission submitQuery(Query query, EventAuthorizer authorizer, String userId); + + /** + * Asynchronously computes the lineage for the FlowFile that is identified by the Provenance Event with the given ID. + * + * @param eventId the ID of the Provenance Event for which the lineage should be calculated + * @param user the NiFi user on whose behalf the computing is being performed + * @param authorizer the authorizer to use in order to determine whether or not a particular event should be included in the result + * + * @return a ComputeLineageSubmission that can be used to retrieve the results later + */ + ComputeLineageSubmission submitLineageComputation(long eventId, NiFiUser user, EventAuthorizer authorizer); + + /** + * Asynchronously computes the lineage for the FlowFile that has the given FlowFile UUID. + * + * @param flowFileUuid the UUID of the FlowFile for which the lineage should be computed + * @param user the NiFi user on whose behalf the computing is being performed + * @param authorizer the authorizer to use in order to determine whether or not a particular event should be included in the result + * + * @return a ComputeLineageSubmission that can be used to retrieve the results later + */ + ComputeLineageSubmission submitLineageComputation(String flowFileUuid, NiFiUser user, EventAuthorizer authorizer); + + /** + * Asynchronously computes the lineage that makes up the 'child flowfiles' generated by the event with the given ID. This method is + * valid only for Events that produce 'child flowfiles' such as FORK, CLONE, REPLAY, etc. + * + * @param eventId the ID of the Provenance Event for which the lineage should be calculated + * @param user the NiFi user on whose behalf the computing is being performed + * @param authorizer the authorizer to use in order to determine whether or not a particular event should be included in the result + * + * @return a ComputeLineageSubmission that can be used to retrieve the results later + */ + ComputeLineageSubmission submitExpandChildren(long eventId, NiFiUser user, EventAuthorizer authorizer); + + /** + * Asynchronously computes the lineage that makes up the 'parent flowfiles' that were involved in the event with the given ID. This method + * is valid only for Events that have 'parent flowfiles' such as FORK, JOIN, etc. + * + * @param eventId the ID of the Provenance Event for which the lineage should be calculated + * @param user the NiFi user on whose behalf the computing is being performed + * @param authorizer the authorizer to use in order to determine whether or not a particular event should be included in the result + * + * @return a ComputeLineageSubmission that can be used to retrieve the results later + */ + ComputeLineageSubmission submitExpandParents(long eventId, NiFiUser user, EventAuthorizer authorizer); + + /** + * Retrieves the ComputeLineageSubmission that was returned by the 'submitLineageComputation' methods + * + * @param lineageIdentifier the identifier of the linage + * @param user the NiFi user on whose behalf the retrieval is being performed + * @return the ComputeLineageSubmission that represents the asynchronous lineage computation that is being performed under the given + * identifier, or null if the identifier cannot be found. + */ + ComputeLineageSubmission retrieveLineageSubmission(String lineageIdentifier, NiFiUser user); + + /** + * Retrieves the QuerySubmission that was returned by the 'submitQuery' method + * + * @param queryIdentifier the identifier of the query + * @param user the NiFi user on whose behalf the retrieval is being performed + * @return the QuerySubmission that represents the asynchronous query that is being performed under the given + * identifier, or null if the identifier cannot be found. + */ + QuerySubmission retrieveQuerySubmission(String queryIdentifier, NiFiUser user); + + /** + * Upon restart of NiFi, it is possible that the Event Index will have lost some events due to frequency of committing the index. + * In such as case, this method is responsible for returning the minimum Provenance Event ID that it knows is safely indexed. If + * any Provenance Event exists in the Event Store with an ID greater than the value returned, that Event should be re-indexed. + * + * @param partitionName the name of the Partition for which the minimum Event ID is desired + * @return the minimum Provenance Event ID that the Index knows is safely indexed for the given partition + */ + long getMinimumEventIdToReindex(String partitionName); + + /** + * Instructs the Event Index to commit any changes that have been made to the partition with the given name + * + * @param partitionName the name of the partition to commit changes + * @throws IOException if unable to commit the changes + */ + void commitChanges(String partitionName) throws IOException; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndexSearcher.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndexSearcher.java new file mode 100644 index 0000000000..83894089da --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndexSearcher.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.index; + +import java.io.Closeable; +import java.io.File; + +import org.apache.lucene.search.IndexSearcher; + +public interface EventIndexSearcher extends Closeable { + IndexSearcher getIndexSearcher(); + + File getIndexDirectory(); +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndexWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndexWriter.java new file mode 100644 index 0000000000..f0af7dcc19 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/EventIndexWriter.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.index; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.List; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.IndexWriter; + +public interface EventIndexWriter extends Closeable { + + boolean index(Document document, int commitThreshold) throws IOException; + + boolean index(List documents, int commitThreshold) throws IOException; + + File getDirectory(); + + long commit() throws IOException; + + int getEventsIndexedSinceCommit(); + + long getEventsIndexed(); + + IndexWriter getIndexWriter(); + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/SearchFailedException.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/SearchFailedException.java new file mode 100644 index 0000000000..ce1bedb1cf --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/SearchFailedException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.index; + +public class SearchFailedException extends RuntimeException { + public SearchFailedException(final String message, final Throwable cause) { + super(message, cause); + } + + public SearchFailedException(final Throwable cause) { + super(cause); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/CachedQuery.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/CachedQuery.java new file mode 100644 index 0000000000..770c4552bf --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/CachedQuery.java @@ -0,0 +1,33 @@ +/* + * 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.index.lucene; + +import java.util.List; +import java.util.Optional; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.search.Query; +import org.apache.nifi.provenance.serialization.StorageSummary; + +public interface CachedQuery { + + void update(ProvenanceEventRecord event, StorageSummary storageSummary); + + Optional> evaluate(Query query); + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/CommitPreference.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/CommitPreference.java new file mode 100644 index 0000000000..2208917162 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/CommitPreference.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.index.lucene; + +public enum CommitPreference { + FORCE_COMMIT, + PREVENT_COMMIT, + NO_PREFERENCE; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/ConvertEventToLuceneDocument.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/ConvertEventToLuceneDocument.java new file mode 100644 index 0000000000..765b81f367 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/ConvertEventToLuceneDocument.java @@ -0,0 +1,143 @@ +/* + * 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.index.lucene; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.Field.Store; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.document.LongField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.lucene.LuceneUtil; +import org.apache.nifi.provenance.search.SearchableField; +import org.apache.nifi.provenance.serialization.StorageSummary; + +public class ConvertEventToLuceneDocument { + private final Set searchableEventFields; + private final Set searchableAttributeFields; + + public ConvertEventToLuceneDocument(final List searchableEventFields, final List searchableAttributes) { + this.searchableEventFields = Collections.unmodifiableSet(new HashSet<>(searchableEventFields)); + this.searchableAttributeFields = Collections.unmodifiableSet(new HashSet<>(searchableAttributes)); + } + + private void addField(final Document doc, final SearchableField field, final String value) { + if (value == null || (!field.isAttribute() && !searchableEventFields.contains(field))) { + return; + } + + doc.add(new StringField(field.getSearchableFieldName(), value.toLowerCase(), Store.NO)); + } + + + public Document convert(final ProvenanceEventRecord record, final StorageSummary persistedEvent) { + final Document doc = new Document(); + addField(doc, SearchableFields.FlowFileUUID, record.getFlowFileUuid()); + addField(doc, SearchableFields.Filename, record.getAttribute(CoreAttributes.FILENAME.key())); + addField(doc, SearchableFields.ComponentID, record.getComponentId()); + addField(doc, SearchableFields.AlternateIdentifierURI, record.getAlternateIdentifierUri()); + addField(doc, SearchableFields.EventType, record.getEventType().name()); + addField(doc, SearchableFields.Relationship, record.getRelationship()); + addField(doc, SearchableFields.Details, record.getDetails()); + addField(doc, SearchableFields.ContentClaimSection, record.getContentClaimSection()); + addField(doc, SearchableFields.ContentClaimContainer, record.getContentClaimContainer()); + addField(doc, SearchableFields.ContentClaimIdentifier, record.getContentClaimIdentifier()); + addField(doc, SearchableFields.SourceQueueIdentifier, record.getSourceQueueIdentifier()); + addField(doc, SearchableFields.TransitURI, record.getTransitUri()); + + for (final SearchableField searchableField : searchableAttributeFields) { + addField(doc, searchableField, LuceneUtil.truncateIndexField(record.getAttribute(searchableField.getSearchableFieldName()))); + } + + // Index the fields that we always index (unless there's nothing else to index at all) + if (!doc.getFields().isEmpty()) { + // Always include Lineage Start Date because it allows us to make our Lineage queries more efficient. + doc.add(new LongField(SearchableFields.LineageStartDate.getSearchableFieldName(), record.getLineageStartDate(), Store.NO)); + // Always include Event Time because most queries are bound by a start and end time. + doc.add(new LongField(SearchableFields.EventTime.getSearchableFieldName(), record.getEventTime(), Store.NO)); + // We always include File Size because the UI wants to always render the controls for specifying this. This idea could be revisited. + doc.add(new LongField(SearchableFields.FileSize.getSearchableFieldName(), record.getFileSize(), Store.NO)); + // We always store the event Event ID in the Document but do not index it. It doesn't make sense to query based on Event ID because + // if we want a particular Event ID, we can just obtain it directly from the EventStore. But when we obtain a Document, this info must + // be stored so that we know how to lookup the event in the store. + doc.add(new UnIndexedLongField(SearchableFields.Identifier.getSearchableFieldName(), persistedEvent.getEventId())); + + // If it's event is a FORK, or JOIN, add the FlowFileUUID for all child/parent UUIDs. + final ProvenanceEventType eventType = record.getEventType(); + if (eventType == ProvenanceEventType.FORK || eventType == ProvenanceEventType.CLONE || eventType == ProvenanceEventType.REPLAY) { + for (final String uuid : record.getChildUuids()) { + if (!uuid.equals(record.getFlowFileUuid())) { + addField(doc, SearchableFields.FlowFileUUID, uuid); + } + } + } else if (eventType == ProvenanceEventType.JOIN) { + for (final String uuid : record.getParentUuids()) { + if (!uuid.equals(record.getFlowFileUuid())) { + addField(doc, SearchableFields.FlowFileUUID, uuid); + } + } + } else if (eventType == ProvenanceEventType.RECEIVE && record.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 = record.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); + } + } + + return doc; + } + + return null; + } + + private static class UnIndexedLongField extends Field { + static final FieldType TYPE = new FieldType(); + static { + TYPE.setIndexed(false); + TYPE.setTokenized(true); + TYPE.setOmitNorms(true); + TYPE.setIndexOptions(IndexOptions.DOCS_ONLY); + TYPE.setNumericType(FieldType.NumericType.LONG); + TYPE.setStored(true); + TYPE.freeze(); + } + + public UnIndexedLongField(String name, long value) { + super(name, TYPE); + fieldsData = Long.valueOf(value); + } + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java new file mode 100644 index 0000000000..f8bbd3be22 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java @@ -0,0 +1,244 @@ +/* + * 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.index.lucene; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import org.apache.lucene.document.Document; +import org.apache.lucene.search.NumericRangeQuery; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.index.EventIndexWriter; +import org.apache.nifi.provenance.lucene.IndexManager; +import org.apache.nifi.reporting.Severity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class EventIndexTask implements Runnable { + private static final Logger logger = LoggerFactory.getLogger(EventIndexTask.class); + private static final String EVENT_CATEGORY = "Provenance Repository"; + public static final int MAX_DOCUMENTS_PER_THREAD = 100; + public static final int DEFAULT_MAX_EVENTS_PER_COMMIT = 1_000_000; + + private final BlockingQueue documentQueue; + private final IndexManager indexManager; + private volatile boolean shutdown = false; + + private final IndexDirectoryManager directoryManager; + private final EventReporter eventReporter; + private final int commitThreshold; + + public EventIndexTask(final BlockingQueue documentQueue, final RepositoryConfiguration repoConfig, final IndexManager indexManager, + final IndexDirectoryManager directoryManager, final int maxEventsPerCommit, final EventReporter eventReporter) { + this.documentQueue = documentQueue; + this.indexManager = indexManager; + this.directoryManager = directoryManager; + this.commitThreshold = maxEventsPerCommit; + this.eventReporter = eventReporter; + } + + public void shutdown() { + this.shutdown = true; + } + + private void fetchDocuments(final List destination) throws InterruptedException { + // We want to fetch up to INDEX_BUFFER_SIZE documents at a time. However, we don't want to continually + // call #drainTo on the queue. So we call poll, blocking for up to 1 second. If we get any event, then + // we will call drainTo to gather the rest. If we get no events, then we just return, having gathered + // no events. + StoredDocument firstDoc = documentQueue.poll(1, TimeUnit.SECONDS); + if (firstDoc == null) { + return; + } + + destination.add(firstDoc); + documentQueue.drainTo(destination, MAX_DOCUMENTS_PER_THREAD - 1); + } + + @Override + public void run() { + final List toIndex = new ArrayList<>(MAX_DOCUMENTS_PER_THREAD); + + while (!shutdown) { + try { + // Get the Documents that we want to index. + toIndex.clear(); + fetchDocuments(toIndex); + + if (toIndex.isEmpty()) { + continue; + } + + // Write documents to the currently active index. + final Map> docsByPartition = toIndex.stream() + .collect(Collectors.groupingBy(doc -> doc.getStorageSummary().getPartitionName().get())); + + for (final Map.Entry> entry : docsByPartition.entrySet()) { + final String partitionName = entry.getKey(); + final List docs = entry.getValue(); + + index(docs, partitionName); + } + } catch (final Exception e) { + logger.error("Failed to index Provenance Events", e); + eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to index Provenance Events. See logs for more information."); + } + } + } + + + /** + * Re-indexes the documents given. The IndexableDocument's provided are required to have the IndexDirectory provided. + */ + void reIndex(final List toIndex, final CommitPreference commitPreference) throws IOException { + if (toIndex.isEmpty()) { + return; + } + + final Map> docsByIndexDir = toIndex.stream().collect(Collectors.groupingBy(doc -> doc.getIndexDirectory())); + for (final Map.Entry> entry : docsByIndexDir.entrySet()) { + final File indexDirectory = entry.getKey(); + final List documentsForIndex = entry.getValue(); + + final EventIndexWriter indexWriter = indexManager.borrowIndexWriter(indexDirectory); + try { + // Remove any documents that already exist in this index that are overlapping. + long minId = Long.MAX_VALUE; + long maxId = Long.MIN_VALUE; + + for (final IndexableDocument doc : toIndex) { + final long eventId = doc.getDocument().getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue(); + if (eventId < minId) { + minId = eventId; + } + if (eventId > maxId) { + maxId = eventId; + } + } + + final NumericRangeQuery query = NumericRangeQuery.newLongRange( + SearchableFields.Identifier.getSearchableFieldName(), minId, maxId, true, true); + indexWriter.getIndexWriter().deleteDocuments(query); + + final List documents = documentsForIndex.stream() + .map(doc -> doc.getDocument()) + .collect(Collectors.toList()); + + indexWriter.index(documents, commitThreshold); + } finally { + indexManager.returnIndexWriter(indexWriter, CommitPreference.FORCE_COMMIT.equals(commitPreference), false); + } + } + } + + + private void index(final List toIndex, final String partitionName) throws IOException { + if (toIndex.isEmpty()) { + return; + } + + // Convert the IndexableDocument list into a List of Documents so that we can pass them to the Index Writer. + final List documents = toIndex.stream() + .map(doc -> doc.getDocument()) + .collect(Collectors.toList()); + + boolean requestClose = false; + boolean requestCommit = false; + + final long minEventTime = toIndex.stream() + .mapToLong(doc -> doc.getDocument().getField(SearchableFields.EventTime.getSearchableFieldName()).numericValue().longValue()) + .min() + .getAsLong(); + + // Synchronize on the directory manager because we don't want the active directory to change + // while we are obtaining an index writer for it. I.e., determining the active directory + // and obtaining an Index Writer for it need to be done atomically. + final EventIndexWriter indexWriter; + final File indexDirectory; + synchronized (directoryManager) { + indexDirectory = directoryManager.getWritableIndexingDirectory(minEventTime, partitionName); + indexWriter = indexManager.borrowIndexWriter(indexDirectory); + } + + try { + // Perform the actual indexing. + boolean writerIndicatesCommit = indexWriter.index(documents, commitThreshold); + + // If we don't need to commit index based on what index writer tells us, we will still want + // to commit the index if it's assigned to a partition and this is no longer the active index + // for that partition. This prevents the following case: + // + // Thread T1: pulls events from queue + // Maps events to Index Directory D1 + // Thread T2: pulls events from queue + // Maps events to Index Directory D1, the active index for Partition P1. + // Writes events to D1. + // Commits Index Writer for D1. + // Closes Index Writer for D1. + // Thread T1: Writes events to D1. + // Determines that Index Writer for D1 does not need to be committed or closed. + // + // In the case outlined above, we would potentially lose those events from the index! To avoid this, + // we simply decide to commit the index if this writer is no longer the active writer for the index. + // However, if we have 10 threads, we don't want all 10 threads trying to commit the index after each + // update. We want to commit when they've all finished. This is what the IndexManager will do if we request + // that it commit the index. It will also close the index if requested, once all writers have finished. + // So when this is the case, we will request that the Index Manager both commit and close the writer. + + final Optional activeIndexDirOption = directoryManager.getActiveIndexDirectory(partitionName); + if (!activeIndexDirOption.isPresent() || !activeIndexDirOption.get().equals(indexDirectory)) { + requestCommit = true; + requestClose = true; + } + + if (writerIndicatesCommit) { + commit(indexWriter); + requestCommit = false; // we've already committed the index writer so no need to request that the index manager do so also. + final boolean directoryManagerIndicatesClose = directoryManager.onIndexCommitted(indexDirectory); + requestClose = requestClose || directoryManagerIndicatesClose; + + if (logger.isDebugEnabled()) { + final long maxId = documents.stream() + .mapToLong(doc -> doc.getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue()) + .max() + .orElse(-1L); + logger.debug("Committed index {} after writing a max Event ID of {}", indexDirectory, maxId); + } + } + } finally { + indexManager.returnIndexWriter(indexWriter, requestCommit, requestClose); + } + } + + + protected void commit(final EventIndexWriter indexWriter) throws IOException { + final long start = System.nanoTime(); + final long approximateCommitCount = indexWriter.commit(); + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + logger.debug("Successfully committed approximately {} Events to {} in {} millis", approximateCommitCount, indexWriter, millis); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexDirectoryManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexDirectoryManager.java new file mode 100644 index 0000000000..09878ff9ab --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexDirectoryManager.java @@ -0,0 +1,358 @@ +/* + * 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.index.lucene; + +import java.io.File; +import java.io.FileFilter; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.util.DirectoryUtils; +import org.apache.nifi.util.Tuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IndexDirectoryManager { + private static final Logger logger = LoggerFactory.getLogger(IndexDirectoryManager.class); + private static final FileFilter INDEX_DIRECTORY_FILTER = f -> f.getName().startsWith("index-"); + private static final Pattern INDEX_FILENAME_PATTERN = Pattern.compile("index-(\\d+)"); + + private final RepositoryConfiguration repoConfig; + + // guarded by synchronizing on 'this' + private final SortedMap> indexLocationByTimestamp = new TreeMap<>(); + private final Map activeIndices = new HashMap<>(); + + public IndexDirectoryManager(final RepositoryConfiguration repoConfig) { + this.repoConfig = repoConfig; + } + + public synchronized void initialize() { + final Map> latestIndexByStorageDir = new HashMap<>(); + + for (final Map.Entry entry : repoConfig.getStorageDirectories().entrySet()) { + final String partitionName = entry.getKey(); + final File storageDir = entry.getValue(); + + final File[] indexDirs = storageDir.listFiles(INDEX_DIRECTORY_FILTER); + if (indexDirs == null) { + logger.warn("Unable to access Provenance Repository storage directory {}", storageDir); + continue; + } + + for (final File indexDir : indexDirs) { + final Matcher matcher = INDEX_FILENAME_PATTERN.matcher(indexDir.getName()); + if (!matcher.matches()) { + continue; + } + + final long startTime = DirectoryUtils.getIndexTimestamp(indexDir); + final List dirsForTimestamp = indexLocationByTimestamp.computeIfAbsent(startTime, t -> new ArrayList<>()); + final IndexLocation indexLoc = new IndexLocation(indexDir, startTime, partitionName, repoConfig.getDesiredIndexSize()); + dirsForTimestamp.add(indexLoc); + + final Tuple tuple = latestIndexByStorageDir.get(storageDir); + if (tuple == null || startTime > tuple.getKey()) { + latestIndexByStorageDir.put(storageDir, new Tuple<>(startTime, indexLoc)); + } + } + } + + // Restore the activeIndices to point at the newest index in each storage location. + for (final Tuple tuple : latestIndexByStorageDir.values()) { + final IndexLocation indexLoc = tuple.getValue(); + activeIndices.put(indexLoc.getPartitionName(), indexLoc); + } + } + + + public synchronized void deleteDirectory(final File directory) { + final Iterator>> itr = indexLocationByTimestamp.entrySet().iterator(); + while (itr.hasNext()) { + final Map.Entry> entry = itr.next(); + final List locations = entry.getValue(); + + final IndexLocation locToRemove = new IndexLocation(directory, DirectoryUtils.getIndexTimestamp(directory), + directory.getName(), repoConfig.getDesiredIndexSize()); + locations.remove(locToRemove); + if (locations.isEmpty()) { + itr.remove(); + } + } + } + + /** + * Returns a List of all indexes where the latest event in the index has an event time before the given timestamp + * + * @param timestamp the cutoff + * @return all Files that belong to an index, where the index has no events later than the given time + */ + public synchronized List getDirectoriesBefore(final long timestamp) { + final List selected = new ArrayList<>(); + + // An index cannot be expired if it is the latest index in the storage directory. As a result, we need to + // separate the indexes by Storage Directory so that we can easily determine if this is the case. + final Map> startTimeWithFileByStorageDirectory = flattenDirectoriesByTimestamp().stream() + .collect(Collectors.groupingBy(indexLoc -> indexLoc.getPartitionName())); + + // Scan through the index directories and the associated index event start time. + // If looking at index N, we can determine the index end time by assuming that it is the same as the + // start time of index N+1. So we determine the time range of each index and select an index only if + // its start time is before the given timestamp and its end time is <= the given timestamp. + for (final List startTimeWithFile : startTimeWithFileByStorageDirectory.values()) { + for (int i = 0; i < startTimeWithFile.size(); i++) { + final IndexLocation indexLoc = startTimeWithFile.get(i); + + final String partition = indexLoc.getPartitionName(); + final IndexLocation activeLocation = activeIndices.get(partition); + if (indexLoc.equals(activeLocation)) { + continue; + } + + final Long indexStartTime = indexLoc.getIndexStartTimestamp(); + if (indexStartTime > timestamp) { + // If the first timestamp in the index is later than the desired timestamp, + // then we are done. We can do this because the list is ordered by monotonically + // increasing timestamp as the Tuple key. + break; + } + + if (i < startTimeWithFile.size() - 1) { + final IndexLocation nextLocation = startTimeWithFile.get(i + 1); + final Long indexEndTime = nextLocation.getIndexStartTimestamp(); + if (indexEndTime <= timestamp) { + logger.debug("Considering Index Location {} older than {} ({}) because its events have an EventTime " + + "ranging from {} ({}) to {} ({}) based on the following IndexLocations: {}", nextLocation, timestamp, new Date(timestamp), + indexStartTime, new Date(indexStartTime), indexEndTime, new Date(indexEndTime), startTimeWithFile); + + selected.add(nextLocation.getIndexDirectory()); + } + } + } + } + + logger.debug("Returning the following list of index locations because they were finished being written to before {}: {}", timestamp, selected); + return selected; + } + + /** + * Convert directoriesByTimestamp to a List of IndexLocations. + * This allows us to easily get the 'next' value when iterating over the elements. + * This is useful because we know that the 'next' value will have a timestamp that is when that + * file started being written to - which is the same as when this index stopped being written to. + * + * @return a List of all IndexLocations known + */ + private List flattenDirectoriesByTimestamp() { + final List startTimeWithFile = new ArrayList<>(); + for (final Map.Entry> entry : indexLocationByTimestamp.entrySet()) { + for (final IndexLocation indexLoc : entry.getValue()) { + startTimeWithFile.add(indexLoc); + } + } + + return startTimeWithFile; + } + + public synchronized List getDirectories(final Long startTime, final Long endTime) { + final List selected = new ArrayList<>(); + + // An index cannot be expired if it is the latest index in the partition. As a result, we need to + // separate the indexes by partition so that we can easily determine if this is the case. + final Map> startTimeWithFileByStorageDirectory = flattenDirectoriesByTimestamp().stream() + .collect(Collectors.groupingBy(indexLoc -> indexLoc.getPartitionName())); + + for (final List locationList : startTimeWithFileByStorageDirectory.values()) { + selected.addAll(getDirectories(startTime, endTime, locationList)); + } + + return selected; + } + + public synchronized List getDirectories(final Long startTime, final Long endTime, final String partitionName) { + // An index cannot be expired if it is the latest index in the partition. As a result, we need to + // separate the indexes by partition so that we can easily determine if this is the case. + final Map> startTimeWithFileByStorageDirectory = flattenDirectoriesByTimestamp().stream() + .collect(Collectors.groupingBy(indexLoc -> indexLoc.getPartitionName())); + + final List indexLocations = startTimeWithFileByStorageDirectory.get(partitionName); + if (indexLocations == null) { + return Collections.emptyList(); + } + + return getDirectories(startTime, endTime, indexLocations); + } + + protected static List getDirectories(final Long startTime, final Long endTime, final List locations) { + final List selected = new ArrayList<>(); + + int overlapCount = 0; + for (int i = 0; i < locations.size(); i++) { + final IndexLocation indexLoc = locations.get(i); + final Long indexStartTimestamp = indexLoc.getIndexStartTimestamp(); + if (endTime != null && indexStartTimestamp > endTime) { + if (overlapCount == 0) { + // Because of how we handle index timestamps and the multi-threading, it is possible + // the we could have some overlap where Thread T1 gets an Event with start time 1,000 + // for instance. Then T2 gets and Event with start time 1,002 and ends up creating a + // new index directory with a start time of 1,002. Then T1 could end up writing events + // with timestamp 1,000 to an index with a 'start time' of 1,002. Because of this, + // the index start times are approximate. To address this, we include one extra Index + // Directory based on start time, so that if we want index directories for Time Range + // 1,000 - 1,001 and have indexes 999 and 1,002 we will include the 999 and the 'overlapping' + // directory of 1,002 since it could potentially have an event with overlapping timestamp. + overlapCount++; + } else { + continue; + } + } + + if (startTime != null) { + final Long indexEndTimestamp; + if (i < locations.size() - 1) { + final IndexLocation nextIndexLoc = locations.get(i + 1); + indexEndTimestamp = nextIndexLoc.getIndexStartTimestamp(); + if (indexEndTimestamp < startTime) { + continue; + } + } + } + + selected.add(indexLoc.getIndexDirectory()); + } + + return selected; + } + + /** + * Notifies the Index Directory Manager that an Index Writer has been committed for the + * given index directory. This allows the Directory Manager to know that it needs to check + * the size of the index directory and not return this directory as a writable directory + * any more if the size has reached the configured threshold. + * + * @param indexDir the directory that was written to + * @return true if the index directory has reached its max threshold and should no + * longer be written to, false if the index directory is not full. + */ + public boolean onIndexCommitted(final File indexDir) { + final long indexSize = getSize(indexDir); + synchronized (this) { + String partitionName = null; + for (final Map.Entry entry : activeIndices.entrySet()) { + if (indexDir.equals(entry.getValue().getIndexDirectory())) { + partitionName = entry.getKey(); + break; + } + } + + // If the index is not the active index directory, it should no longer be written to. + if (partitionName == null) { + logger.debug("Size of Provenance Index at {} is now {}. However, was unable to find the appropriate Active Index to roll over.", indexDir, indexSize); + return true; + } + + // If the index size >= desired index size, it should no longer be written to. + if (indexSize >= repoConfig.getDesiredIndexSize()) { + logger.info("Size of Provenance Index at {} is now {}. Will close this index and roll over to a new one.", indexDir, indexSize); + activeIndices.remove(partitionName); + + return true; + } + + // Index directory is the active index directory and has not yet exceeded the desired size. + return false; + } + } + + public synchronized Optional getActiveIndexDirectory(final String partitionName) { + final IndexLocation indexLocation = activeIndices.get(partitionName); + if (indexLocation == null) { + return Optional.empty(); + } + + return Optional.of(indexLocation.getIndexDirectory()); + } + + private long getSize(final File indexDir) { + if (!indexDir.exists()) { + return 0L; + } + if (!indexDir.isDirectory()) { + throw new IllegalArgumentException("Must specify a directory but specified " + indexDir); + } + + // List all files in the Index Directory. + final File[] files = indexDir.listFiles(); + if (files == null) { + return 0L; + } + + long sum = 0L; + for (final File file : files) { + sum += file.length(); + } + + return sum; + } + + /** + * Provides the File that is the directory for the index that should be written to. If there is no index yet + * to be written to, or if the index has reached its max size, a new one will be created. The given {@code earliestTimestamp} + * should represent the event time of the first event that will go into the index. This is used for file naming purposes so + * that the appropriate directories can be looked up quickly later. + * + * @param earliestTimestamp the event time of the first event that will go into a new index, if a new index is created by this call. + * @param partitionName the name of the partition to write to + * @return the directory that should be written to + */ + public synchronized File getWritableIndexingDirectory(final long earliestTimestamp, final String partitionName) { + IndexLocation indexLoc = activeIndices.get(partitionName); + if (indexLoc == null || indexLoc.isIndexFull()) { + indexLoc = new IndexLocation(createIndex(earliestTimestamp, partitionName), earliestTimestamp, partitionName, repoConfig.getDesiredIndexSize()); + logger.debug("Created new Index Directory {}", indexLoc); + + indexLocationByTimestamp.computeIfAbsent(earliestTimestamp, t -> new ArrayList<>()).add(indexLoc); + activeIndices.put(partitionName, indexLoc); + } + + return indexLoc.getIndexDirectory(); + } + + private File createIndex(final long earliestTimestamp, final String partitionName) { + final File storageDir = repoConfig.getStorageDirectories().entrySet().stream() + .filter(e -> e.getKey().equals(partitionName)) + .map(Map.Entry::getValue) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException("Invalid Partition: " + partitionName)); + final File indexDir = new File(storageDir, "index-" + earliestTimestamp); + + return indexDir; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexLocation.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexLocation.java new file mode 100644 index 0000000000..33867c6d6a --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexLocation.java @@ -0,0 +1,90 @@ +/* + * 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.index.lucene; + +import java.io.File; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.provenance.util.DirectoryUtils; + +public class IndexLocation { + private static final long SIZE_CHECK_MILLIS = TimeUnit.SECONDS.toMillis(30L); + + private final File indexDirectory; + private final long indexStartTimestamp; + private final String partitionName; + private final long desiredIndexSize; + private volatile long lastSizeCheckTime = System.currentTimeMillis(); + + public IndexLocation(final File indexDirectory, final long indexStartTimestamp, final String partitionName, final long desiredIndexSize) { + this.indexDirectory = indexDirectory; + this.indexStartTimestamp = indexStartTimestamp; + this.partitionName = partitionName; + this.desiredIndexSize = desiredIndexSize; + } + + public File getIndexDirectory() { + return indexDirectory; + } + + public long getIndexStartTimestamp() { + return indexStartTimestamp; + } + + public String getPartitionName() { + return partitionName; + } + + public boolean isIndexFull() { + final long now = System.currentTimeMillis(); + final long millisSinceLastSizeCheck = now - lastSizeCheckTime; + if (millisSinceLastSizeCheck < SIZE_CHECK_MILLIS) { + return false; + } + + lastSizeCheckTime = now; + return DirectoryUtils.getSize(indexDirectory) >= desiredIndexSize; + } + + @Override + public int hashCode() { + return 31 + 41 * indexDirectory.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + + if (!(obj instanceof IndexLocation)) { + return false; + } + + final IndexLocation other = (IndexLocation) obj; + return indexDirectory.equals(other.getIndexDirectory()); + } + + @Override + public String toString() { + return "IndexLocation[directory=" + indexDirectory + "]"; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexableDocument.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexableDocument.java new file mode 100644 index 0000000000..1fc163f2ea --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/IndexableDocument.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.index.lucene; + +import java.io.File; + +import org.apache.lucene.document.Document; +import org.apache.nifi.provenance.serialization.StorageSummary; + +public class IndexableDocument { + private final Document document; + private final StorageSummary persistenceLocation; + private final File indexDirectory; + + public IndexableDocument(final Document document, final StorageSummary location, final File indexDirectory) { + this.document = document; + this.persistenceLocation = location; + this.indexDirectory = indexDirectory; + } + + public Document getDocument() { + return document; + } + + public StorageSummary getPersistenceLocation() { + return persistenceLocation; + } + + public File getIndexDirectory() { + return indexDirectory; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LatestEventsPerProcessorQuery.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LatestEventsPerProcessorQuery.java new file mode 100644 index 0000000000..73b0a14b3a --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LatestEventsPerProcessorQuery.java @@ -0,0 +1,77 @@ +/* + * 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.index.lucene; + +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.search.Query; +import org.apache.nifi.provenance.search.SearchTerm; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.util.RingBuffer; + +public class LatestEventsPerProcessorQuery implements CachedQuery { + private static final String COMPONENT_ID_FIELD_NAME = SearchableFields.ComponentID.getSearchableFieldName(); + private final ConcurrentMap> latestRecords = new ConcurrentHashMap<>(); + + @Override + public void update(final ProvenanceEventRecord event, final StorageSummary storageSummary) { + final String componentId = event.getComponentId(); + final RingBuffer ringBuffer = latestRecords.computeIfAbsent(componentId, id -> new RingBuffer<>(1000)); + ringBuffer.add(storageSummary.getEventId()); + } + + @Override + public Optional> evaluate(final Query query) { + if (query.getMaxResults() > 1000) { + // If query max results > 1000 then we know we don't have enough results. So just return empty. + return Optional.empty(); + } + + final List terms = query.getSearchTerms(); + if (terms.size() != 1) { + return Optional.empty(); + } + + final SearchTerm term = terms.get(0); + if (!COMPONENT_ID_FIELD_NAME.equals(term.getSearchableField().getSearchableFieldName())) { + return Optional.empty(); + } + + if (query.getEndDate() != null || query.getStartDate() != null) { + return Optional.empty(); + } + + final RingBuffer ringBuffer = latestRecords.get(term.getValue()); + if (ringBuffer == null || ringBuffer.getSize() < query.getMaxResults()) { + return Optional.empty(); + } + + List eventIds = ringBuffer.asList(); + if (eventIds.size() > query.getMaxResults()) { + eventIds = eventIds.subList(0, query.getMaxResults()); + } + + return Optional.of(eventIds); + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LatestEventsQuery.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LatestEventsQuery.java new file mode 100644 index 0000000000..94cd013285 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LatestEventsQuery.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.index.lucene; + +import java.util.List; +import java.util.Optional; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.search.Query; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.util.RingBuffer; + +public class LatestEventsQuery implements CachedQuery { + + final RingBuffer latestRecords = new RingBuffer<>(1000); + + @Override + public void update(final ProvenanceEventRecord event, final StorageSummary storageSummary) { + latestRecords.add(storageSummary.getEventId()); + } + + @Override + public Optional> evaluate(final Query query) { + if (latestRecords.getSize() < query.getMaxResults()) { + return Optional.empty(); + } + + if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) { + final List eventList = latestRecords.asList(); + if (eventList.size() > query.getMaxResults()) { + return Optional.of(eventList.subList(0, query.getMaxResults())); + } else { + return Optional.of(eventList); + } + } else { + return Optional.empty(); + } + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneCacheWarmer.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneCacheWarmer.java new file mode 100644 index 0000000000..15b11b4e65 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneCacheWarmer.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.index.lucene; + +import java.io.File; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.lucene.IndexManager; +import org.apache.nifi.provenance.util.DirectoryUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LuceneCacheWarmer implements Runnable { + private static final Logger logger = LoggerFactory.getLogger(LuceneCacheWarmer.class); + + private final File storageDir; + private final IndexManager indexManager; + + public LuceneCacheWarmer(final File storageDir, final IndexManager indexManager) { + this.storageDir = storageDir; + this.indexManager = indexManager; + } + + @Override + public void run() { + try { + final File[] indexDirs = storageDir.listFiles(DirectoryUtils.INDEX_FILE_FILTER); + if (indexDirs == null) { + logger.info("Cannot warm Lucene Index Cache for " + storageDir + " because the directory could not be read"); + return; + } + + logger.info("Beginning warming of Lucene Index Cache for " + storageDir); + final long startNanos = System.nanoTime(); + for (final File indexDir : indexDirs) { + final long indexStartNanos = System.nanoTime(); + + final EventIndexSearcher eventSearcher = indexManager.borrowIndexSearcher(indexDir); + indexManager.returnIndexSearcher(eventSearcher); + + final long indexWarmMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - indexStartNanos); + logger.debug("Took {} ms to warm Lucene Index {}", indexWarmMillis, indexDir); + } + + final long warmSecs = TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - startNanos); + logger.info("Finished warming all Lucene Indexes for {} in {} seconds", storageDir, warmSecs); + } catch (final Exception e) { + logger.error("Failed to warm Lucene Index Cache for " + storageDir, e); + } + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java new file mode 100644 index 0000000000..a58340343b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java @@ -0,0 +1,737 @@ +/* + * 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.index.lucene; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.TermQuery; +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.AsyncLineageSubmission; +import org.apache.nifi.provenance.AsyncQuerySubmission; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.StandardLineageResult; +import org.apache.nifi.provenance.StandardQueryResult; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.EventTransformer; +import org.apache.nifi.provenance.index.EventIndex; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.EventIndexWriter; +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; +import org.apache.nifi.provenance.lineage.LineageComputationType; +import org.apache.nifi.provenance.lucene.IndexManager; +import org.apache.nifi.provenance.lucene.LuceneUtil; +import org.apache.nifi.provenance.search.Query; +import org.apache.nifi.provenance.search.QuerySubmission; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.provenance.store.EventStore; +import org.apache.nifi.provenance.util.DirectoryUtils; +import org.apache.nifi.provenance.util.NamedThreadFactory; +import org.apache.nifi.reporting.Severity; +import org.apache.nifi.util.file.FileUtils; +import org.apache.nifi.util.timebuffer.LongEntityAccess; +import org.apache.nifi.util.timebuffer.TimedBuffer; +import org.apache.nifi.util.timebuffer.TimestampedLong; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class LuceneEventIndex implements EventIndex { + private static final Logger logger = LoggerFactory.getLogger(LuceneEventIndex.class); + private static final String EVENT_CATEGORY = "Provenance Repository"; + + public static final int MAX_UNDELETED_QUERY_RESULTS = 10; + public static final int MAX_DELETE_INDEX_WAIT_SECONDS = 30; + public static final int MAX_LINEAGE_NODES = 1000; + public static final int MAX_INDEX_THREADS = 100; + + private final ConcurrentMap querySubmissionMap = new ConcurrentHashMap<>(); + private final ConcurrentMap lineageSubmissionMap = new ConcurrentHashMap<>(); + private final BlockingQueue documentQueue = new LinkedBlockingQueue<>(1000); + private final List indexTasks = Collections.synchronizedList(new ArrayList<>()); + private final ExecutorService queryExecutor; + private final ExecutorService indexExecutor; + private final RepositoryConfiguration config; + private final IndexManager indexManager; + private final ConvertEventToLuceneDocument eventConverter; + private final IndexDirectoryManager directoryManager; + private volatile boolean closed = false; + + private final TimedBuffer queuePauseNanos = new TimedBuffer<>(TimeUnit.SECONDS, 300, new LongEntityAccess()); + private final TimedBuffer eventsIndexed = new TimedBuffer<>(TimeUnit.SECONDS, 300, new LongEntityAccess()); + private final AtomicLong eventCount = new AtomicLong(0L); + private final EventReporter eventReporter; + + private final List cachedQueries = new ArrayList<>(); + + private ScheduledExecutorService maintenanceExecutor; // effectively final + private ScheduledExecutorService cacheWarmerExecutor; + private EventStore eventStore; + + public LuceneEventIndex(final RepositoryConfiguration config, final IndexManager indexManager, final EventReporter eventReporter) { + this(config, indexManager, EventIndexTask.DEFAULT_MAX_EVENTS_PER_COMMIT, eventReporter); + } + + public LuceneEventIndex(final RepositoryConfiguration config, final IndexManager indexManager, final int maxEventsPerCommit, final EventReporter eventReporter) { + this.eventReporter = eventReporter; + queryExecutor = Executors.newFixedThreadPool(config.getQueryThreadPoolSize(), new NamedThreadFactory("Provenance Query")); + indexExecutor = Executors.newFixedThreadPool(config.getIndexThreadPoolSize(), new NamedThreadFactory("Index Provenance Events")); + cacheWarmerExecutor = Executors.newScheduledThreadPool(config.getStorageDirectories().size(), new NamedThreadFactory("Warm Lucene Index", true)); + directoryManager = new IndexDirectoryManager(config); + + // Limit number of indexing threads to 100. When we restore the repository on restart, + // we have to re-index up to MAX_THREADS * MAX_DOCUMENTS_PER_THREADS events prior to + // the last event that the index holds. This is done because we could have that many + // events 'in flight', waiting to be indexed when the last index writer was committed, + // so even though the index says the largest event ID is 1,000,000 for instance, Event + // with ID 999,999 may still not have been indexed because another thread was in the + // process of writing the event to the index. + final int configuredIndexPoolSize = config.getIndexThreadPoolSize(); + final int numIndexThreads; + if (configuredIndexPoolSize > MAX_INDEX_THREADS) { + logger.warn("The Provenance Repository is configured to perform indexing of events using {} threads. This number exceeds the maximum allowable number of threads, which is {}. " + + "Will proceed using {} threads. This value is limited because the performance of indexing will decrease and startup times will increase when setting this value too high.", + configuredIndexPoolSize, MAX_INDEX_THREADS, MAX_INDEX_THREADS); + numIndexThreads = MAX_INDEX_THREADS; + } else { + numIndexThreads = configuredIndexPoolSize; + } + + for (int i = 0; i < numIndexThreads; i++) { + final EventIndexTask task = new EventIndexTask(documentQueue, config, indexManager, directoryManager, maxEventsPerCommit, eventReporter); + indexTasks.add(task); + indexExecutor.submit(task); + } + this.config = config; + this.indexManager = indexManager; + this.eventConverter = new ConvertEventToLuceneDocument(config.getSearchableFields(), config.getSearchableAttributes()); + } + + @Override + public void initialize(final EventStore eventStore) { + this.eventStore = eventStore; + directoryManager.initialize(); + + maintenanceExecutor = Executors.newScheduledThreadPool(1, new NamedThreadFactory("Provenance Repository Maintenance")); + maintenanceExecutor.scheduleWithFixedDelay(() -> performMaintenance(), 1, 1, TimeUnit.MINUTES); + maintenanceExecutor.scheduleWithFixedDelay(new RemoveExpiredQueryResults(), 30, 30, TimeUnit.SECONDS); + + cachedQueries.add(new LatestEventsQuery()); + cachedQueries.add(new LatestEventsPerProcessorQuery()); + + final Optional warmCacheMinutesOption = config.getWarmCacheFrequencyMinutes(); + if (warmCacheMinutesOption.isPresent() && warmCacheMinutesOption.get() > 0) { + for (final File storageDir : config.getStorageDirectories().values()) { + final int minutes = warmCacheMinutesOption.get(); + cacheWarmerExecutor.scheduleWithFixedDelay(new LuceneCacheWarmer(storageDir, indexManager), 1, minutes, TimeUnit.MINUTES); + } + } + } + + @Override + public long getMinimumEventIdToReindex(final String partitionName) { + return Math.max(0, getMaxEventId(partitionName) - EventIndexTask.MAX_DOCUMENTS_PER_THREAD * LuceneEventIndex.MAX_INDEX_THREADS); + } + + protected IndexDirectoryManager getDirectoryManager() { + return directoryManager; + } + + @Override + public void close() throws IOException { + closed = true; + queryExecutor.shutdownNow(); + indexExecutor.shutdown(); + cacheWarmerExecutor.shutdown(); + + if (maintenanceExecutor != null) { + maintenanceExecutor.shutdown(); + } + + for (final EventIndexTask task : indexTasks) { + task.shutdown(); + } + } + + long getMaxEventId(final String partitionName) { + final List allDirectories = getDirectoryManager().getDirectories(0L, Long.MAX_VALUE, partitionName); + if (allDirectories.isEmpty()) { + return -1L; + } + + Collections.sort(allDirectories, DirectoryUtils.NEWEST_INDEX_FIRST); + + for (final File directory : allDirectories) { + final EventIndexSearcher searcher; + try { + searcher = indexManager.borrowIndexSearcher(directory); + } catch (final IOException ioe) { + logger.warn("Unable to read from Index Directory {}. Will assume that the index is incomplete and not consider this index when determining max event ID", directory); + continue; + } + + try { + final IndexReader reader = searcher.getIndexSearcher().getIndexReader(); + final int maxDocId = reader.maxDoc() - 1; + final Document document = reader.document(maxDocId); + final long eventId = document.getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue(); + logger.info("Determined that Max Event ID indexed for Partition {} is approximately {} based on index {}", partitionName, eventId, directory); + return eventId; + } catch (final IOException ioe) { + logger.warn("Unable to search Index Directory {}. Will assume that the index is incomplete and not consider this index when determining max event ID", directory, ioe); + } finally { + indexManager.returnIndexSearcher(searcher); + } + } + + return -1L; + } + + @Override + public void reindexEvents(final Map events) { + final EventIndexTask indexTask = new EventIndexTask(documentQueue, config, indexManager, directoryManager, EventIndexTask.DEFAULT_MAX_EVENTS_PER_COMMIT, eventReporter); + + File lastIndexDir = null; + long lastEventTime = -2L; + + final List indexableDocs = new ArrayList<>(events.size()); + for (final Map.Entry entry : events.entrySet()) { + final ProvenanceEventRecord event = entry.getKey(); + final StorageSummary summary = entry.getValue(); + + for (final CachedQuery cachedQuery : cachedQueries) { + cachedQuery.update(event, summary); + } + + final Document document = eventConverter.convert(event, summary); + if (document == null) { + logger.debug("Received Provenance Event {} to index but it contained no information that should be indexed, so skipping it", event); + } else { + final File indexDir; + if (event.getEventTime() == lastEventTime) { + indexDir = lastIndexDir; + } else { + final List files = getDirectoryManager().getDirectories(event.getEventTime(), null); + indexDir = files.isEmpty() ? null : files.get(0); + lastIndexDir = indexDir; + } + + final IndexableDocument doc = new IndexableDocument(document, summary, indexDir); + indexableDocs.add(doc); + } + } + + try { + indexTask.reIndex(indexableDocs, CommitPreference.PREVENT_COMMIT); + } catch (final IOException ioe) { + logger.error("Failed to reindex some Provenance Events", ioe); + eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to re-index some Provenance Events. " + + "Some Provenance Events may not be available for querying. See logs for more information."); + } + } + + @Override + public void commitChanges(final String partitionName) throws IOException { + final Optional indexDir = directoryManager.getActiveIndexDirectory(partitionName); + if (indexDir.isPresent()) { + final EventIndexWriter eventIndexWriter = indexManager.borrowIndexWriter(indexDir.get()); + try { + eventIndexWriter.commit(); + } finally { + indexManager.returnIndexWriter(eventIndexWriter, false, false); + } + } + } + + protected void addEvent(final ProvenanceEventRecord event, final StorageSummary location) { + for (final CachedQuery cachedQuery : cachedQueries) { + cachedQuery.update(event, location); + } + + final Document document = eventConverter.convert(event, location); + if (document == null) { + logger.debug("Received Provenance Event {} to index but it contained no information that should be indexed, so skipping it", event); + } else { + final StoredDocument doc = new StoredDocument(document, location); + boolean added = false; + while (!added && !closed) { + + added = documentQueue.offer(doc); + if (!added) { + final long start = System.nanoTime(); + try { + added = documentQueue.offer(doc, 1, TimeUnit.SECONDS); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + logger.warn("Interrupted while attempting to enqueue Provenance Event for indexing; this event will not be indexed"); + return; + } + final long nanos = System.nanoTime() - start; + queuePauseNanos.add(new TimestampedLong(nanos)); + } + + if (added) { + final long totalEventCount = eventCount.incrementAndGet(); + if (totalEventCount % 1_000_000 == 0 && logger.isDebugEnabled()) { + incrementAndReportStats(); + } + } + } + } + } + + private void incrementAndReportStats() { + final long fiveMinutesAgo = System.currentTimeMillis() - TimeUnit.MINUTES.toMillis(5); + final TimestampedLong nanosLastFive = queuePauseNanos.getAggregateValue(fiveMinutesAgo); + if (nanosLastFive == null) { + return; + } + + final TimestampedLong eventsLast5 = eventsIndexed.getAggregateValue(fiveMinutesAgo); + if (eventsLast5 == null) { + return; + } + + final long numEventsLast5 = eventsLast5.getValue(); + + final long millis = TimeUnit.NANOSECONDS.toMillis(nanosLastFive.getValue()); + logger.debug("In the last 5 minutes, have spent {} CPU-millis waiting to enqueue events for indexing and have indexed {} events ({} since NiFi started)", + millis, numEventsLast5, eventCount.get()); + } + + @Override + public void addEvents(final Map events) { + eventsIndexed.add(new TimestampedLong((long) events.size())); + + for (final Map.Entry entry : events.entrySet()) { + addEvent(entry.getKey(), entry.getValue()); + } + } + + + @Override + public ComputeLineageSubmission submitLineageComputation(final long eventId, final NiFiUser user, final EventAuthorizer eventAuthorizer) { + final Optional eventOption; + try { + eventOption = eventStore.getEvent(eventId); + } catch (final Exception e) { + logger.error("Failed to retrieve Provenance Event with ID " + eventId + " to calculate data lineage due to: " + e, e); + final AsyncLineageSubmission result = new AsyncLineageSubmission(LineageComputationType.FLOWFILE_LINEAGE, eventId, Collections. emptySet(), 1, user.getIdentity()); + result.getResult().setError("Failed to retrieve Provenance Event with ID " + eventId + ". See logs for more information."); + return result; + } + + if (!eventOption.isPresent()) { + final AsyncLineageSubmission result = new AsyncLineageSubmission(LineageComputationType.FLOWFILE_LINEAGE, eventId, Collections. emptySet(), 1, user.getIdentity()); + result.getResult().setError("Could not find Provenance Event with ID " + eventId); + lineageSubmissionMap.put(result.getLineageIdentifier(), result); + return result; + } + + final ProvenanceEventRecord event = eventOption.get(); + return submitLineageComputation(Collections.singleton(event.getFlowFileUuid()), user, eventAuthorizer, LineageComputationType.FLOWFILE_LINEAGE, + eventId, event.getLineageStartDate(), Long.MAX_VALUE); + } + + + private ComputeLineageSubmission submitLineageComputation(final Collection flowFileUuids, final NiFiUser user, final EventAuthorizer eventAuthorizer, + final LineageComputationType computationType, final Long eventId, final long startTimestamp, final long endTimestamp) { + + final List indexDirs = directoryManager.getDirectories(startTimestamp, endTimestamp); + final AsyncLineageSubmission submission = new AsyncLineageSubmission(computationType, eventId, flowFileUuids, indexDirs.size(), user.getIdentity()); + lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); + + final BooleanQuery lineageQuery = buildLineageQuery(flowFileUuids); + final List indexDirectories = directoryManager.getDirectories(startTimestamp, endTimestamp); + if (indexDirectories.isEmpty()) { + submission.getResult().update(Collections.emptyList(), 0L); + } else { + Collections.sort(indexDirectories, DirectoryUtils.OLDEST_INDEX_FIRST); + + for (final File indexDir : indexDirectories) { + queryExecutor.submit(new QueryTask(lineageQuery, submission.getResult(), MAX_LINEAGE_NODES, indexManager, indexDir, + eventStore, eventAuthorizer, EventTransformer.PLACEHOLDER_TRANSFORMER)); + } + } + + // Some computations will complete very quickly. In this case, we don't want to wait + // for the client to submit a second query to obtain the result. Instead, we want to just + // wait some short period of time for the computation to complete before returning the submission. + try { + submission.getResult().awaitCompletion(500, TimeUnit.MILLISECONDS); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + } + + return submission; + } + + private BooleanQuery buildLineageQuery(final Collection flowFileUuids) { + // 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 lineageQuery; + if (flowFileUuids == null || flowFileUuids.isEmpty()) { + lineageQuery = null; + } else { + lineageQuery = new BooleanQuery(); + for (final String flowFileUuid : flowFileUuids) { + lineageQuery.add(new TermQuery(new Term(SearchableFields.FlowFileUUID.getSearchableFieldName(), flowFileUuid)), Occur.SHOULD); + } + lineageQuery.setMinimumNumberShouldMatch(1); + } + + return lineageQuery; + } + + @Override + public QuerySubmission submitQuery(final Query query, final EventAuthorizer authorizer, final String userId) { + validate(query); + + // Check if we have any cached queries first that can give us the answer + for (final CachedQuery cachedQuery : cachedQueries) { + final Optional> eventIdListOption = cachedQuery.evaluate(query); + if (eventIdListOption.isPresent()) { + final AsyncQuerySubmission submission = new AsyncQuerySubmission(query, 1, userId); + querySubmissionMap.put(query.getIdentifier(), submission); + + final List eventIds = eventIdListOption.get(); + + queryExecutor.submit(() -> { + List events; + try { + events = eventStore.getEvents(eventIds, authorizer, EventTransformer.EMPTY_TRANSFORMER); + submission.getResult().update(events, eventIds.size()); + } catch (final Exception e) { + submission.getResult().setError("Failed to retrieve Provenance Events from store; see logs for more details"); + logger.error("Failed to retrieve Provenance Events from store", e); + } + }); + + // There are some queries that are optimized and will complete very quickly. As a result, + // we don't want to wait for the client to issue a second request, so we will give the query + // up to 500 milliseconds to complete before running. + try { + submission.getResult().awaitCompletion(500, TimeUnit.MILLISECONDS); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + + return submission; + } + } + + final List indexDirectories = directoryManager.getDirectories( + query.getStartDate() == null ? null : query.getStartDate().getTime(), + query.getEndDate() == null ? null : query.getEndDate().getTime()); + + final AsyncQuerySubmission submission = new AsyncQuerySubmission(query, indexDirectories.size(), userId); + querySubmissionMap.put(query.getIdentifier(), submission); + + final org.apache.lucene.search.Query luceneQuery = LuceneUtil.convertQuery(query); + logger.debug("Submitting query {} with identifier {} against index directories {}", luceneQuery, query.getIdentifier(), indexDirectories); + + if (indexDirectories.isEmpty()) { + submission.getResult().update(Collections.emptyList(), 0L); + } else { + Collections.sort(indexDirectories, DirectoryUtils.NEWEST_INDEX_FIRST); + + for (final File indexDir : indexDirectories) { + queryExecutor.submit(new QueryTask(luceneQuery, submission.getResult(), query.getMaxResults(), indexManager, indexDir, + eventStore, authorizer, EventTransformer.EMPTY_TRANSFORMER)); + } + } + + // There are some queries that are optimized and will complete very quickly. As a result, + // we don't want to wait for the client to issue a second request, so we will give the query + // up to 500 milliseconds to complete before running. + try { + submission.getResult().awaitCompletion(500, TimeUnit.MILLISECONDS); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + + return submission; + } + + + @Override + public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid, final NiFiUser user, final EventAuthorizer eventAuthorizer) { + return submitLineageComputation(Collections.singleton(flowFileUuid), user, eventAuthorizer, LineageComputationType.FLOWFILE_LINEAGE, null, 0L, Long.MAX_VALUE); + } + + @Override + public ComputeLineageSubmission submitExpandChildren(final long eventId, final NiFiUser user, final EventAuthorizer authorizer) { + final String userId = user.getIdentity(); + + try { + final Optional eventOption = eventStore.getEvent(eventId); + if (!eventOption.isPresent()) { + final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.emptyList(), 1, userId); + lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); + submission.getResult().update(Collections.emptyList(), 0L); + return submission; + } + + final ProvenanceEventRecord event = eventOption.get(); + switch (event.getEventType()) { + case CLONE: + case FORK: + case JOIN: + case REPLAY: { + return submitLineageComputation(event.getChildUuids(), user, authorizer, LineageComputationType.EXPAND_CHILDREN, + eventId, event.getEventTime(), Long.MAX_VALUE); + } + default: { + final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, + eventId, Collections. emptyList(), 1, userId); + + lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); + submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its children cannot be expanded"); + return submission; + } + } + } catch (final Exception e) { + final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, + eventId, Collections. emptyList(), 1, userId); + lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); + submission.getResult().setError("Failed to expand children for lineage of event with ID " + eventId + " due to: " + e); + return submission; + } + } + + @Override + public ComputeLineageSubmission submitExpandParents(final long eventId, final NiFiUser user, final EventAuthorizer authorizer) { + final String userId = user.getIdentity(); + + try { + final Optional eventOption = eventStore.getEvent(eventId); + if (!eventOption.isPresent()) { + final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.emptyList(), 1, userId); + lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); + submission.getResult().update(Collections.emptyList(), 0L); + return submission; + } + + final ProvenanceEventRecord event = eventOption.get(); + switch (event.getEventType()) { + case JOIN: + case FORK: + case CLONE: + case REPLAY: { + return submitLineageComputation(event.getParentUuids(), user, authorizer, LineageComputationType.EXPAND_PARENTS, + eventId, event.getLineageStartDate(), event.getEventTime()); + } + default: { + final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, + eventId, Collections. emptyList(), 1, userId); + + lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); + submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its parents cannot be expanded"); + return submission; + } + } + } catch (final Exception e) { + final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, + eventId, Collections. emptyList(), 1, userId); + lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); + + submission.getResult().setError("Failed to expand parents for lineage of event with ID " + eventId + " due to: " + e); + return submission; + } + } + + @Override + public AsyncLineageSubmission retrieveLineageSubmission(final String lineageIdentifier, final NiFiUser user) { + final AsyncLineageSubmission submission = lineageSubmissionMap.get(lineageIdentifier); + final String userId = submission.getSubmitterIdentity(); + + if (user == null && userId == null) { + return submission; + } + + if (user == null) { + throw new AccessDeniedException("Cannot retrieve Provenance Lineage Submission because no user id was provided"); + } + + if (userId == null || userId.equals(user.getIdentity())) { + return submission; + } + + throw new AccessDeniedException("Cannot retrieve Provenance Lineage Submission because " + user.getIdentity() + " is not the user who submitted the request"); + } + + @Override + public QuerySubmission retrieveQuerySubmission(final String queryIdentifier, final NiFiUser user) { + final QuerySubmission submission = querySubmissionMap.get(queryIdentifier); + + final String userId = submission.getSubmitterIdentity(); + + if (user == null && userId == null) { + return submission; + } + + if (user == null) { + throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because no user id was provided"); + } + + if (userId == null || userId.equals(user.getIdentity())) { + return submission; + } + + throw new AccessDeniedException("Cannot retrieve Provenance Query Submission because " + user.getIdentity() + " is not the user who submitted the request"); + } + + @Override + public long getSize() { + long total = 0; + for (final File file : directoryManager.getDirectories(null, null)) { + total += DirectoryUtils.getSize(file); + } + return total; + } + + private void validate(final Query query) { + final int numQueries = querySubmissionMap.size(); + if (numQueries > MAX_UNDELETED_QUERY_RESULTS) { + throw new IllegalStateException("Cannot process query because there are currently " + numQueries + " queries whose results have not " + + "been deleted 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"); + } + } + + void performMaintenance() { + try { + final List firstEvents = eventStore.getEvents(0, 1); + if (firstEvents.isEmpty()) { + return; + } + + final ProvenanceEventRecord firstEvent = firstEvents.get(0); + final long earliestEventTime = firstEvent.getEventTime(); + logger.debug("First Event Time is {} ({}) with Event ID {}; will delete any Lucene Index that is older than this", + earliestEventTime, new Date(earliestEventTime), firstEvent.getEventId()); + final List indicesBeforeEarliestEvent = directoryManager.getDirectoriesBefore(earliestEventTime); + + for (final File index : indicesBeforeEarliestEvent) { + logger.debug("Index directory {} is now expired. Attempting to remove index", index); + tryDeleteIndex(index); + } + } catch (final Exception e) { + logger.error("Failed to perform background maintenance procedures", e); + eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to perform maintenance of Provenance Repository. See logs for more information."); + } + } + + protected boolean tryDeleteIndex(final File indexDirectory) { + final long startNanos = System.nanoTime(); + boolean removed = false; + while (!removed && System.nanoTime() - startNanos < TimeUnit.SECONDS.toNanos(MAX_DELETE_INDEX_WAIT_SECONDS)) { + removed = indexManager.removeIndex(indexDirectory); + + if (!removed) { + try { + Thread.sleep(5000L); + } catch (final InterruptedException ie) { + logger.debug("Interrupted when trying to remove index {} from IndexManager; will not remove index", indexDirectory); + Thread.currentThread().interrupt(); + return false; + } + } + } + + if (removed) { + try { + FileUtils.deleteFile(indexDirectory, true); + logger.debug("Successfully deleted directory {}", indexDirectory); + } catch (final IOException e) { + logger.warn("The Lucene Index located at " + indexDirectory + " has expired and contains no Provenance Events that still exist in the respository. " + + "However, the directory could not be deleted.", e); + } + + directoryManager.deleteDirectory(indexDirectory); + logger.info("Successfully removed expired Lucene Index {}", indexDirectory); + } else { + logger.warn("The Lucene Index located at {} has expired and contains no Provenance Events that still exist in the respository. " + + "However, the directory could not be deleted because it is still actively being used. Will continue to try to delete " + + "in a subsequent maintenance cycle.", indexDirectory); + } + + return removed; + } + + private class RemoveExpiredQueryResults implements Runnable { + @Override + public void run() { + try { + final Date now = new Date(); + + final Iterator> queryIterator = querySubmissionMap.entrySet().iterator(); + while (queryIterator.hasNext()) { + final Map.Entry entry = queryIterator.next(); + + final StandardQueryResult result = entry.getValue().getResult(); + if (entry.getValue().isCanceled() || result.isFinished() && result.getExpiration().before(now)) { + queryIterator.remove(); + } + } + + final Iterator> lineageIterator = lineageSubmissionMap.entrySet().iterator(); + while (lineageIterator.hasNext()) { + final Map.Entry entry = lineageIterator.next(); + + final StandardLineageResult result = entry.getValue().getResult(); + if (entry.getValue().isCanceled() || result.isFinished() && result.getExpiration().before(now)) { + lineageIterator.remove(); + } + } + } catch (final Exception e) { + logger.error("Failed to expire Provenance Query Results due to {}", e.toString()); + logger.error("", e); + } + } + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/QueryTask.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/QueryTask.java new file mode 100644 index 0000000000..38d3f618f2 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/QueryTask.java @@ -0,0 +1,208 @@ +/* + * 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.index.lucene; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TopDocs; +import org.apache.nifi.provenance.ProgressiveResult; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.EventTransformer; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.SearchFailedException; +import org.apache.nifi.provenance.lucene.IndexManager; +import org.apache.nifi.provenance.store.EventStore; +import org.apache.nifi.util.Tuple; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class QueryTask implements Runnable { + private static final Logger logger = LoggerFactory.getLogger(QueryTask.class); + private static final Set LUCENE_FIELDS_TO_LOAD = Collections.singleton(SearchableFields.Identifier.getSearchableFieldName()); + + private final Query query; + private final ProgressiveResult queryResult; + private final int maxResults; + private final IndexManager indexManager; + private final File indexDir; + private final EventStore eventStore; + private final EventAuthorizer authorizer; + private final EventTransformer transformer; + + public QueryTask(final Query query, final ProgressiveResult result, final int maxResults, final IndexManager indexManager, + final File indexDir, final EventStore eventStore, final EventAuthorizer authorizer, + final EventTransformer unauthorizedTransformer) { + this.query = query; + this.queryResult = result; + this.maxResults = maxResults; + this.indexManager = indexManager; + this.indexDir = indexDir; + this.eventStore = eventStore; + this.authorizer = authorizer; + this.transformer = unauthorizedTransformer; + } + + @Override + public void run() { + if (queryResult.getTotalHitCount() >= maxResults) { + logger.debug("Will not query lucene index {} because maximum results have already been obtained", indexDir); + queryResult.update(Collections.emptyList(), 0L); + return; + } + + if (queryResult.isFinished()) { + logger.debug("Will not query lucene index {} because the query is already finished", indexDir); + return; + } + + + final long borrowStart = System.nanoTime(); + final EventIndexSearcher searcher; + try { + searcher = indexManager.borrowIndexSearcher(indexDir); + } catch (final FileNotFoundException fnfe) { + // We do not consider this an error because it may well just be the case that the event index has aged off and + // been deleted or that we've just created the index and haven't yet committed the writer. So instead, we just + // update the result ot indicate that this index search is complete with no results. + queryResult.update(Collections.emptyList(), 0); + + // nothing has been indexed yet, or the data has already aged off + logger.info("Attempted to search Provenance Index {} but could not find the directory or the directory did not contain a valid Lucene index. " + + "This usually indicates that either the index was just created and hasn't fully been initialized, or that the index was recently aged off.", indexDir); + return; + } catch (final IOException ioe) { + queryResult.setError("Failed to query index " + indexDir + "; see logs for more details"); + logger.error("Failed to query index " + indexDir, ioe); + return; + } + + try { + final long borrowMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - borrowStart); + logger.debug("Borrowing index searcher for {} took {} ms", indexDir, borrowMillis); + final long startNanos = System.nanoTime(); + + // If max number of results are retrieved, do not bother querying lucene + if (queryResult.getTotalHitCount() >= maxResults) { + logger.debug("Will not query lucene index {} because maximum results have already been obtained", indexDir); + queryResult.update(Collections.emptyList(), 0L); + return; + } + + if (queryResult.isFinished()) { + logger.debug("Will not query lucene index {} because the query is already finished", indexDir); + return; + } + + // Query lucene + final IndexReader indexReader = searcher.getIndexSearcher().getIndexReader(); + final TopDocs topDocs; + try { + topDocs = searcher.getIndexSearcher().search(query, maxResults); + } catch (final Exception e) { + logger.error("Failed to query Lucene for index " + indexDir, e); + queryResult.setError("Failed to query Lucene for index " + indexDir + " due to " + e); + return; + } finally { + final long ms = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + logger.debug("Querying Lucene for index {} took {} ms", indexDir, ms); + } + + // If max number of results are retrieved, do not bother reading docs + if (queryResult.getTotalHitCount() >= maxResults) { + logger.debug("Will not read events from store for {} because maximum results have already been obtained", indexDir); + queryResult.update(Collections.emptyList(), 0L); + return; + } + + if (queryResult.isFinished()) { + logger.debug("Will not read events from store for {} because the query has already finished", indexDir); + return; + } + + final Tuple, Integer> eventsAndTotalHits = readDocuments(topDocs, indexReader); + + if (eventsAndTotalHits == null) { + queryResult.update(Collections.emptyList(), 0L); + logger.info("Will not update query results for queried index {} for query {} because the maximum number of results have been reached already", + indexDir, query); + } else { + queryResult.update(eventsAndTotalHits.getKey(), eventsAndTotalHits.getValue()); + + final long searchNanos = System.nanoTime() - startNanos; + final long millis = TimeUnit.NANOSECONDS.toMillis(searchNanos); + logger.info("Successfully queried index {} for query {}; retrieved {} events with a total of {} hits in {} millis", + indexDir, query, eventsAndTotalHits.getKey().size(), eventsAndTotalHits.getValue(), millis); + } + } catch (final Exception e) { + logger.error("Failed to query events against index " + indexDir, e); + queryResult.setError("Failed to complete query due to " + e); + } finally { + indexManager.returnIndexSearcher(searcher); + } + } + + private Tuple, Integer> readDocuments(final TopDocs topDocs, final IndexReader indexReader) { + // If no topDocs is supplied, just provide a Tuple that has no records and a hit count of 0. + if (topDocs == null || topDocs.totalHits == 0) { + return new Tuple<>(Collections. emptyList(), 0); + } + + final long start = System.nanoTime(); + final List eventIds = Arrays.stream(topDocs.scoreDocs) + .mapToInt(scoreDoc -> scoreDoc.doc) + .mapToObj(docId -> { + try { + return indexReader.document(docId, LUCENE_FIELDS_TO_LOAD); + } catch (final Exception e) { + throw new SearchFailedException("Failed to read Provenance Events from Event File", e); + } + }) + .map(doc -> doc.getField(SearchableFields.Identifier.getSearchableFieldName()).numericValue().longValue()) + .collect(Collectors.toList()); + + final long endConvert = System.nanoTime(); + final long ms = TimeUnit.NANOSECONDS.toMillis(endConvert - start); + logger.debug("Converting documents took {} ms", ms); + + List events; + try { + events = eventStore.getEvents(eventIds, authorizer, transformer); + } catch (IOException e) { + throw new SearchFailedException("Unable to retrieve events from the Provenance Store", e); + } + + final long fetchEventNanos = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - endConvert); + logger.debug("Fetching {} events from Event Store took {} ms ({} events actually fetched)", eventIds.size(), fetchEventNanos, events.size()); + + final int totalHits = topDocs.totalHits; + return new Tuple<>(events, totalHits); + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/StoredDocument.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/StoredDocument.java new file mode 100644 index 0000000000..207ba9f983 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/StoredDocument.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.index.lucene; + +import org.apache.lucene.document.Document; +import org.apache.nifi.provenance.serialization.StorageSummary; + +public class StoredDocument { + private final Document document; + private final StorageSummary storageSummary; + + public StoredDocument(final Document document, final StorageSummary summary) { + this.document = document; + this.storageSummary = summary; + } + + public Document getDocument() { + return document; + } + + public StorageSummary getStorageSummary() { + return storageSummary; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/CachingIndexManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/CachingIndexManager.java index ddfa0db67a..eefcecb3ba 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/CachingIndexManager.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/CachingIndexManager.java @@ -36,6 +36,8 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.EventIndexWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,53 +49,61 @@ public class CachingIndexManager implements Closeable, IndexManager { private final Map> activeSearchers = new HashMap<>(); - public void removeIndex(final File indexDirectory) { + @Override + public boolean removeIndex(final File indexDirectory) { final File absoluteFile = indexDirectory.getAbsoluteFile(); logger.info("Removing index {}", indexDirectory); lock.lock(); try { final IndexWriterCount count = writerCounts.remove(absoluteFile); - if ( count != null ) { + if (count != null) { try { count.close(); } catch (final IOException ioe) { logger.warn("Failed to close Index Writer {} for {}", count.getWriter(), absoluteFile); - if ( logger.isDebugEnabled() ) { + if (logger.isDebugEnabled()) { logger.warn("", ioe); } + + return false; } } final List searcherList = activeSearchers.remove(absoluteFile); if (searcherList != null) { - for ( final ActiveIndexSearcher searcher : searcherList ) { + for (final ActiveIndexSearcher searcher : searcherList) { try { searcher.close(); } catch (final IOException ioe) { logger.warn("Failed to close Index Searcher {} for {} due to {}", - searcher.getSearcher(), absoluteFile, ioe); - if ( logger.isDebugEnabled() ) { + searcher.getSearcher(), absoluteFile, ioe); + if (logger.isDebugEnabled()) { logger.warn("", ioe); } + + return false; } } } } finally { lock.unlock(); } + + return true; } - public IndexWriter borrowIndexWriter(final File indexingDirectory) throws IOException { - final File absoluteFile = indexingDirectory.getAbsoluteFile(); - logger.trace("Borrowing index writer for {}", indexingDirectory); + @Override + public EventIndexWriter borrowIndexWriter(final File indexDirectory) throws IOException { + final File absoluteFile = indexDirectory.getAbsoluteFile(); + logger.trace("Borrowing index writer for {}", indexDirectory); lock.lock(); try { IndexWriterCount writerCount = writerCounts.remove(absoluteFile); - if ( writerCount == null ) { + if (writerCount == null) { final List closeables = new ArrayList<>(); - final Directory directory = FSDirectory.open(indexingDirectory); + final Directory directory = FSDirectory.open(indexDirectory); closeables.add(directory); try { @@ -104,10 +114,11 @@ public class CachingIndexManager implements Closeable, IndexManager { config.setWriteLockTimeout(300000L); final IndexWriter indexWriter = new IndexWriter(directory, config); - writerCount = new IndexWriterCount(indexWriter, analyzer, directory, 1); - logger.debug("Providing new index writer for {}", indexingDirectory); + final EventIndexWriter eventIndexWriter = new LuceneEventIndexWriter(indexWriter, indexDirectory); + writerCount = new IndexWriterCount(eventIndexWriter, analyzer, directory, 1); + logger.debug("Providing new index writer for {}", indexDirectory); } catch (final IOException ioe) { - for ( final Closeable closeable : closeables ) { + for (final Closeable closeable : closeables) { try { closeable.close(); } catch (final IOException ioe2) { @@ -122,16 +133,16 @@ public class CachingIndexManager implements Closeable, IndexManager { // Mark any active searchers as poisoned because we are updating the index final List searchers = activeSearchers.get(absoluteFile); - if ( searchers != null ) { + if (searchers != null) { for (final ActiveIndexSearcher activeSearcher : searchers) { - logger.debug("Poisoning {} because it is searching {}, which is getting updated", activeSearcher, indexingDirectory); + logger.debug("Poisoning {} because it is searching {}, which is getting updated", activeSearcher, indexDirectory); activeSearcher.poison(); } } } else { - logger.debug("Providing existing index writer for {} and incrementing count to {}", indexingDirectory, writerCount.getCount() + 1); + logger.debug("Providing existing index writer for {} and incrementing count to {}", indexDirectory, writerCount.getCount() + 1); writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(), - writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1)); + writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1)); } return writerCount.getWriter(); @@ -140,31 +151,53 @@ public class CachingIndexManager implements Closeable, IndexManager { } } - public void returnIndexWriter(final File indexingDirectory, final IndexWriter writer) { - final File absoluteFile = indexingDirectory.getAbsoluteFile(); - logger.trace("Returning Index Writer for {} to IndexManager", indexingDirectory); + + @Override + public void returnIndexWriter(final EventIndexWriter writer) { + returnIndexWriter(writer, true, true); + } + + @Override + public void returnIndexWriter(final EventIndexWriter writer, final boolean commit, final boolean isCloseable) { + final File indexDirectory = writer.getDirectory(); + final File absoluteFile = indexDirectory.getAbsoluteFile(); + logger.trace("Returning Index Writer for {} to IndexManager", indexDirectory); lock.lock(); try { - final IndexWriterCount count = writerCounts.remove(absoluteFile); + final IndexWriterCount count = writerCounts.get(absoluteFile); try { - if ( count == null ) { + if (count == null) { logger.warn("Index Writer {} was returned to IndexManager for {}, but this writer is not known. " - + "This could potentially lead to a resource leak", writer, indexingDirectory); + + "This could potentially lead to a resource leak", writer, indexDirectory); writer.close(); - } else if ( count.getCount() <= 1 ) { + } else if (count.getCount() <= 1) { // we are finished with this writer. - logger.debug("Decrementing count for Index Writer for {} to {}; Closing writer", indexingDirectory, count.getCount() - 1); - count.close(); + logger.info("Decrementing count for Index Writer for {} to {}. Now finished writing to this Index Directory", + indexDirectory, count.getCount() - 1); + + try { + if (commit) { + writer.commit(); + } + } finally { + if (isCloseable) { + try { + count.close(); + } finally { + writerCounts.remove(absoluteFile); + } + } + } } else { // decrement the count. - logger.debug("Decrementing count for Index Writer for {} to {}", indexingDirectory, count.getCount() - 1); + logger.debug("Decrementing count for Index Writer for {} to {}", indexDirectory, count.getCount() - 1); writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), count.getDirectory(), count.getCount() - 1)); } } catch (final IOException ioe) { logger.warn("Failed to close Index Writer {} due to {}", writer, ioe); - if ( logger.isDebugEnabled() ) { + if (logger.isDebugEnabled()) { logger.warn("", ioe); } } @@ -174,7 +207,8 @@ public class CachingIndexManager implements Closeable, IndexManager { } - public IndexSearcher borrowIndexSearcher(final File indexDir) throws IOException { + @Override + public EventIndexSearcher borrowIndexSearcher(final File indexDir) throws IOException { final File absoluteFile = indexDir.getAbsoluteFile(); logger.trace("Borrowing index searcher for {}", indexDir); @@ -182,7 +216,7 @@ public class CachingIndexManager implements Closeable, IndexManager { try { // check if we already have a reader cached. List currentlyCached = activeSearchers.get(absoluteFile); - if ( currentlyCached == null ) { + if (currentlyCached == null) { currentlyCached = new ArrayList<>(); activeSearchers.put(absoluteFile, currentlyCached); } else { @@ -197,7 +231,7 @@ public class CachingIndexManager implements Closeable, IndexManager { // if there are no references to the reader, it will have been closed. Since there is no // isClosed() method, this is how we determine whether it's been closed or not. - final int refCount = searcher.getSearcher().getIndexReader().getRefCount(); + final int refCount = searcher.getSearcher().getIndexSearcher().getIndexReader().getRefCount(); if (refCount <= 0) { // if refCount == 0, then the reader has been closed, so we cannot use the searcher logger.debug("Reference count for cached Index Searcher for {} is currently {}; " @@ -216,16 +250,17 @@ public class CachingIndexManager implements Closeable, IndexManager { // if we have an Index Writer, and if so create a Reader based on the Index Writer. // This will provide us a 'near real time' index reader. final IndexWriterCount writerCount = writerCounts.remove(absoluteFile); - if ( writerCount == null ) { + if (writerCount == null) { final Directory directory = FSDirectory.open(absoluteFile); logger.debug("No Index Writer currently exists for {}; creating a cachable reader", indexDir); try { final DirectoryReader directoryReader = DirectoryReader.open(directory); final IndexSearcher searcher = new IndexSearcher(directoryReader); + final EventIndexSearcher eventIndexSearcher = new LuceneEventIndexSearcher(searcher, indexDir, directory, directoryReader); // we want to cache the searcher that we create, since it's just a reader. - final ActiveIndexSearcher cached = new ActiveIndexSearcher(searcher, absoluteFile, directoryReader, directory, true); + final ActiveIndexSearcher cached = new ActiveIndexSearcher(eventIndexSearcher, absoluteFile, directoryReader, directory, true); currentlyCached.add(cached); return cached.getSearcher(); @@ -243,22 +278,23 @@ public class CachingIndexManager implements Closeable, IndexManager { } } else { logger.debug("Index Writer currently exists for {}; creating a non-cachable reader and incrementing " - + "counter to {}", indexDir, writerCount.getCount() + 1); + + "counter to {}", indexDir, writerCount.getCount() + 1); // increment the writer count to ensure that it's kept open. writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(), - writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1)); + writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1)); // create a new Index Searcher from the writer so that we don't have an issue with trying // to read from a directory that's locked. If we get the "no segments* file found" with // Lucene, this indicates that an IndexWriter already has the directory open. - final IndexWriter writer = writerCount.getWriter(); - final DirectoryReader directoryReader = DirectoryReader.open(writer, false); + final EventIndexWriter writer = writerCount.getWriter(); + final DirectoryReader directoryReader = DirectoryReader.open(writer.getIndexWriter(), false); final IndexSearcher searcher = new IndexSearcher(directoryReader); + final EventIndexSearcher eventIndexSearcher = new LuceneEventIndexSearcher(searcher, indexDir, null, directoryReader); // we don't want to cache this searcher because it's based on a writer, so we want to get // new values the next time that we search. - final ActiveIndexSearcher activeSearcher = new ActiveIndexSearcher(searcher, absoluteFile, directoryReader, null, false); + final ActiveIndexSearcher activeSearcher = new ActiveIndexSearcher(eventIndexSearcher, absoluteFile, directoryReader, null, false); currentlyCached.add(activeSearcher); return activeSearcher.getSearcher(); @@ -269,7 +305,9 @@ public class CachingIndexManager implements Closeable, IndexManager { } - public void returnIndexSearcher(final File indexDirectory, final IndexSearcher searcher) { + @Override + public void returnIndexSearcher(final EventIndexSearcher searcher) { + final File indexDirectory = searcher.getIndexDirectory(); final File absoluteFile = indexDirectory.getAbsoluteFile(); logger.trace("Returning index searcher for {} to IndexManager", indexDirectory); @@ -277,9 +315,9 @@ public class CachingIndexManager implements Closeable, IndexManager { try { // check if we already have a reader cached. final List currentlyCached = activeSearchers.get(absoluteFile); - if ( currentlyCached == null ) { + if (currentlyCached == null) { logger.warn("Received Index Searcher for {} but no searcher was provided for that directory; this could " - + "result in a resource leak", indexDirectory); + + "result in a resource leak", indexDirectory); return; } @@ -289,20 +327,20 @@ public class CachingIndexManager implements Closeable, IndexManager { boolean activeSearcherFound = false; while (itr.hasNext()) { final ActiveIndexSearcher activeSearcher = itr.next(); - if ( activeSearcher.getSearcher().equals(searcher) ) { + if (activeSearcher.getSearcher().equals(searcher)) { activeSearcherFound = true; - if ( activeSearcher.isCache() ) { + if (activeSearcher.isCache()) { // if the searcher is poisoned, close it and remove from "pool". Otherwise, // just decrement the count. Note here that when we call close() it won't actually close // the underlying directory reader unless there are no more references to it - if ( activeSearcher.isPoisoned() ) { + if (activeSearcher.isPoisoned()) { itr.remove(); try { activeSearcher.close(); } catch (final IOException ioe) { logger.warn("Failed to close Index Searcher for {} due to {}", absoluteFile, ioe); - if ( logger.isDebugEnabled() ) { + if (logger.isDebugEnabled()) { logger.warn("", ioe); } } @@ -322,26 +360,26 @@ public class CachingIndexManager implements Closeable, IndexManager { // decrement the writer count because we incremented it when creating the searcher final IndexWriterCount writerCount = writerCounts.remove(absoluteFile); - if ( writerCount != null ) { - if ( writerCount.getCount() <= 1 ) { + if (writerCount != null) { + if (writerCount.getCount() <= 1) { try { logger.debug("Index searcher for {} is not cached. Writer count is " - + "decremented to {}; closing writer", indexDirectory, writerCount.getCount() - 1); + + "decremented to {}; closing writer", indexDirectory, writerCount.getCount() - 1); writerCount.close(); } catch (final IOException ioe) { logger.warn("Failed to close Index Writer for {} due to {}", absoluteFile, ioe); - if ( logger.isDebugEnabled() ) { + if (logger.isDebugEnabled()) { logger.warn("", ioe); } } } else { logger.debug("Index searcher for {} is not cached. Writer count is decremented " - + "to {}; leaving writer open", indexDirectory, writerCount.getCount() - 1); + + "to {}; leaving writer open", indexDirectory, writerCount.getCount() - 1); writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(), - writerCount.getAnalyzer(), writerCount.getDirectory(), - writerCount.getCount() - 1)); + writerCount.getAnalyzer(), writerCount.getDirectory(), + writerCount.getCount() - 1)); } } @@ -353,7 +391,7 @@ public class CachingIndexManager implements Closeable, IndexManager { } } catch (final IOException ioe) { logger.warn("Failed to close Index Searcher for {} due to {}", absoluteFile, ioe); - if ( logger.isDebugEnabled() ) { + if (logger.isDebugEnabled()) { logger.warn("", ioe); } } @@ -378,11 +416,11 @@ public class CachingIndexManager implements Closeable, IndexManager { try { IOException ioe = null; - for ( final IndexWriterCount count : writerCounts.values() ) { + for (final IndexWriterCount count : writerCounts.values()) { try { count.close(); } catch (final IOException e) { - if ( ioe == null ) { + if (ioe == null) { ioe = e; } else { ioe.addSuppressed(e); @@ -395,7 +433,7 @@ public class CachingIndexManager implements Closeable, IndexManager { try { searcher.close(); } catch (final IOException e) { - if ( ioe == null ) { + if (ioe == null) { ioe = e; } else { ioe.addSuppressed(e); @@ -404,7 +442,7 @@ public class CachingIndexManager implements Closeable, IndexManager { } } - if ( ioe != null ) { + if (ioe != null) { throw ioe; } } finally { @@ -415,15 +453,15 @@ public class CachingIndexManager implements Closeable, IndexManager { private static void close(final Closeable... closeables) throws IOException { IOException ioe = null; - for ( final Closeable closeable : closeables ) { - if ( closeable == null ) { + for (final Closeable closeable : closeables) { + if (closeable == null) { continue; } try { closeable.close(); } catch (final IOException e) { - if ( ioe == null ) { + if (ioe == null) { ioe = e; } else { ioe.addSuppressed(e); @@ -431,14 +469,14 @@ public class CachingIndexManager implements Closeable, IndexManager { } } - if ( ioe != null ) { + if (ioe != null) { throw ioe; } } private static class ActiveIndexSearcher { - private final IndexSearcher searcher; + private final EventIndexSearcher searcher; private final DirectoryReader directoryReader; private final File indexDirectory; private final Directory directory; @@ -446,8 +484,8 @@ public class CachingIndexManager implements Closeable, IndexManager { private final AtomicInteger referenceCount = new AtomicInteger(1); private volatile boolean poisoned = false; - public ActiveIndexSearcher(final IndexSearcher searcher, final File indexDirectory, final DirectoryReader directoryReader, - final Directory directory, final boolean cache) { + public ActiveIndexSearcher(final EventIndexSearcher searcher, final File indexDirectory, final DirectoryReader directoryReader, + final Directory directory, final boolean cache) { this.searcher = searcher; this.directoryReader = directoryReader; this.indexDirectory = indexDirectory; @@ -459,7 +497,7 @@ public class CachingIndexManager implements Closeable, IndexManager { return cache; } - public IndexSearcher getSearcher() { + public EventIndexSearcher getSearcher() { return searcher; } @@ -499,12 +537,12 @@ public class CachingIndexManager implements Closeable, IndexManager { private static class IndexWriterCount implements Closeable { - private final IndexWriter writer; + private final EventIndexWriter writer; private final Analyzer analyzer; private final Directory directory; private final int count; - public IndexWriterCount(final IndexWriter writer, final Analyzer analyzer, final Directory directory, final int count) { + public IndexWriterCount(final EventIndexWriter writer, final Analyzer analyzer, final Directory directory, final int count) { this.writer = writer; this.analyzer = analyzer; this.directory = directory; @@ -519,7 +557,7 @@ public class CachingIndexManager implements Closeable, IndexManager { return directory; } - public IndexWriter getWriter() { + public EventIndexWriter getWriter() { return writer; } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java index 7707352064..f372a2d3a5 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DeleteIndexAction.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.Term; import org.apache.nifi.provenance.IndexConfiguration; import org.apache.nifi.provenance.PersistentProvenanceRepository; import org.apache.nifi.provenance.expiration.ExpirationAction; +import org.apache.nifi.provenance.index.EventIndexWriter; import org.apache.nifi.provenance.serialization.RecordReader; import org.apache.nifi.provenance.serialization.RecordReaders; import org.slf4j.Logger; @@ -60,15 +61,16 @@ public class DeleteIndexAction implements ExpirationAction { final Term term = new Term(FieldNames.STORAGE_FILENAME, LuceneUtil.substringBefore(expiredFile.getName(), ".")); boolean deleteDir = false; - final IndexWriter writer = indexManager.borrowIndexWriter(indexingDirectory); + final EventIndexWriter writer = indexManager.borrowIndexWriter(indexingDirectory); try { - writer.deleteDocuments(term); - writer.commit(); - final int docsLeft = writer.numDocs(); + final IndexWriter indexWriter = writer.getIndexWriter(); + indexWriter.deleteDocuments(term); + indexWriter.commit(); + final int docsLeft = indexWriter.numDocs(); deleteDir = docsLeft <= 0; logger.debug("After expiring {}, there are {} docs left for index {}", expiredFile, docsLeft, indexingDirectory); } finally { - indexManager.returnIndexWriter(indexingDirectory, writer); + indexManager.returnIndexWriter(writer); } // we've confirmed that all documents have been removed. Delete the index directory. diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java index ce62152506..0e96b6248f 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocsReader.java @@ -30,25 +30,25 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.nifi.provenance.ProvenanceEventRecord; -import org.apache.nifi.provenance.SearchableFields; -import org.apache.nifi.provenance.StandardProvenanceEventRecord; -import org.apache.nifi.provenance.authorization.AuthorizationCheck; -import org.apache.nifi.provenance.serialization.RecordReader; -import org.apache.nifi.provenance.serialization.RecordReaders; -import org.apache.nifi.provenance.toc.TocReader; import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.StandardProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.serialization.RecordReader; +import org.apache.nifi.provenance.serialization.RecordReaders; +import org.apache.nifi.provenance.toc.TocReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class DocsReader { +public class DocsReader { private final Logger logger = LoggerFactory.getLogger(DocsReader.class); - public Set read(final TopDocs topDocs, final AuthorizationCheck authCheck, final IndexReader indexReader, final Collection allProvenanceLogFiles, + public Set read(final TopDocs topDocs, final EventAuthorizer authorizer, final IndexReader indexReader, final Collection allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException { if (retrievalCount.get() >= maxResults) { return Collections.emptySet(); @@ -67,7 +67,7 @@ class DocsReader { final long readDocuments = System.nanoTime() - start; logger.debug("Reading {} Lucene Documents took {} millis", docs.size(), TimeUnit.NANOSECONDS.toMillis(readDocuments)); - return read(docs, authCheck, allProvenanceLogFiles, retrievalCount, maxResults, maxAttributeChars); + return read(docs, authorizer, allProvenanceLogFiles, retrievalCount, maxResults, maxAttributeChars); } @@ -106,7 +106,7 @@ class DocsReader { return record; } - public Set read(final List docs, final AuthorizationCheck authCheck, final Collection allProvenanceLogFiles, + public Set read(final List docs, final EventAuthorizer authorizer, final Collection allProvenanceLogFiles, final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException { if (retrievalCount.get() >= maxResults) { @@ -114,38 +114,33 @@ class DocsReader { } final long start = System.nanoTime(); - - Set matchingRecords = new LinkedHashSet<>(); - - Map> byStorageNameDocGroups = LuceneUtil.groupDocsByStorageFileName(docs); + final Set matchingRecords = new LinkedHashSet<>(); + final Map> byStorageNameDocGroups = LuceneUtil.groupDocsByStorageFileName(docs); int eventsReadThisFile = 0; int logFileCount = 0; for (String storageFileName : byStorageNameDocGroups.keySet()) { - File provenanceEventFile = LuceneUtil.getProvenanceLogFile(storageFileName, allProvenanceLogFiles); - if (provenanceEventFile != null) { - try (RecordReader reader = RecordReaders.newRecordReader(provenanceEventFile, allProvenanceLogFiles, - maxAttributeChars)) { - - Iterator docIter = byStorageNameDocGroups.get(storageFileName).iterator(); - while (docIter.hasNext() && retrievalCount.getAndIncrement() < maxResults) { - ProvenanceEventRecord event = this.getRecord(docIter.next(), reader); - if (event != null && authCheck.isAuthorized(event)) { - matchingRecords.add(event); - eventsReadThisFile++; - } - } - - } catch (Exception e) { - logger.warn("Failed while trying to read Provenance Events. The event file '" - + provenanceEventFile.getAbsolutePath() + - "' may be missing or corrupted.", e); - } - } else { + final File provenanceEventFile = LuceneUtil.getProvenanceLogFile(storageFileName, allProvenanceLogFiles); + if (provenanceEventFile == null) { logger.warn("Could not find Provenance Log File with " - + "basename {} in the Provenance Repository; assuming " - + "file has expired and continuing without it", storageFileName); + + "basename {} in the Provenance Repository; assuming " + + "file has expired and continuing without it", storageFileName); + continue; + } + + try (final RecordReader reader = RecordReaders.newRecordReader(provenanceEventFile, allProvenanceLogFiles, maxAttributeChars)) { + final Iterator docIter = byStorageNameDocGroups.get(storageFileName).iterator(); + while (docIter.hasNext() && retrievalCount.getAndIncrement() < maxResults) { + final ProvenanceEventRecord event = getRecord(docIter.next(), reader); + if (event != null && authorizer.isAuthorized(event)) { + matchingRecords.add(event); + eventsReadThisFile++; + } + } + } catch (final Exception e) { + logger.warn("Failed to read Provenance Events. The event file '" + + provenanceEventFile.getAbsolutePath() + "' may be missing or corrupt.", e); } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocumentToEventConverter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocumentToEventConverter.java new file mode 100644 index 0000000000..18d38604ee --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/DocumentToEventConverter.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.lucene; + +import java.io.IOException; +import java.util.Set; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.TopDocs; +import org.apache.nifi.provenance.ProvenanceEventRecord; + +public interface DocumentToEventConverter { + + Set convert(TopDocs topDocs, IndexReader indexReader) throws IOException; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java index f84021f85c..331d1410ae 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexManager.java @@ -21,17 +21,19 @@ import java.io.Closeable; import java.io.File; import java.io.IOException; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.search.IndexSearcher; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.EventIndexWriter; public interface IndexManager extends Closeable { - IndexSearcher borrowIndexSearcher(File indexDir) throws IOException; + EventIndexSearcher borrowIndexSearcher(File indexDir) throws IOException; - IndexWriter borrowIndexWriter(File indexingDirectory) throws IOException; + EventIndexWriter borrowIndexWriter(File indexDirectory) throws IOException; - void removeIndex(final File indexDirectory); + boolean removeIndex(final File indexDirectory); - void returnIndexSearcher(File indexDirectory, IndexSearcher searcher); + void returnIndexSearcher(EventIndexSearcher searcher); - void returnIndexWriter(File indexingDirectory, IndexWriter writer); + void returnIndexWriter(EventIndexWriter writer, boolean commit, boolean isCloseable); + + void returnIndexWriter(EventIndexWriter writer); } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java index 8d7df8bc98..514af38c7a 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexSearch.java @@ -21,18 +21,20 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collections; import java.util.Date; +import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.TopDocs; +import org.apache.nifi.authorization.AccessDeniedException; import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.provenance.PersistentProvenanceRepository; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.StandardQueryResult; -import org.apache.nifi.provenance.authorization.AuthorizationCheck; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.index.EventIndexSearcher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,14 +88,14 @@ public class IndexSearch { final Query luceneQuery = LuceneUtil.convertQuery(provenanceQuery); final long start = System.nanoTime(); - IndexSearcher searcher = null; + EventIndexSearcher searcher = null; try { searcher = indexManager.borrowIndexSearcher(indexDirectory); final long searchStartNanos = System.nanoTime(); final long openSearcherNanos = searchStartNanos - start; logger.debug("Searching {} for {}", this, provenanceQuery); - final TopDocs topDocs = searcher.search(luceneQuery, provenanceQuery.getMaxResults()); + final TopDocs topDocs = searcher.getIndexSearcher().search(luceneQuery, provenanceQuery.getMaxResults()); final long finishSearch = System.nanoTime(); final long searchNanos = finishSearch - searchStartNanos; @@ -107,9 +109,29 @@ public class IndexSearch { final DocsReader docsReader = new DocsReader(); - final AuthorizationCheck authCheck = event -> repository.isAuthorized(event, user); + final EventAuthorizer authorizer = new EventAuthorizer() { + @Override + public boolean isAuthorized(ProvenanceEventRecord event) { + return repository.isAuthorized(event, user); + } - matchingRecords = docsReader.read(topDocs, authCheck, searcher.getIndexReader(), repository.getAllLogFiles(), retrievedCount, + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + repository.authorize(event, user); + } + + @Override + public List filterUnauthorizedEvents(List events) { + return repository.filterUnauthorizedEvents(events, user); + } + + @Override + public Set replaceUnauthorizedWithPlaceholders(Set events) { + return repository.replaceUnauthorizedWithPlaceholders(events, user); + } + }; + + matchingRecords = docsReader.read(topDocs, authorizer, searcher.getIndexSearcher().getIndexReader(), repository.getAllLogFiles(), retrievedCount, provenanceQuery.getMaxResults(), maxAttributeChars); final long readRecordsNanos = System.nanoTime() - finishSearch; @@ -133,7 +155,7 @@ public class IndexSearch { return sqr; } finally { if ( searcher != null ) { - indexManager.returnIndexSearcher(indexDirectory, searcher); + indexManager.returnIndexSearcher(searcher); } } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java index f725208b10..a0be319ddc 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/IndexingAction.java @@ -19,6 +19,7 @@ package org.apache.nifi.provenance.lucene; import java.io.IOException; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Set; import org.apache.lucene.document.Document; @@ -28,23 +29,22 @@ import org.apache.lucene.document.LongField; import org.apache.lucene.document.StringField; import org.apache.lucene.index.IndexWriter; import org.apache.nifi.flowfile.attributes.CoreAttributes; -import org.apache.nifi.provenance.PersistentProvenanceRepository; import org.apache.nifi.provenance.ProvenanceEventType; import org.apache.nifi.provenance.SearchableFields; import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.apache.nifi.provenance.search.SearchableField; public class IndexingAction { - private final Set nonAttributeSearchableFields; - private final Set attributeSearchableFields; + private final Set searchableEventFields; + private final Set searchableAttributeFields; - public IndexingAction(final PersistentProvenanceRepository repo) { - attributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(repo.getConfiguration().getSearchableAttributes())); - nonAttributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(repo.getConfiguration().getSearchableFields())); + public IndexingAction(final List searchableEventFields, final List searchableAttributes) { + this.searchableEventFields = Collections.unmodifiableSet(new HashSet<>(searchableEventFields)); + this.searchableAttributeFields = Collections.unmodifiableSet(new HashSet<>(searchableAttributes)); } private void addField(final Document doc, final SearchableField field, final String value, final Store store) { - if (value == null || (!field.isAttribute() && !nonAttributeSearchableFields.contains(field))) { + if (value == null || (!field.isAttribute() && !searchableEventFields.contains(field))) { return; } @@ -67,7 +67,7 @@ public class IndexingAction { addField(doc, SearchableFields.SourceQueueIdentifier, record.getSourceQueueIdentifier(), Store.NO); addField(doc, SearchableFields.TransitURI, record.getTransitUri(), Store.NO); - for (final SearchableField searchableField : attributeSearchableFields) { + for (final SearchableField searchableField : searchableAttributeFields) { addField(doc, searchableField, LuceneUtil.truncateIndexField(record.getAttribute(searchableField.getSearchableFieldName())), Store.NO); } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java index 1b13504d63..2388483580 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LineageQuery.java @@ -25,18 +25,15 @@ import java.util.Collection; import java.util.Collections; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; 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.TermQuery; import org.apache.lucene.search.TopDocs; -import org.apache.nifi.provenance.PersistentProvenanceRepository; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.SearchableFields; -import org.apache.nifi.provenance.authorization.AuthorizationCheck; +import org.apache.nifi.provenance.index.EventIndexSearcher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,8 +43,8 @@ public class LineageQuery { public static final int MAX_LINEAGE_UUIDS = 100; private static final Logger logger = LoggerFactory.getLogger(LineageQuery.class); - public static Set computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final IndexManager indexManager, final File indexDirectory, - final String lineageIdentifier, final Collection flowFileUuids, final int maxAttributeChars) throws IOException { + public static Set computeLineageForFlowFiles(final IndexManager indexManager, final File indexDirectory, + final String lineageIdentifier, final Collection flowFileUuids, final DocumentToEventConverter docsToEventConverter) 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())); } @@ -56,7 +53,7 @@ public class LineageQuery { throw new IllegalArgumentException("Must specify either Lineage Identifier or FlowFile UUIDs to compute lineage"); } - final IndexSearcher searcher; + final EventIndexSearcher searcher; try { searcher = indexManager.borrowIndexSearcher(indexDirectory); try { @@ -75,16 +72,10 @@ public class LineageQuery { final long searchStart = System.nanoTime(); logger.debug("Searching {} for {}", indexDirectory, flowFileIdQuery); - final TopDocs uuidQueryTopDocs = searcher.search(flowFileIdQuery, MAX_QUERY_RESULTS); + final TopDocs uuidQueryTopDocs = searcher.getIndexSearcher().search(flowFileIdQuery, MAX_QUERY_RESULTS); final long searchEnd = System.nanoTime(); - // Always authorized. We do this because we need to pull back the event, regardless of whether or not - // the user is truly authorized, because instead of ignoring unauthorized events, we want to replace them. - final AuthorizationCheck authCheck = event -> true; - - final DocsReader docsReader = new DocsReader(); - final Set recs = docsReader.read(uuidQueryTopDocs, authCheck, searcher.getIndexReader(), repo.getAllLogFiles(), - new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars); + final Set recs = docsToEventConverter.convert(uuidQueryTopDocs, searcher.getIndexSearcher().getIndexReader()); final long readDocsEnd = System.nanoTime(); logger.debug("Finished Lineage Query against {}; Lucene search took {} millis, reading records took {} millis", @@ -92,7 +83,7 @@ public class LineageQuery { return recs; } finally { - indexManager.returnIndexSearcher(indexDirectory, searcher); + indexManager.returnIndexSearcher(searcher); } } catch (final FileNotFoundException fnfe) { // nothing has been indexed yet, or the data has already aged off diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneEventIndexSearcher.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneEventIndexSearcher.java new file mode 100644 index 0000000000..07b9167cca --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneEventIndexSearcher.java @@ -0,0 +1,92 @@ +/* + * 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.lucene; + +import java.io.Closeable; +import java.io.File; + +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.store.Directory; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class LuceneEventIndexSearcher implements EventIndexSearcher { + private static final Logger logger = LoggerFactory.getLogger(LuceneEventIndexSearcher.class); + + private final IndexSearcher indexSearcher; + private final File indexDirectory; + private final Directory directory; + private final DirectoryReader directoryReader; + + // guarded by synchronizing on 'this' + private int usageCounter = 0; + private boolean closed = false; + + public LuceneEventIndexSearcher(final IndexSearcher indexSearcher, final File indexDirectory, final Directory directory, final DirectoryReader directoryReader) { + this.indexSearcher = indexSearcher; + this.indexDirectory = indexDirectory; + this.directory = directory; + this.directoryReader = directoryReader; + } + + @Override + public IndexSearcher getIndexSearcher() { + return indexSearcher; + } + + @Override + public File getIndexDirectory() { + return indexDirectory; + } + + @Override + public synchronized void close() { + closed = true; + if (usageCounter == 0) { + closeQuietly(directoryReader); + closeQuietly(directory); + } + } + + public synchronized void incrementUsageCounter() { + usageCounter++; + } + + public synchronized void decrementUsageCounter() { + usageCounter--; + if (usageCounter == 0 && closed) { + closeQuietly(directoryReader); + closeQuietly(directory); + } + } + + private void closeQuietly(final Closeable closeable) { + if (closeable == null) { + return; + } + + try { + closeable.close(); + } catch (final Exception e) { + logger.warn("Failed to close {} due to {}", closeable, e); + } + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneEventIndexWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneEventIndexWriter.java new file mode 100644 index 0000000000..db8c5287d7 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/LuceneEventIndexWriter.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.lucene; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.IndexWriter; +import org.apache.nifi.provenance.index.EventIndexWriter; + +public class LuceneEventIndexWriter implements EventIndexWriter { + private final IndexWriter indexWriter; + private final File directory; + private final long maxCommitNanos; + + private final AtomicReference commitStats = new AtomicReference<>(); + private final AtomicLong totalIndexed = new AtomicLong(0L); + private final AtomicLong lastCommitTotalIndexed = new AtomicLong(0L); + + public LuceneEventIndexWriter(final IndexWriter indexWriter, final File directory) { + this(indexWriter, directory, TimeUnit.SECONDS.toNanos(30L)); + } + + public LuceneEventIndexWriter(final IndexWriter indexWriter, final File directory, final long maxCommitNanos) { + this.indexWriter = indexWriter; + this.directory = directory; + this.maxCommitNanos = maxCommitNanos; + + commitStats.set(new CommitStats(0, System.nanoTime() + maxCommitNanos)); + } + + @Override + public void close() throws IOException { + indexWriter.close(); + } + + @Override + public boolean index(final Document document, final int commitThreshold) throws IOException { + return index(Collections.singletonList(document), commitThreshold); + } + + @Override + public boolean index(List documents, final int commitThreshold) throws IOException { + if (documents.isEmpty()) { + return false; + } + + final int numDocs = documents.size(); + indexWriter.addDocuments(documents); + totalIndexed.addAndGet(numDocs); + + boolean updated = false; + while (!updated) { + final CommitStats stats = commitStats.get(); + CommitStats updatedStats = new CommitStats(stats.getIndexedSinceCommit() + numDocs, stats.getNextCommitTimestamp()); + + if (updatedStats.getIndexedSinceCommit() >= commitThreshold || System.nanoTime() >= updatedStats.getNextCommitTimestamp()) { + updatedStats = new CommitStats(0, System.nanoTime() + maxCommitNanos); + updated = commitStats.compareAndSet(stats, updatedStats); + if (updated) { + return true; + } + } else { + updated = commitStats.compareAndSet(stats, updatedStats); + } + } + + return false; + } + + @Override + public File getDirectory() { + return directory; + } + + @Override + public long commit() throws IOException { + final long lastCommitCount = lastCommitTotalIndexed.get(); + final long currentCommitCount = totalIndexed.get(); + indexWriter.commit(); + commitStats.set(new CommitStats(0, System.nanoTime() + maxCommitNanos)); + lastCommitTotalIndexed.set(currentCommitCount); + return currentCommitCount - lastCommitCount; + } + + @Override + public int getEventsIndexedSinceCommit() { + return commitStats.get().getIndexedSinceCommit(); + } + + @Override + public long getEventsIndexed() { + return totalIndexed.get(); + } + + @Override + public IndexWriter getIndexWriter() { + return indexWriter; + } + + @Override + public String toString() { + return "LuceneEventIndexWriter[dir=" + directory + "]"; + } + + private static class CommitStats { + private final long nextCommitTimestamp; + private final int indexedSinceCommit; + + public CommitStats(final int indexedCount, final long nextCommitTime) { + this.nextCommitTimestamp = nextCommitTime; + this.indexedSinceCommit = indexedCount; + } + + public long getNextCommitTimestamp() { + return nextCommitTimestamp; + } + + public int getIndexedSinceCommit() { + return indexedSinceCommit; + } + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java index 81816d26c0..b0b01e592b 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java @@ -24,155 +24,306 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.index.ConcurrentMergeScheduler; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.EventIndexWriter; +import org.apache.nifi.provenance.util.NamedThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SimpleIndexManager implements IndexManager { private static final Logger logger = LoggerFactory.getLogger(SimpleIndexManager.class); - private final ConcurrentMap> closeables = new ConcurrentHashMap<>(); - private final Map writerCounts = new HashMap<>(); - - private final ExecutorService searchExecutor = Executors.newCachedThreadPool(); + private final Map writerCounts = new HashMap<>(); // guarded by synchronizing on map itself + private final ExecutorService searchExecutor; + private final RepositoryConfiguration repoConfig; + public SimpleIndexManager(final RepositoryConfiguration repoConfig) { + this.repoConfig = repoConfig; + this.searchExecutor = Executors.newFixedThreadPool(repoConfig.getQueryThreadPoolSize(), new NamedThreadFactory("Search Lucene Index")); + } @Override public void close() throws IOException { logger.debug("Shutting down SimpleIndexManager search executor"); - this.searchExecutor.shutdown(); + + searchExecutor.shutdown(); try { - if (!this.searchExecutor.awaitTermination(5, TimeUnit.SECONDS)) { - this.searchExecutor.shutdownNow(); + if (!searchExecutor.awaitTermination(5, TimeUnit.SECONDS)) { + searchExecutor.shutdownNow(); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - this.searchExecutor.shutdownNow(); + searchExecutor.shutdownNow(); } } @Override - public IndexSearcher borrowIndexSearcher(final File indexDir) throws IOException { - logger.debug("Creating index searcher for {}", indexDir); - final Directory directory = FSDirectory.open(indexDir); - final DirectoryReader directoryReader = DirectoryReader.open(directory); + public EventIndexSearcher borrowIndexSearcher(final File indexDir) throws IOException { + final File absoluteFile = indexDir.getAbsoluteFile(); + + final IndexWriterCount writerCount; + synchronized (writerCounts) { + writerCount = writerCounts.remove(absoluteFile); + + if (writerCount != null) { + // Increment writer count and create an Index Searcher based on the writer + writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(), writerCount.getAnalyzer(), + writerCount.getDirectory(), writerCount.getCount() + 1, writerCount.isCloseableWhenUnused())); + } + } + + final DirectoryReader directoryReader; + if (writerCount == null) { + logger.trace("Creating index searcher for {}", indexDir); + final Directory directory = FSDirectory.open(indexDir); + directoryReader = DirectoryReader.open(directory); + } else { + final EventIndexWriter eventIndexWriter = writerCount.getWriter(); + directoryReader = DirectoryReader.open(eventIndexWriter.getIndexWriter(), false); + } + final IndexSearcher searcher = new IndexSearcher(directoryReader, this.searchExecutor); - final List closeableList = new ArrayList<>(2); - closeableList.add(directoryReader); - closeableList.add(directory); - closeables.put(searcher, closeableList); - logger.debug("Created index searcher {} for {}", searcher, indexDir); - - return searcher; + logger.trace("Created index searcher {} for {}", searcher, indexDir); + return new LuceneEventIndexSearcher(searcher, indexDir, null, directoryReader); } @Override - public void returnIndexSearcher(final File indexDirectory, final IndexSearcher searcher) { + public void returnIndexSearcher(final EventIndexSearcher searcher) { + final File indexDirectory = searcher.getIndexDirectory(); logger.debug("Closing index searcher {} for {}", searcher, indexDirectory); + closeQuietly(searcher); + logger.debug("Closed index searcher {}", searcher); - final List closeableList = closeables.get(searcher); - if (closeableList != null) { - for (final Closeable closeable : closeableList) { - closeQuietly(closeable); + final IndexWriterCount count; + boolean closeWriter = false; + synchronized (writerCounts) { + final File absoluteFile = searcher.getIndexDirectory().getAbsoluteFile(); + count = writerCounts.get(absoluteFile); + if (count == null) { + logger.debug("Returning EventIndexSearcher for {}; there is no active writer for this searcher so will not decrement writerCounts", absoluteFile); + return; + } + + if (count.getCount() <= 1) { + // we are finished with this writer. + final boolean close = count.isCloseableWhenUnused(); + logger.debug("Decrementing count for Index Writer for {} to {}{}", indexDirectory, count.getCount() - 1, close ? "; closing writer" : ""); + + if (close) { + writerCounts.remove(absoluteFile); + closeWriter = true; + } else { + writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), count.getDirectory(), + count.getCount() - 1, count.isCloseableWhenUnused())); + } + } else { + writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), count.getDirectory(), + count.getCount() - 1, count.isCloseableWhenUnused())); } } - logger.debug("Closed index searcher {}", searcher); - } - - @Override - public void removeIndex(final File indexDirectory) { - } - - - @Override - public synchronized IndexWriter borrowIndexWriter(final File indexingDirectory) throws IOException { - final File absoluteFile = indexingDirectory.getAbsoluteFile(); - logger.trace("Borrowing index writer for {}", indexingDirectory); - - IndexWriterCount writerCount = writerCounts.remove(absoluteFile); - if (writerCount == null) { - final List closeables = new ArrayList<>(); - final Directory directory = FSDirectory.open(indexingDirectory); - closeables.add(directory); - + if (closeWriter) { try { - final Analyzer analyzer = new StandardAnalyzer(); - closeables.add(analyzer); + close(count); + } catch (final Exception e) { + logger.warn("Failed to close Index Writer {} due to {}", count.getWriter(), e.toString(), e); + } + } + } - final IndexWriterConfig config = new IndexWriterConfig(LuceneUtil.LUCENE_VERSION, analyzer); - config.setWriteLockTimeout(300000L); + @Override + public boolean removeIndex(final File indexDirectory) { + final File absoluteFile = indexDirectory.getAbsoluteFile(); + logger.debug("Attempting to remove index {} from SimpleIndexManager", absoluteFile); - final IndexWriter indexWriter = new IndexWriter(directory, config); - writerCount = new IndexWriterCount(indexWriter, analyzer, directory, 1); - logger.debug("Providing new index writer for {}", indexingDirectory); - } catch (final IOException ioe) { - for (final Closeable closeable : closeables) { - try { - closeable.close(); - } catch (final IOException ioe2) { - ioe.addSuppressed(ioe2); - } - } - - throw ioe; + IndexWriterCount writerCount; + synchronized (writerCounts) { + writerCount = writerCounts.remove(absoluteFile); + if (writerCount == null) { + logger.debug("Allowing removal of index {} because there is no IndexWriterCount for this directory", absoluteFile); + return true; // return true since directory has no writers } - writerCounts.put(absoluteFile, writerCount); - } else { - logger.debug("Providing existing index writer for {} and incrementing count to {}", indexingDirectory, writerCount.getCount() + 1); - writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(), - writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1)); + if (writerCount.getCount() > 0) { + logger.debug("Not allowing removal of index {} because the active writer count for this directory is {}", absoluteFile, writerCount.getCount()); + writerCounts.put(absoluteFile, writerCount); + return false; + } + } + + try { + logger.debug("Removing index {} from SimpleIndexManager and closing the writer", absoluteFile); + + close(writerCount); + } catch (final Exception e) { + logger.error("Failed to close Index Writer for {} while removing Index from the repository;" + + "this directory may need to be cleaned up manually.", e); + } + + return true; + } + + + private IndexWriterCount createWriter(final File indexDirectory) throws IOException { + final List closeables = new ArrayList<>(); + final Directory directory = FSDirectory.open(indexDirectory); + closeables.add(directory); + + try { + final Analyzer analyzer = new StandardAnalyzer(); + closeables.add(analyzer); + + final IndexWriterConfig config = new IndexWriterConfig(LuceneUtil.LUCENE_VERSION, analyzer); + + final ConcurrentMergeScheduler mergeScheduler = new ConcurrentMergeScheduler(); + final int mergeThreads = repoConfig.getConcurrentMergeThreads(); + mergeScheduler.setMaxMergesAndThreads(mergeThreads, mergeThreads); + config.setMergeScheduler(mergeScheduler); + + final IndexWriter indexWriter = new IndexWriter(directory, config); + final EventIndexWriter eventIndexWriter = new LuceneEventIndexWriter(indexWriter, indexDirectory); + + final IndexWriterCount writerCount = new IndexWriterCount(eventIndexWriter, analyzer, directory, 1, false); + logger.debug("Providing new index writer for {}", indexDirectory); + return writerCount; + } catch (final IOException ioe) { + for (final Closeable closeable : closeables) { + try { + closeable.close(); + } catch (final IOException ioe2) { + ioe.addSuppressed(ioe2); + } + } + + throw ioe; + } + } + + @Override + public EventIndexWriter borrowIndexWriter(final File indexDirectory) throws IOException { + final File absoluteFile = indexDirectory.getAbsoluteFile(); + logger.trace("Borrowing index writer for {}", indexDirectory); + + IndexWriterCount writerCount = null; + synchronized (writerCounts) { + writerCount = writerCounts.get(absoluteFile); + + if (writerCount == null) { + writerCount = createWriter(indexDirectory); + writerCounts.put(absoluteFile, writerCount); + } else { + logger.trace("Providing existing index writer for {} and incrementing count to {}", indexDirectory, writerCount.getCount() + 1); + writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(), + writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1, writerCount.isCloseableWhenUnused())); + } + + if (writerCounts.size() > repoConfig.getStorageDirectories().size() * 2) { + logger.debug("Index Writer returned; writer count map now has size {}; writerCount = {}; full writerCounts map = {}", + writerCounts.size(), writerCount, writerCounts); + } } return writerCount.getWriter(); } + @Override + public void returnIndexWriter(final EventIndexWriter writer) { + returnIndexWriter(writer, true, true); + } @Override - public synchronized void returnIndexWriter(final File indexingDirectory, final IndexWriter writer) { - final File absoluteFile = indexingDirectory.getAbsoluteFile(); - logger.trace("Returning Index Writer for {} to IndexManager", indexingDirectory); - - final IndexWriterCount count = writerCounts.remove(absoluteFile); + public void returnIndexWriter(final EventIndexWriter writer, final boolean commit, final boolean isCloseable) { + final File indexDirectory = writer.getDirectory(); + final File absoluteFile = indexDirectory.getAbsoluteFile(); + logger.trace("Returning Index Writer for {} to IndexManager", indexDirectory); + boolean unused = false; + IndexWriterCount count = null; + boolean close = isCloseable; try { - if (count == null) { - logger.warn("Index Writer {} was returned to IndexManager for {}, but this writer is not known. " - + "This could potentially lead to a resource leak", writer, indexingDirectory); - writer.close(); - } else if (count.getCount() <= 1) { - // we are finished with this writer. - logger.debug("Decrementing count for Index Writer for {} to {}; Closing writer", indexingDirectory, count.getCount() - 1); - try { - writer.commit(); - } finally { - count.close(); + synchronized (writerCounts) { + count = writerCounts.get(absoluteFile); + if (count != null && count.isCloseableWhenUnused()) { + close = true; + } + + if (count == null) { + logger.warn("Index Writer {} was returned to IndexManager for {}, but this writer is not known. " + + "This could potentially lead to a resource leak", writer, indexDirectory); + writer.close(); + } else if (count.getCount() <= 1) { + // we are finished with this writer. + unused = true; + if (close) { + logger.debug("Decrementing count for Index Writer for {} to {}; closing writer", indexDirectory, count.getCount() - 1); + writerCounts.remove(absoluteFile); + } else { + logger.trace("Decrementing count for Index Writer for {} to {}", indexDirectory, count.getCount() - 1); + + // If writer is not closeable, then we need to decrement its count. + writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), count.getDirectory(), + count.getCount() - 1, close)); + } + } else { + // decrement the count. + if (close) { + logger.debug("Decrementing count for Index Writer for {} to {} and marking as closeable when no longer in use", indexDirectory, count.getCount() - 1); + } else { + logger.trace("Decrementing count for Index Writer for {} to {}", indexDirectory, count.getCount() - 1); + } + + writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), + count.getDirectory(), count.getCount() - 1, close)); + } + + if (writerCounts.size() > repoConfig.getStorageDirectories().size() * 2) { + logger.debug("Index Writer returned; writer count map now has size {}; writer = {}, commit = {}, isCloseable = {}, writerCount = {}; full writerCounts Map = {}", + writerCounts.size(), writer, commit, isCloseable, count, writerCounts); } - } else { - // decrement the count. - logger.debug("Decrementing count for Index Writer for {} to {}", indexingDirectory, count.getCount() - 1); - writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), count.getDirectory(), count.getCount() - 1)); } - } catch (final IOException ioe) { - logger.warn("Failed to close Index Writer {} due to {}", writer, ioe); - if (logger.isDebugEnabled()) { - logger.warn("", ioe); + + // Committing and closing are very expensive, so we want to do those outside of the synchronized block. + // So we use an 'unused' variable to tell us whether or not we should actually do so. + if (unused) { + try { + if (commit) { + writer.commit(); + } + } finally { + if (close) { + logger.info("Index Writer for {} has been returned to Index Manager and is no longer in use. Closing Index Writer", indexDirectory); + close(count); + } + } } + } catch (final Exception e) { + logger.warn("Failed to close Index Writer {} due to {}", writer, e.toString(), e); + } + } + + // This method exists solely for unit testing purposes. + protected void close(final IndexWriterCount count) throws IOException { + count.close(); + } + + protected int getWriterCount() { + synchronized (writerCounts) { + return writerCounts.size(); } } @@ -191,17 +342,23 @@ public class SimpleIndexManager implements IndexManager { } - private static class IndexWriterCount implements Closeable { - private final IndexWriter writer; + protected static class IndexWriterCount implements Closeable { + private final EventIndexWriter writer; private final Analyzer analyzer; private final Directory directory; private final int count; + private final boolean closeableWhenUnused; - public IndexWriterCount(final IndexWriter writer, final Analyzer analyzer, final Directory directory, final int count) { + public IndexWriterCount(final EventIndexWriter writer, final Analyzer analyzer, final Directory directory, final int count, final boolean closeableWhenUnused) { this.writer = writer; this.analyzer = analyzer; this.directory = directory; this.count = count; + this.closeableWhenUnused = closeableWhenUnused; + } + + public boolean isCloseableWhenUnused() { + return closeableWhenUnused; } public Analyzer getAnalyzer() { @@ -212,7 +369,7 @@ public class SimpleIndexManager implements IndexManager { return directory; } - public IndexWriter getWriter() { + public EventIndexWriter getWriter() { return writer; } @@ -224,5 +381,10 @@ public class SimpleIndexManager implements IndexManager { public void close() throws IOException { closeQuietly(writer, analyzer, directory); } + + @Override + public String toString() { + return "IndexWriterCount[count=" + count + ", writer=" + writer + ", closeableWhenUnused=" + closeableWhenUnused + "]"; + } } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventFieldNames.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventFieldNames.java new file mode 100644 index 0000000000..d6f50dda2e --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventFieldNames.java @@ -0,0 +1,59 @@ +/* + * 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.schema; + +public class EventFieldNames { + public static final String EVENT_IDENTIFIER = "Event ID"; + public static final String EVENT_TYPE = "Event Type"; + public static final String EVENT_TIME = "Event Time"; + public static final String FLOWFILE_ENTRY_DATE = "FlowFile Entry Date"; + public static final String EVENT_DURATION = "Event Duration"; + public static final String LINEAGE_START_DATE = "Lineage Start Date"; + public static final String COMPONENT_ID = "Component ID"; + public static final String COMPONENT_TYPE = "Component Type"; + public static final String FLOWFILE_UUID = "FlowFile UUID"; + public static final String EVENT_DETAILS = "Event Details"; + public static final String SOURCE_QUEUE_IDENTIFIER = "Source Queue Identifier"; + public static final String CONTENT_CLAIM = "Content Claim"; + public static final String PREVIOUS_CONTENT_CLAIM = "Previous Content Claim"; + public static final String EXPLICIT_CURRENT_CONTENT_CLAIM = "Full Current Content Claim"; + public static final String PARENT_UUIDS = "Parent UUIDs"; + public static final String CHILD_UUIDS = "Child UUIDs"; + + public static final String ATTRIBUTE_NAME = "Attribute Name"; + public static final String ATTRIBUTE_VALUE = "Attribute Value"; + public static final String PREVIOUS_ATTRIBUTES = "Previous Attributes"; + public static final String UPDATED_ATTRIBUTES = "Updated Attributes"; + + public static final String CONTENT_CLAIM_CONTAINER = "Content Claim Container"; + public static final String CONTENT_CLAIM_SECTION = "Content Claim Section"; + public static final String CONTENT_CLAIM_IDENTIFIER = "Content Claim Identifier"; + public static final String CONTENT_CLAIM_OFFSET = "Content Claim Offset"; + public static final String CONTENT_CLAIM_SIZE = "Content Claim Size"; + + public static final String TRANSIT_URI = "Transit URI"; + public static final String SOURCE_SYSTEM_FLOWFILE_IDENTIFIER = "Source System FlowFile Identifier"; + public static final String ALTERNATE_IDENTIFIER = "Alternate Identifier"; + public static final String RELATIONSHIP = "Relationship"; + + // For Lookup Tables + public static final String NO_VALUE = "No Value"; + public static final String EXPLICIT_VALUE = "Explicit Value"; + public static final String LOOKUP_VALUE = "Lookup Value"; + public static final String UNCHANGED_VALUE = "Unchanged"; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventIdFirstHeaderSchema.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventIdFirstHeaderSchema.java new file mode 100644 index 0000000000..1c35c5aaaa --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventIdFirstHeaderSchema.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.schema; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.repository.schema.FieldType; +import org.apache.nifi.repository.schema.RecordField; +import org.apache.nifi.repository.schema.RecordSchema; +import org.apache.nifi.repository.schema.Repetition; +import org.apache.nifi.repository.schema.SimpleRecordField; + +public class EventIdFirstHeaderSchema { + + public static RecordSchema SCHEMA = buildSchema(); + + public static final class FieldNames { + public static final String FIRST_EVENT_ID = "First Event ID"; + public static final String TIMESTAMP_OFFSET = "Timestamp Offset"; + public static final String COMPONENT_IDS = "Component Identifiers"; + public static final String COMPONENT_TYPES = "Component Types"; + public static final String QUEUE_IDS = "Queue Identifiers"; + public static final String EVENT_TYPES = "Event Types"; + } + + private static RecordSchema buildSchema() { + final List fields = new ArrayList<>(); + fields.add(new SimpleRecordField(FieldNames.FIRST_EVENT_ID, FieldType.LONG, Repetition.EXACTLY_ONE)); + fields.add(new SimpleRecordField(FieldNames.TIMESTAMP_OFFSET, FieldType.LONG, Repetition.EXACTLY_ONE)); + fields.add(new SimpleRecordField(FieldNames.COMPONENT_IDS, FieldType.STRING, Repetition.ZERO_OR_MORE)); + fields.add(new SimpleRecordField(FieldNames.COMPONENT_TYPES, FieldType.STRING, Repetition.ZERO_OR_MORE)); + fields.add(new SimpleRecordField(FieldNames.QUEUE_IDS, FieldType.STRING, Repetition.ZERO_OR_MORE)); + fields.add(new SimpleRecordField(FieldNames.EVENT_TYPES, FieldType.STRING, Repetition.ZERO_OR_MORE)); + return new RecordSchema(fields); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecord.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecord.java index c9e7dc8b62..8c82b110fd 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecord.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecord.java @@ -69,47 +69,47 @@ public class EventRecord implements Record { @Override public Object getFieldValue(final String fieldName) { switch (fieldName) { - case EventRecordFields.Names.EVENT_IDENTIFIER: + case EventFieldNames.EVENT_IDENTIFIER: return eventId; - case EventRecordFields.Names.ALTERNATE_IDENTIFIER: + case EventFieldNames.ALTERNATE_IDENTIFIER: return event.getAlternateIdentifierUri(); - case EventRecordFields.Names.CHILD_UUIDS: + case EventFieldNames.CHILD_UUIDS: return event.getChildUuids(); - case EventRecordFields.Names.COMPONENT_ID: + case EventFieldNames.COMPONENT_ID: return event.getComponentId(); - case EventRecordFields.Names.COMPONENT_TYPE: + case EventFieldNames.COMPONENT_TYPE: return event.getComponentType(); - case EventRecordFields.Names.CONTENT_CLAIM: + case EventFieldNames.CONTENT_CLAIM: return contentClaimRecord; - case EventRecordFields.Names.EVENT_DETAILS: + case EventFieldNames.EVENT_DETAILS: return event.getDetails(); - case EventRecordFields.Names.EVENT_DURATION: + case EventFieldNames.EVENT_DURATION: return event.getEventDuration(); - case EventRecordFields.Names.EVENT_TIME: + case EventFieldNames.EVENT_TIME: return event.getEventTime(); - case EventRecordFields.Names.EVENT_TYPE: + case EventFieldNames.EVENT_TYPE: return event.getEventType().name(); - case EventRecordFields.Names.FLOWFILE_ENTRY_DATE: + case EventFieldNames.FLOWFILE_ENTRY_DATE: return event.getFlowFileEntryDate(); - case EventRecordFields.Names.FLOWFILE_UUID: + case EventFieldNames.FLOWFILE_UUID: return event.getFlowFileUuid(); - case EventRecordFields.Names.LINEAGE_START_DATE: + case EventFieldNames.LINEAGE_START_DATE: return event.getLineageStartDate(); - case EventRecordFields.Names.PARENT_UUIDS: + case EventFieldNames.PARENT_UUIDS: return event.getParentUuids(); - case EventRecordFields.Names.PREVIOUS_ATTRIBUTES: + case EventFieldNames.PREVIOUS_ATTRIBUTES: return event.getPreviousAttributes(); - case EventRecordFields.Names.PREVIOUS_CONTENT_CLAIM: + case EventFieldNames.PREVIOUS_CONTENT_CLAIM: return previousClaimRecord; - case EventRecordFields.Names.RELATIONSHIP: + case EventFieldNames.RELATIONSHIP: return event.getRelationship(); - case EventRecordFields.Names.SOURCE_QUEUE_IDENTIFIER: + case EventFieldNames.SOURCE_QUEUE_IDENTIFIER: return event.getSourceQueueIdentifier(); - case EventRecordFields.Names.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER: + case EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER: return event.getSourceSystemFlowFileIdentifier(); - case EventRecordFields.Names.TRANSIT_URI: + case EventFieldNames.TRANSIT_URI: return event.getTransitUri(); - case EventRecordFields.Names.UPDATED_ATTRIBUTES: + case EventFieldNames.UPDATED_ATTRIBUTES: return event.getUpdatedAttributes(); } @@ -119,48 +119,52 @@ public class EventRecord implements Record { @SuppressWarnings("unchecked") public static StandardProvenanceEventRecord getEvent(final Record record, final String storageFilename, final long storageByteOffset, final int maxAttributeLength) { final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder(); - builder.setAlternateIdentifierUri((String) record.getFieldValue(EventRecordFields.Names.ALTERNATE_IDENTIFIER)); - builder.setChildUuids((List) record.getFieldValue(EventRecordFields.Names.CHILD_UUIDS)); - builder.setComponentId((String) record.getFieldValue(EventRecordFields.Names.COMPONENT_ID)); - builder.setComponentType((String) record.getFieldValue(EventRecordFields.Names.COMPONENT_TYPE)); - builder.setDetails((String) record.getFieldValue(EventRecordFields.Names.EVENT_DETAILS)); - builder.setEventDuration((Long) record.getFieldValue(EventRecordFields.Names.EVENT_DURATION)); - builder.setEventTime((Long) record.getFieldValue(EventRecordFields.Names.EVENT_TIME)); - builder.setEventType(ProvenanceEventType.valueOf((String) record.getFieldValue(EventRecordFields.Names.EVENT_TYPE))); - builder.setFlowFileEntryDate((Long) record.getFieldValue(EventRecordFields.Names.FLOWFILE_ENTRY_DATE)); - builder.setFlowFileUUID((String) record.getFieldValue(EventRecordFields.Names.FLOWFILE_UUID)); - builder.setLineageStartDate((Long) record.getFieldValue(EventRecordFields.Names.LINEAGE_START_DATE)); - builder.setParentUuids((List) record.getFieldValue(EventRecordFields.Names.PARENT_UUIDS)); - builder.setPreviousAttributes(truncateAttributes((Map) record.getFieldValue(EventRecordFields.Names.PREVIOUS_ATTRIBUTES), maxAttributeLength)); - builder.setEventId((Long) record.getFieldValue(EventRecordFields.Names.EVENT_IDENTIFIER)); - builder.setRelationship((String) record.getFieldValue(EventRecordFields.Names.RELATIONSHIP)); - builder.setSourceQueueIdentifier((String) record.getFieldValue(EventRecordFields.Names.SOURCE_QUEUE_IDENTIFIER)); - builder.setSourceSystemFlowFileIdentifier((String) record.getFieldValue(EventRecordFields.Names.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER)); - builder.setTransitUri((String) record.getFieldValue(EventRecordFields.Names.TRANSIT_URI)); - builder.setUpdatedAttributes(truncateAttributes((Map) record.getFieldValue(EventRecordFields.Names.UPDATED_ATTRIBUTES), maxAttributeLength)); + builder.setAlternateIdentifierUri((String) record.getFieldValue(EventFieldNames.ALTERNATE_IDENTIFIER)); + builder.setChildUuids((List) record.getFieldValue(EventFieldNames.CHILD_UUIDS)); + builder.setComponentId((String) record.getFieldValue(EventFieldNames.COMPONENT_ID)); + builder.setComponentType((String) record.getFieldValue(EventFieldNames.COMPONENT_TYPE)); + builder.setDetails((String) record.getFieldValue(EventFieldNames.EVENT_DETAILS)); + builder.setEventDuration((Long) record.getFieldValue(EventFieldNames.EVENT_DURATION)); + builder.setEventTime((Long) record.getFieldValue(EventFieldNames.EVENT_TIME)); + builder.setEventType(ProvenanceEventType.valueOf((String) record.getFieldValue(EventFieldNames.EVENT_TYPE))); + builder.setFlowFileEntryDate((Long) record.getFieldValue(EventFieldNames.FLOWFILE_ENTRY_DATE)); + builder.setFlowFileUUID((String) record.getFieldValue(EventFieldNames.FLOWFILE_UUID)); + builder.setLineageStartDate((Long) record.getFieldValue(EventFieldNames.LINEAGE_START_DATE)); + builder.setParentUuids((List) record.getFieldValue(EventFieldNames.PARENT_UUIDS)); + builder.setPreviousAttributes(truncateAttributes((Map) record.getFieldValue(EventFieldNames.PREVIOUS_ATTRIBUTES), maxAttributeLength)); + builder.setRelationship((String) record.getFieldValue(EventFieldNames.RELATIONSHIP)); + builder.setSourceQueueIdentifier((String) record.getFieldValue(EventFieldNames.SOURCE_QUEUE_IDENTIFIER)); + builder.setSourceSystemFlowFileIdentifier((String) record.getFieldValue(EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER)); + builder.setTransitUri((String) record.getFieldValue(EventFieldNames.TRANSIT_URI)); + builder.setUpdatedAttributes(truncateAttributes((Map) record.getFieldValue(EventFieldNames.UPDATED_ATTRIBUTES), maxAttributeLength)); + + final Long eventId = (Long) record.getFieldValue(EventFieldNames.EVENT_IDENTIFIER); + if (eventId != null) { + builder.setEventId(eventId); + } builder.setStorageLocation(storageFilename, storageByteOffset); - final Record currentClaimRecord = (Record) record.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM); + final Record currentClaimRecord = (Record) record.getFieldValue(EventFieldNames.CONTENT_CLAIM); if (currentClaimRecord == null) { builder.setCurrentContentClaim(null, null, null, null, 0L); } else { builder.setCurrentContentClaim( - (String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_CONTAINER), - (String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SECTION), - (String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_IDENTIFIER), - (Long) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_OFFSET), - (Long) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SIZE)); + (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_CONTAINER), + (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SECTION), + (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_IDENTIFIER), + (Long) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_OFFSET), + (Long) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SIZE)); } - final Record previousClaimRecord = (Record) record.getFieldValue(EventRecordFields.Names.PREVIOUS_CONTENT_CLAIM); + final Record previousClaimRecord = (Record) record.getFieldValue(EventFieldNames.PREVIOUS_CONTENT_CLAIM); if (previousClaimRecord != null) { builder.setPreviousContentClaim( - (String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_CONTAINER), - (String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SECTION), - (String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_IDENTIFIER), - (Long) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_OFFSET), - (Long) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SIZE)); + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_CONTAINER), + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SECTION), + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_IDENTIFIER), + (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_OFFSET), + (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SIZE)); } return builder.build(); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecordFields.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecordFields.java index 0582dd89b6..3d79ab4f56 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecordFields.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/EventRecordFields.java @@ -29,82 +29,48 @@ import org.apache.nifi.repository.schema.SimpleRecordField; public class EventRecordFields { - public static class Names { - public static final String EVENT_IDENTIFIER = "Event ID"; - public static final String EVENT_TYPE = "Event Type"; - public static final String EVENT_TIME = "Event Time"; - public static final String FLOWFILE_ENTRY_DATE = "FlowFile Entry Date"; - public static final String EVENT_DURATION = "Event Duration"; - public static final String LINEAGE_START_DATE = "Lineage Start Date"; - public static final String COMPONENT_ID = "Component ID"; - public static final String COMPONENT_TYPE = "Component Type"; - public static final String FLOWFILE_UUID = "FlowFile UUID"; - public static final String EVENT_DETAILS = "Event Details"; - public static final String SOURCE_QUEUE_IDENTIFIER = "Source Queue Identifier"; - public static final String CONTENT_CLAIM = "Content Claim"; - public static final String PREVIOUS_CONTENT_CLAIM = "Previous Content Claim"; - public static final String PARENT_UUIDS = "Parent UUIDs"; - public static final String CHILD_UUIDS = "Child UUIDs"; - - public static final String ATTRIBUTE_NAME = "Attribute Name"; - public static final String ATTRIBUTE_VALUE = "Attribute Value"; - public static final String PREVIOUS_ATTRIBUTES = "Previous Attributes"; - public static final String UPDATED_ATTRIBUTES = "Updated Attributes"; - - public static final String CONTENT_CLAIM_CONTAINER = "Content Claim Container"; - public static final String CONTENT_CLAIM_SECTION = "Content Claim Section"; - public static final String CONTENT_CLAIM_IDENTIFIER = "Content Claim Identifier"; - public static final String CONTENT_CLAIM_OFFSET = "Content Claim Offset"; - public static final String CONTENT_CLAIM_SIZE = "Content Claim Size"; - - public static final String TRANSIT_URI = "Transit URI"; - public static final String SOURCE_SYSTEM_FLOWFILE_IDENTIFIER = "Source System FlowFile Identifier"; - public static final String ALTERNATE_IDENTIFIER = "Alternate Identifier"; - public static final String RELATIONSHIP = "Relationship"; - } - // General Event fields. - public static final RecordField RECORD_IDENTIFIER = new SimpleRecordField(Names.EVENT_IDENTIFIER, FieldType.LONG, EXACTLY_ONE); - public static final RecordField EVENT_TYPE = new SimpleRecordField(Names.EVENT_TYPE, FieldType.STRING, EXACTLY_ONE); - public static final RecordField EVENT_TIME = new SimpleRecordField(Names.EVENT_TIME, FieldType.LONG, EXACTLY_ONE); - public static final RecordField FLOWFILE_ENTRY_DATE = new SimpleRecordField(Names.FLOWFILE_ENTRY_DATE, FieldType.LONG, EXACTLY_ONE); - public static final RecordField EVENT_DURATION = new SimpleRecordField(Names.EVENT_DURATION, FieldType.LONG, EXACTLY_ONE); - public static final RecordField LINEAGE_START_DATE = new SimpleRecordField(Names.LINEAGE_START_DATE, FieldType.LONG, EXACTLY_ONE); - public static final RecordField COMPONENT_ID = new SimpleRecordField(Names.COMPONENT_ID, FieldType.STRING, ZERO_OR_ONE); - public static final RecordField COMPONENT_TYPE = new SimpleRecordField(Names.COMPONENT_TYPE, FieldType.STRING, ZERO_OR_ONE); - public static final RecordField FLOWFILE_UUID = new SimpleRecordField(Names.FLOWFILE_UUID, FieldType.STRING, EXACTLY_ONE); - public static final RecordField EVENT_DETAILS = new SimpleRecordField(Names.EVENT_DETAILS, FieldType.STRING, ZERO_OR_ONE); - public static final RecordField SOURCE_QUEUE_IDENTIFIER = new SimpleRecordField(Names.SOURCE_QUEUE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField RECORD_IDENTIFIER = new SimpleRecordField(EventFieldNames.EVENT_IDENTIFIER, FieldType.LONG, EXACTLY_ONE); + public static final RecordField EVENT_TYPE = new SimpleRecordField(EventFieldNames.EVENT_TYPE, FieldType.STRING, EXACTLY_ONE); + public static final RecordField EVENT_TIME = new SimpleRecordField(EventFieldNames.EVENT_TIME, FieldType.LONG, EXACTLY_ONE); + public static final RecordField FLOWFILE_ENTRY_DATE = new SimpleRecordField(EventFieldNames.FLOWFILE_ENTRY_DATE, FieldType.LONG, EXACTLY_ONE); + public static final RecordField EVENT_DURATION = new SimpleRecordField(EventFieldNames.EVENT_DURATION, FieldType.LONG, EXACTLY_ONE); + public static final RecordField LINEAGE_START_DATE = new SimpleRecordField(EventFieldNames.LINEAGE_START_DATE, FieldType.LONG, EXACTLY_ONE); + public static final RecordField COMPONENT_ID = new SimpleRecordField(EventFieldNames.COMPONENT_ID, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField COMPONENT_TYPE = new SimpleRecordField(EventFieldNames.COMPONENT_TYPE, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField FLOWFILE_UUID = new SimpleRecordField(EventFieldNames.FLOWFILE_UUID, FieldType.STRING, EXACTLY_ONE); + public static final RecordField EVENT_DETAILS = new SimpleRecordField(EventFieldNames.EVENT_DETAILS, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField SOURCE_QUEUE_IDENTIFIER = new SimpleRecordField(EventFieldNames.SOURCE_QUEUE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); // Attributes - public static final RecordField ATTRIBUTE_NAME = new SimpleRecordField(Names.ATTRIBUTE_NAME, FieldType.LONG_STRING, EXACTLY_ONE); - public static final RecordField ATTRIBUTE_VALUE_REQUIRED = new SimpleRecordField(Names.ATTRIBUTE_VALUE, FieldType.LONG_STRING, EXACTLY_ONE); - public static final RecordField ATTRIBUTE_VALUE_OPTIONAL = new SimpleRecordField(Names.ATTRIBUTE_VALUE, FieldType.LONG_STRING, ZERO_OR_ONE); + public static final RecordField ATTRIBUTE_NAME = new SimpleRecordField(EventFieldNames.ATTRIBUTE_NAME, FieldType.LONG_STRING, EXACTLY_ONE); + public static final RecordField ATTRIBUTE_VALUE_REQUIRED = new SimpleRecordField(EventFieldNames.ATTRIBUTE_VALUE, FieldType.LONG_STRING, EXACTLY_ONE); + public static final RecordField ATTRIBUTE_VALUE_OPTIONAL = new SimpleRecordField(EventFieldNames.ATTRIBUTE_VALUE, FieldType.LONG_STRING, ZERO_OR_ONE); - public static final RecordField PREVIOUS_ATTRIBUTES = new MapRecordField(Names.PREVIOUS_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_REQUIRED, EXACTLY_ONE); - public static final RecordField UPDATED_ATTRIBUTES = new MapRecordField(Names.UPDATED_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_OPTIONAL, EXACTLY_ONE); + public static final RecordField PREVIOUS_ATTRIBUTES = new MapRecordField(EventFieldNames.PREVIOUS_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_REQUIRED, EXACTLY_ONE); + public static final RecordField UPDATED_ATTRIBUTES = new MapRecordField(EventFieldNames.UPDATED_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_OPTIONAL, EXACTLY_ONE); // Content Claims - public static final RecordField CONTENT_CLAIM_CONTAINER = new SimpleRecordField(Names.CONTENT_CLAIM_CONTAINER, FieldType.STRING, EXACTLY_ONE); - public static final RecordField CONTENT_CLAIM_SECTION = new SimpleRecordField(Names.CONTENT_CLAIM_SECTION, FieldType.STRING, EXACTLY_ONE); - public static final RecordField CONTENT_CLAIM_IDENTIFIER = new SimpleRecordField(Names.CONTENT_CLAIM_IDENTIFIER, FieldType.STRING, EXACTLY_ONE); - public static final RecordField CONTENT_CLAIM_OFFSET = new SimpleRecordField(Names.CONTENT_CLAIM_OFFSET, FieldType.LONG, EXACTLY_ONE); - public static final RecordField CONTENT_CLAIM_SIZE = new SimpleRecordField(Names.CONTENT_CLAIM_SIZE, FieldType.LONG, EXACTLY_ONE); - public static final RecordField CURRENT_CONTENT_CLAIM = new ComplexRecordField(Names.CONTENT_CLAIM, ZERO_OR_ONE, + public static final RecordField CONTENT_CLAIM_CONTAINER = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_CONTAINER, FieldType.STRING, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_SECTION = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_SECTION, FieldType.STRING, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_IDENTIFIER = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_IDENTIFIER, FieldType.STRING, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_OFFSET = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_OFFSET, FieldType.LONG, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_SIZE = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_SIZE, FieldType.LONG, EXACTLY_ONE); + public static final RecordField CURRENT_CONTENT_CLAIM = new ComplexRecordField(EventFieldNames.CONTENT_CLAIM, ZERO_OR_ONE, CONTENT_CLAIM_CONTAINER, CONTENT_CLAIM_SECTION, CONTENT_CLAIM_IDENTIFIER, CONTENT_CLAIM_OFFSET, CONTENT_CLAIM_SIZE); - public static final RecordField PREVIOUS_CONTENT_CLAIM = new ComplexRecordField(Names.PREVIOUS_CONTENT_CLAIM, ZERO_OR_ONE, + public static final RecordField PREVIOUS_CONTENT_CLAIM = new ComplexRecordField(EventFieldNames.PREVIOUS_CONTENT_CLAIM, ZERO_OR_ONE, CONTENT_CLAIM_CONTAINER, CONTENT_CLAIM_SECTION, CONTENT_CLAIM_IDENTIFIER, CONTENT_CLAIM_OFFSET, CONTENT_CLAIM_SIZE); // EventType-Specific fields // for FORK, JOIN, CLONE, REPLAY - public static final RecordField PARENT_UUIDS = new SimpleRecordField(Names.PARENT_UUIDS, FieldType.STRING, ZERO_OR_MORE); - public static final RecordField CHILD_UUIDS = new SimpleRecordField(Names.CHILD_UUIDS, FieldType.STRING, ZERO_OR_MORE); + public static final RecordField PARENT_UUIDS = new SimpleRecordField(EventFieldNames.PARENT_UUIDS, FieldType.STRING, ZERO_OR_MORE); + public static final RecordField CHILD_UUIDS = new SimpleRecordField(EventFieldNames.CHILD_UUIDS, FieldType.STRING, ZERO_OR_MORE); // for SEND/RECEIVE/FETCH - public static final RecordField TRANSIT_URI = new SimpleRecordField(Names.TRANSIT_URI, FieldType.STRING, ZERO_OR_ONE); - public static final RecordField SOURCE_SYSTEM_FLOWFILE_IDENTIFIER = new SimpleRecordField(Names.SOURCE_QUEUE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField TRANSIT_URI = new SimpleRecordField(EventFieldNames.TRANSIT_URI, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField SOURCE_SYSTEM_FLOWFILE_IDENTIFIER = new SimpleRecordField(EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); // for ADD_INFO - public static final RecordField ALTERNATE_IDENTIFIER = new SimpleRecordField(Names.ALTERNATE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); - public static final RecordField RELATIONSHIP = new SimpleRecordField(Names.RELATIONSHIP, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField ALTERNATE_IDENTIFIER = new SimpleRecordField(EventFieldNames.ALTERNATE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField RELATIONSHIP = new SimpleRecordField(EventFieldNames.RELATIONSHIP, FieldType.STRING, ZERO_OR_ONE); } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecord.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecord.java new file mode 100644 index 0000000000..eccff2aefb --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecord.java @@ -0,0 +1,363 @@ +/* + * 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.schema; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.StandardProvenanceEventRecord; +import org.apache.nifi.repository.schema.FieldMapRecord; +import org.apache.nifi.repository.schema.NamedValue; +import org.apache.nifi.repository.schema.Record; +import org.apache.nifi.repository.schema.RecordField; +import org.apache.nifi.repository.schema.RecordSchema; + +public class LookupTableEventRecord implements Record { + private final RecordSchema schema; + private final ProvenanceEventRecord event; + private final long eventId; + private final Record contentClaimRecord; + private final Record previousClaimRecord; + + private final long eventIdStartOffset; + private final long startTimeOffset; + private final Map componentIdMap; + private final Map componentTypeMap; + private final Map queueIdMap; + private final Map eventTypeMap; + + public LookupTableEventRecord(final ProvenanceEventRecord event, final long eventId, final RecordSchema schema, final RecordSchema contentClaimSchema, + final RecordSchema previousContentClaimSchema, final long eventIdStartOffset, final long startTimeOffset, final Map componentIdMap, + final Map componentTypeMap, final Map queueIdMap, final Map eventTypeMap) { + this.schema = schema; + this.event = event; + this.eventId = eventId; + this.previousClaimRecord = createPreviousContentClaimRecord(previousContentClaimSchema, event.getPreviousContentClaimContainer(), event.getPreviousContentClaimSection(), + event.getPreviousContentClaimIdentifier(), event.getPreviousContentClaimOffset(), event.getPreviousFileSize()); + this.contentClaimRecord = createContentClaimRecord(contentClaimSchema, event.getContentClaimContainer(), event.getContentClaimSection(), + event.getContentClaimIdentifier(), event.getContentClaimOffset(), event.getFileSize()); + + this.eventIdStartOffset = eventIdStartOffset; + this.startTimeOffset = startTimeOffset; + this.componentIdMap = componentIdMap; + this.componentTypeMap = componentTypeMap; + this.queueIdMap = queueIdMap; + this.eventTypeMap = eventTypeMap; + } + + @Override + public RecordSchema getSchema() { + return schema; + } + + + private static Record createPreviousContentClaimRecord(final RecordSchema contentClaimSchema, final String container, final String section, + final String identifier, final Long offset, final Long size) { + + if (container == null || section == null || identifier == null) { + return null; + } + + final Map fieldValues = new HashMap<>(); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_CONTAINER, container); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_SECTION, section); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_IDENTIFIER, identifier); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_OFFSET, offset); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_SIZE, size); + return new FieldMapRecord(fieldValues, contentClaimSchema); + + } + + private static Record createContentClaimRecord(final RecordSchema contentClaimSchema, final String container, final String section, + final String identifier, final Long offset, final Long size) { + + if (container == null || section == null || identifier == null) { + final Map lookupValues = Collections.singletonMap(LookupTableEventRecordFields.NO_VALUE, EventFieldNames.NO_VALUE); + final List noValueFields = Collections.singletonList(contentClaimSchema.getField(EventFieldNames.NO_VALUE)); + return new FieldMapRecord(lookupValues, new RecordSchema(noValueFields)); + } + + final Map fieldValues = new HashMap<>(); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_CONTAINER, container); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_SECTION, section); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_IDENTIFIER, identifier); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_OFFSET, offset); + fieldValues.put(EventRecordFields.CONTENT_CLAIM_SIZE, size); + + final List explicitClaimFields = contentClaimSchema.getField(EventFieldNames.EXPLICIT_VALUE).getSubFields(); + final Record explicitClaimRecord = new FieldMapRecord(fieldValues, new RecordSchema(explicitClaimFields)); + return explicitClaimRecord; + } + + private static String readLookupValue(final Object recordValue, final List lookup) { + if (recordValue == null) { + return null; + } + + // NO_VALUE type + if (recordValue instanceof Boolean) { + return null; + } + + // LOOKUP type + if (recordValue instanceof Integer) { + final Integer indexValue = (Integer) recordValue; + final int index = indexValue.intValue(); + if (index > lookup.size() - 1) { + return null; + } + + return lookup.get(index); + } + + // EXPLICIT_VALUE type + if (recordValue instanceof String) { + return (String) recordValue; + } + + return null; + } + + private NamedValue createLookupValue(final String literalValue, final Map lookup) { + if (literalValue == null) { + final Map lookupValues = Collections.singletonMap(LookupTableEventRecordFields.NO_VALUE, EventFieldNames.NO_VALUE); + final Record record = new FieldMapRecord(lookupValues, LookupTableEventSchema.NO_VALUE_SCHEMA); + final NamedValue namedValue = new NamedValue(EventFieldNames.NO_VALUE, record); + return namedValue; + } + + final Integer index = lookup.get(literalValue); + if (index == null) { + final Map lookupValues = Collections.singletonMap(LookupTableEventRecordFields.EXPLICIT_STRING, literalValue); + final Record record = new FieldMapRecord(lookupValues, LookupTableEventSchema.EXPLICIT_STRING_SCHEMA); + final NamedValue namedValue = new NamedValue(EventFieldNames.EXPLICIT_VALUE, record); + return namedValue; + } else { + final Map lookupValues = Collections.singletonMap(LookupTableEventRecordFields.LOOKUP_VALUE, index); + final Record record = new FieldMapRecord(lookupValues, LookupTableEventSchema.LOOKUP_VALUE_SCHEMA); + final NamedValue namedValue = new NamedValue(EventFieldNames.LOOKUP_VALUE, record); + return namedValue; + } + } + + private NamedValue createExplicitSameOrNoneValue(final Record newValue, final Record oldValue, final Supplier recordSupplier) { + if (newValue == null || EventFieldNames.NO_VALUE.equals(newValue.getSchema().getFields().get(0).getFieldName())) { + final Map lookupValues = Collections.singletonMap(LookupTableEventRecordFields.NO_VALUE, EventFieldNames.NO_VALUE); + final Record record = new FieldMapRecord(lookupValues, LookupTableEventSchema.NO_VALUE_SCHEMA); + final NamedValue namedValue = new NamedValue(EventFieldNames.NO_VALUE, record); + return namedValue; + } else if (newValue.equals(oldValue)) { + final Map lookupValues = Collections.singletonMap(LookupTableEventRecordFields.UNCHANGED_VALUE, EventFieldNames.UNCHANGED_VALUE); + final Record record = new FieldMapRecord(lookupValues, LookupTableEventSchema.UNCHANGED_VALUE_SCHEMA); + final NamedValue namedValue = new NamedValue(EventFieldNames.UNCHANGED_VALUE, record); + return namedValue; + } + + final Record record = recordSupplier.get(); + final NamedValue namedValue = new NamedValue(EventFieldNames.EXPLICIT_VALUE, record); + return namedValue; + } + + @Override + public Object getFieldValue(final String fieldName) { + switch (fieldName) { + case EventFieldNames.EVENT_IDENTIFIER: + return (int) (eventId - eventIdStartOffset); + case EventFieldNames.ALTERNATE_IDENTIFIER: + return event.getAlternateIdentifierUri(); + case EventFieldNames.CHILD_UUIDS: + return event.getChildUuids(); + case EventFieldNames.COMPONENT_ID: + return createLookupValue(event.getComponentId(), componentIdMap); + case EventFieldNames.COMPONENT_TYPE: + return createLookupValue(event.getComponentType(), componentTypeMap); + case EventFieldNames.CONTENT_CLAIM: + return createExplicitSameOrNoneValue(contentClaimRecord, previousClaimRecord, () -> contentClaimRecord); + case EventFieldNames.EVENT_DETAILS: + return event.getDetails(); + case EventFieldNames.EVENT_DURATION: + return (int) event.getEventDuration(); + case EventFieldNames.EVENT_TIME: + return (int) (event.getEventTime() - startTimeOffset); + case EventFieldNames.EVENT_TYPE: + return eventTypeMap.get(event.getEventType().name()); + case EventFieldNames.FLOWFILE_ENTRY_DATE: + return (int) (event.getFlowFileEntryDate() - startTimeOffset); + case EventFieldNames.LINEAGE_START_DATE: + return (int) (event.getLineageStartDate() - startTimeOffset); + case EventFieldNames.PARENT_UUIDS: + return event.getParentUuids(); + case EventFieldNames.PREVIOUS_ATTRIBUTES: + return event.getPreviousAttributes(); + case EventFieldNames.PREVIOUS_CONTENT_CLAIM: + return previousClaimRecord; + case EventFieldNames.RELATIONSHIP: + return event.getRelationship(); + case EventFieldNames.SOURCE_QUEUE_IDENTIFIER: + return createLookupValue(event.getSourceQueueIdentifier(), queueIdMap); + case EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER: + return event.getSourceSystemFlowFileIdentifier(); + case EventFieldNames.TRANSIT_URI: + return event.getTransitUri(); + case EventFieldNames.UPDATED_ATTRIBUTES: + return event.getUpdatedAttributes(); + case EventFieldNames.FLOWFILE_UUID: + return event.getAttribute(CoreAttributes.UUID.key()); + } + + return null; + } + + private static Long addLong(final Integer optionalValue, final long requiredValue) { + if (optionalValue == null) { + return null; + } + + return optionalValue.longValue() + requiredValue; + } + + @SuppressWarnings("unchecked") + public static StandardProvenanceEventRecord getEvent(final Record record, final String storageFilename, final long storageByteOffset, final int maxAttributeLength, + final long eventIdStartOffset, final long startTimeOffset, final List componentIds, final List componentTypes, + final List queueIds, final List eventTypes) { + + final Map previousAttributes = truncateAttributes((Map) record.getFieldValue(EventFieldNames.PREVIOUS_ATTRIBUTES), maxAttributeLength); + final Map updatedAttributes = truncateAttributes((Map) record.getFieldValue(EventFieldNames.UPDATED_ATTRIBUTES), maxAttributeLength); + + final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder(); + builder.setAlternateIdentifierUri((String) record.getFieldValue(EventFieldNames.ALTERNATE_IDENTIFIER)); + builder.setChildUuids((List) record.getFieldValue(EventFieldNames.CHILD_UUIDS)); + builder.setDetails((String) record.getFieldValue(EventFieldNames.EVENT_DETAILS)); + builder.setParentUuids((List) record.getFieldValue(EventFieldNames.PARENT_UUIDS)); + builder.setPreviousAttributes(previousAttributes); + builder.setRelationship((String) record.getFieldValue(EventFieldNames.RELATIONSHIP)); + builder.setSourceSystemFlowFileIdentifier((String) record.getFieldValue(EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER)); + builder.setTransitUri((String) record.getFieldValue(EventFieldNames.TRANSIT_URI)); + builder.setUpdatedAttributes(updatedAttributes); + + + builder.setComponentId(readLookupValue(record.getFieldValue(EventFieldNames.COMPONENT_ID), componentIds)); + builder.setComponentType(readLookupValue(record.getFieldValue(EventFieldNames.COMPONENT_TYPE), componentTypes)); + builder.setSourceQueueIdentifier(readLookupValue(record.getFieldValue(EventFieldNames.SOURCE_QUEUE_IDENTIFIER), queueIds)); + + // Determine the event type + final Integer eventTypeOrdinal = (Integer) record.getFieldValue(EventFieldNames.EVENT_TYPE); + if (eventTypeOrdinal == null || eventTypeOrdinal > eventTypes.size() || eventTypeOrdinal < 0) { + builder.setEventType(ProvenanceEventType.UNKNOWN); + } else { + try { + builder.setEventType(ProvenanceEventType.valueOf(eventTypes.get(eventTypeOrdinal))); + } catch (final Exception e) { + builder.setEventType(ProvenanceEventType.UNKNOWN); + } + } + + String uuid = updatedAttributes == null ? null : updatedAttributes.get(CoreAttributes.UUID.key()); + if (uuid == null) { + uuid = previousAttributes == null ? null : previousAttributes.get(CoreAttributes.UUID.key()); + } + builder.setFlowFileUUID(uuid); + + builder.setEventDuration((Integer) record.getFieldValue(EventFieldNames.EVENT_DURATION)); + builder.setEventTime(addLong((Integer) record.getFieldValue(EventFieldNames.EVENT_TIME), startTimeOffset)); + builder.setFlowFileEntryDate(addLong((Integer) record.getFieldValue(EventFieldNames.FLOWFILE_ENTRY_DATE), startTimeOffset)); + builder.setLineageStartDate(addLong((Integer) record.getFieldValue(EventFieldNames.LINEAGE_START_DATE), startTimeOffset)); + + final Integer eventId = (Integer) record.getFieldValue(EventFieldNames.EVENT_IDENTIFIER); + if (eventId != null) { + builder.setEventId(eventId.longValue() + eventIdStartOffset); + } + + builder.setStorageLocation(storageFilename, storageByteOffset); + + final Record previousClaimRecord = (Record) record.getFieldValue(EventFieldNames.PREVIOUS_CONTENT_CLAIM); + if (previousClaimRecord != null) { + builder.setPreviousContentClaim( + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_CONTAINER), + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SECTION), + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_IDENTIFIER), + (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_OFFSET), + (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SIZE)); + } + + final Object contentClaimObject = record.getFieldValue(EventFieldNames.CONTENT_CLAIM); + + // NO_VALUE type + builder.setCurrentContentClaim(null, null, null, null, 0L); + if (contentClaimObject != null) { + if (contentClaimObject instanceof String) { + final String contentClaimDescription = (String) contentClaimObject; + switch (contentClaimDescription) { + case EventFieldNames.UNCHANGED_VALUE: + builder.setCurrentContentClaim((String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_CONTAINER), + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SECTION), + (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_IDENTIFIER), + (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_OFFSET), + (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SIZE)); + break; + } + } else if (contentClaimObject instanceof Record) { + final Record currentClaimRecord = (Record) contentClaimObject; + builder.setCurrentContentClaim( + (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_CONTAINER), + (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SECTION), + (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_IDENTIFIER), + (Long) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_OFFSET), + (Long) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SIZE)); + } + } + + return builder.build(); + } + + private static Map truncateAttributes(final Map attributes, final int maxAttributeLength) { + if (attributes == null) { + return null; + } + + // Check if any attribute value exceeds the attribute length + final boolean anyExceedsLength = attributes.values().stream() + .filter(value -> value != null) + .anyMatch(value -> value.length() > maxAttributeLength); + + if (!anyExceedsLength) { + return attributes; + } + + final Map truncated = new HashMap<>(); + for (final Map.Entry entry : attributes.entrySet()) { + final String key = entry.getKey(); + final String value = entry.getValue(); + + if (value == null || value.length() <= maxAttributeLength) { + truncated.put(key, value); + continue; + } + + truncated.put(key, value.substring(0, maxAttributeLength)); + } + + return truncated; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecordFields.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecordFields.java new file mode 100644 index 0000000000..7b33ded095 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventRecordFields.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.schema; + +import static org.apache.nifi.repository.schema.Repetition.EXACTLY_ONE; +import static org.apache.nifi.repository.schema.Repetition.ZERO_OR_MORE; +import static org.apache.nifi.repository.schema.Repetition.ZERO_OR_ONE; + +import org.apache.nifi.repository.schema.ComplexRecordField; +import org.apache.nifi.repository.schema.FieldType; +import org.apache.nifi.repository.schema.MapRecordField; +import org.apache.nifi.repository.schema.RecordField; +import org.apache.nifi.repository.schema.Repetition; +import org.apache.nifi.repository.schema.SimpleRecordField; +import org.apache.nifi.repository.schema.UnionRecordField; + +public class LookupTableEventRecordFields { + + // General Event fields. + public static final RecordField RECORD_IDENTIFIER_OFFSET = new SimpleRecordField(EventFieldNames.EVENT_IDENTIFIER, FieldType.INT, EXACTLY_ONE); + public static final RecordField EVENT_TYPE_ORDINAL = new SimpleRecordField(EventFieldNames.EVENT_TYPE, FieldType.INT, EXACTLY_ONE); + public static final RecordField EVENT_TIME_OFFSET = new SimpleRecordField(EventFieldNames.EVENT_TIME, FieldType.INT, EXACTLY_ONE); + public static final RecordField FLOWFILE_ENTRY_DATE_OFFSET = new SimpleRecordField(EventFieldNames.FLOWFILE_ENTRY_DATE, FieldType.INT, EXACTLY_ONE); + public static final RecordField EVENT_DURATION = new SimpleRecordField(EventFieldNames.EVENT_DURATION, FieldType.INT, EXACTLY_ONE); + public static final RecordField LINEAGE_START_DATE_OFFSET = new SimpleRecordField(EventFieldNames.LINEAGE_START_DATE, FieldType.INT, EXACTLY_ONE); + public static final RecordField EVENT_DETAILS = new SimpleRecordField(EventFieldNames.EVENT_DETAILS, FieldType.STRING, ZERO_OR_ONE); + + // Make lookup id or a string, depending on whether or not available in header. + public static final RecordField NO_VALUE = new SimpleRecordField(EventFieldNames.NO_VALUE, FieldType.STRING, Repetition.EXACTLY_ONE); + public static final RecordField EXPLICIT_STRING = new SimpleRecordField(EventFieldNames.EXPLICIT_VALUE, FieldType.STRING, Repetition.EXACTLY_ONE); + public static final RecordField LOOKUP_VALUE = new SimpleRecordField(EventFieldNames.LOOKUP_VALUE, FieldType.INT, Repetition.EXACTLY_ONE); + public static final RecordField UNCHANGED_VALUE = new SimpleRecordField(EventFieldNames.UNCHANGED_VALUE, FieldType.STRING, Repetition.EXACTLY_ONE); + + public static final RecordField COMPONENT_ID = new UnionRecordField(EventFieldNames.COMPONENT_ID, Repetition.EXACTLY_ONE, NO_VALUE, EXPLICIT_STRING, LOOKUP_VALUE); + public static final RecordField SOURCE_QUEUE_ID = new UnionRecordField(EventFieldNames.SOURCE_QUEUE_IDENTIFIER, Repetition.EXACTLY_ONE, NO_VALUE, EXPLICIT_STRING, LOOKUP_VALUE); + public static final RecordField COMPONENT_TYPE = new UnionRecordField(EventFieldNames.COMPONENT_TYPE, Repetition.EXACTLY_ONE, EXPLICIT_STRING, LOOKUP_VALUE); + + // Attributes + public static final RecordField ATTRIBUTE_NAME = new SimpleRecordField(EventFieldNames.ATTRIBUTE_NAME, FieldType.LONG_STRING, EXACTLY_ONE); + public static final RecordField ATTRIBUTE_VALUE_REQUIRED = new SimpleRecordField(EventFieldNames.ATTRIBUTE_VALUE, FieldType.LONG_STRING, EXACTLY_ONE); + public static final RecordField ATTRIBUTE_VALUE_OPTIONAL = new SimpleRecordField(EventFieldNames.ATTRIBUTE_VALUE, FieldType.LONG_STRING, ZERO_OR_ONE); + + public static final RecordField PREVIOUS_ATTRIBUTES = new MapRecordField(EventFieldNames.PREVIOUS_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_REQUIRED, EXACTLY_ONE); + public static final RecordField UPDATED_ATTRIBUTES = new MapRecordField(EventFieldNames.UPDATED_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_OPTIONAL, EXACTLY_ONE); + + // Content Claims + public static final RecordField CONTENT_CLAIM_CONTAINER = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_CONTAINER, FieldType.STRING, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_SECTION = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_SECTION, FieldType.STRING, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_IDENTIFIER = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_IDENTIFIER, FieldType.STRING, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_OFFSET = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_OFFSET, FieldType.LONG, EXACTLY_ONE); + public static final RecordField CONTENT_CLAIM_SIZE = new SimpleRecordField(EventFieldNames.CONTENT_CLAIM_SIZE, FieldType.LONG, EXACTLY_ONE); + + public static final RecordField PREVIOUS_CONTENT_CLAIM = new ComplexRecordField(EventFieldNames.PREVIOUS_CONTENT_CLAIM, ZERO_OR_ONE, + CONTENT_CLAIM_CONTAINER, CONTENT_CLAIM_SECTION, CONTENT_CLAIM_IDENTIFIER, CONTENT_CLAIM_OFFSET, CONTENT_CLAIM_SIZE); + + public static final RecordField CURRENT_CONTENT_CLAIM_EXPLICIT = new ComplexRecordField(EventFieldNames.EXPLICIT_VALUE, EXACTLY_ONE, + CONTENT_CLAIM_CONTAINER, CONTENT_CLAIM_SECTION, CONTENT_CLAIM_IDENTIFIER, CONTENT_CLAIM_OFFSET, CONTENT_CLAIM_SIZE); + public static final RecordField CURRENT_CONTENT_CLAIM = new UnionRecordField(EventFieldNames.CONTENT_CLAIM, + Repetition.EXACTLY_ONE, NO_VALUE, UNCHANGED_VALUE, CURRENT_CONTENT_CLAIM_EXPLICIT); + + + // EventType-Specific fields + // for FORK, JOIN, CLONE, REPLAY + public static final RecordField PARENT_UUIDS = new SimpleRecordField(EventFieldNames.PARENT_UUIDS, FieldType.STRING, ZERO_OR_MORE); + public static final RecordField CHILD_UUIDS = new SimpleRecordField(EventFieldNames.CHILD_UUIDS, FieldType.STRING, ZERO_OR_MORE); + + // for SEND/RECEIVE/FETCH + public static final RecordField TRANSIT_URI = new SimpleRecordField(EventFieldNames.TRANSIT_URI, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField SOURCE_SYSTEM_FLOWFILE_IDENTIFIER = new SimpleRecordField(EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); + + // for ADD_INFO + public static final RecordField ALTERNATE_IDENTIFIER = new SimpleRecordField(EventFieldNames.ALTERNATE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE); + public static final RecordField RELATIONSHIP = new SimpleRecordField(EventFieldNames.RELATIONSHIP, FieldType.STRING, ZERO_OR_ONE); +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventSchema.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventSchema.java new file mode 100644 index 0000000000..7110336422 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/LookupTableEventSchema.java @@ -0,0 +1,94 @@ +/* + * 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.schema; + +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.ALTERNATE_IDENTIFIER; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.CHILD_UUIDS; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.COMPONENT_ID; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.COMPONENT_TYPE; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.CURRENT_CONTENT_CLAIM; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.EVENT_DETAILS; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.EVENT_DURATION; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.EVENT_TIME_OFFSET; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.EVENT_TYPE_ORDINAL; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.EXPLICIT_STRING; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.FLOWFILE_ENTRY_DATE_OFFSET; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.LINEAGE_START_DATE_OFFSET; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.LOOKUP_VALUE; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.NO_VALUE; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.PARENT_UUIDS; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.PREVIOUS_ATTRIBUTES; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.PREVIOUS_CONTENT_CLAIM; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.RECORD_IDENTIFIER_OFFSET; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.RELATIONSHIP; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.SOURCE_QUEUE_ID; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.TRANSIT_URI; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.UNCHANGED_VALUE; +import static org.apache.nifi.provenance.schema.LookupTableEventRecordFields.UPDATED_ATTRIBUTES; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.repository.schema.RecordField; +import org.apache.nifi.repository.schema.RecordSchema; + +public class LookupTableEventSchema { + public static final RecordSchema EVENT_SCHEMA = buildSchemaV1(false); + + public static final RecordSchema NO_VALUE_SCHEMA = new RecordSchema(Collections.singletonList(NO_VALUE)); + public static final RecordSchema EXPLICIT_STRING_SCHEMA = new RecordSchema(Collections.singletonList(EXPLICIT_STRING)); + public static final RecordSchema UNCHANGED_VALUE_SCHEMA = new RecordSchema(Collections.singletonList(UNCHANGED_VALUE)); + public static final RecordSchema LOOKUP_VALUE_SCHEMA = new RecordSchema(Collections.singletonList(LOOKUP_VALUE)); + + public static final RecordSchema CONTENT_CLAIM_SCHEMA = new RecordSchema(Collections.singletonList(CURRENT_CONTENT_CLAIM)); + + private static RecordSchema buildSchemaV1(final boolean includeEventId) { + final List fields = new ArrayList<>(); + if (includeEventId) { + fields.add(RECORD_IDENTIFIER_OFFSET); + } + + fields.add(EVENT_TYPE_ORDINAL); + fields.add(EVENT_TIME_OFFSET); + fields.add(FLOWFILE_ENTRY_DATE_OFFSET); + fields.add(EVENT_DURATION); + fields.add(LINEAGE_START_DATE_OFFSET); + fields.add(COMPONENT_ID); + fields.add(COMPONENT_TYPE); + fields.add(EVENT_DETAILS); + fields.add(PREVIOUS_ATTRIBUTES); + fields.add(UPDATED_ATTRIBUTES); + fields.add(CURRENT_CONTENT_CLAIM); + fields.add(PREVIOUS_CONTENT_CLAIM); + fields.add(SOURCE_QUEUE_ID); + + // EventType-Specific fields + fields.add(PARENT_UUIDS); // for FORK, JOIN, CLONE, REPLAY events + fields.add(CHILD_UUIDS); // for FORK, JOIN, CLONE, REPLAY events + fields.add(TRANSIT_URI); // for SEND/RECEIVE/FETCH events + fields.add(SOURCE_SYSTEM_FLOWFILE_IDENTIFIER); // for SEND/RECEIVE events + fields.add(ALTERNATE_IDENTIFIER); // for ADD_INFO events + fields.add(RELATIONSHIP); // for ROUTE events + + final RecordSchema schema = new RecordSchema(fields); + return schema; + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/ProvenanceEventSchema.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/ProvenanceEventSchema.java index d70bd392a5..4655613065 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/ProvenanceEventSchema.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/schema/ProvenanceEventSchema.java @@ -46,11 +46,15 @@ import org.apache.nifi.repository.schema.RecordField; import org.apache.nifi.repository.schema.RecordSchema; public class ProvenanceEventSchema { - public static final RecordSchema PROVENANCE_EVENT_SCHEMA_V1 = buildSchemaV1(); + public static final RecordSchema PROVENANCE_EVENT_SCHEMA_V1 = buildSchemaV1(true); + public static final RecordSchema PROVENANCE_EVENT_SCHEMA_V1_WITHOUT_EVENT_ID = buildSchemaV1(false); - private static RecordSchema buildSchemaV1() { + private static RecordSchema buildSchemaV1(final boolean includeEventId) { final List fields = new ArrayList<>(); - fields.add(RECORD_IDENTIFIER); + if (includeEventId) { + fields.add(RECORD_IDENTIFIER); + } + fields.add(EVENT_TYPE); fields.add(EVENT_TIME); fields.add(FLOWFILE_ENTRY_DATE); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordReader.java index 056829ad9f..1a6c3c55dc 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordReader.java @@ -17,18 +17,18 @@ package org.apache.nifi.provenance.serialization; +import java.io.BufferedInputStream; import java.io.DataInputStream; import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.util.Optional; import java.util.zip.GZIPInputStream; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.StandardProvenanceEventRecord; -import org.apache.nifi.provenance.StandardRecordReader; import org.apache.nifi.provenance.toc.TocReader; -import org.apache.nifi.stream.io.BufferedInputStream; import org.apache.nifi.stream.io.ByteCountingInputStream; import org.apache.nifi.stream.io.LimitingInputStream; import org.apache.nifi.stream.io.StreamUtils; @@ -36,7 +36,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; public abstract class CompressableRecordReader implements RecordReader { - private static final Logger logger = LoggerFactory.getLogger(StandardRecordReader.class); + private static final Logger logger = LoggerFactory.getLogger(CompressableRecordReader.class); private final ByteCountingInputStream rawInputStream; private final String filename; @@ -48,6 +48,7 @@ public abstract class CompressableRecordReader implements RecordReader { private DataInputStream dis; private ByteCountingInputStream byteCountingIn; + private StandardProvenanceEventRecord pushbackEvent = null; public CompressableRecordReader(final InputStream in, final String filename, final int maxAttributeChars) throws IOException { this(in, filename, null, maxAttributeChars); @@ -120,6 +121,8 @@ public abstract class CompressableRecordReader implements RecordReader { try { StreamUtils.skip(rawInputStream, bytesToSkip); logger.debug("Skipped stream from offset {} to {} ({} bytes skipped)", curOffset, offset, bytesToSkip); + } catch (final EOFException eof) { + throw new EOFException("Attempted to skip to byte offset " + offset + " for " + filename + " but file does not have that many bytes (TOC Reader=" + getTocReader() + ")"); } catch (final IOException e) { throw new IOException("Failed to skip to offset " + offset + " for block " + blockIndex + " of Provenance Log " + filename, e); } @@ -177,24 +180,29 @@ public abstract class CompressableRecordReader implements RecordReader { return byteCountingIn.getBytesConsumed(); } - private boolean isData() throws IOException { - byteCountingIn.mark(1); - int nextByte = byteCountingIn.read(); - byteCountingIn.reset(); + @Override + public boolean isData() { + try { + byteCountingIn.mark(1); + int nextByte = byteCountingIn.read(); + byteCountingIn.reset(); - if (nextByte < 0) { - try { - resetStreamForNextBlock(); - } catch (final EOFException eof) { - return false; + if (nextByte < 0) { + try { + resetStreamForNextBlock(); + } catch (final EOFException eof) { + return false; + } + + byteCountingIn.mark(1); + nextByte = byteCountingIn.read(); + byteCountingIn.reset(); } - byteCountingIn.mark(1); - nextByte = byteCountingIn.read(); - byteCountingIn.reset(); + return nextByte >= 0; + } catch (final IOException ioe) { + return false; } - - return nextByte >= 0; } @Override @@ -268,6 +276,12 @@ public abstract class CompressableRecordReader implements RecordReader { @Override public StandardProvenanceEventRecord nextRecord() throws IOException { + if (pushbackEvent != null) { + final StandardProvenanceEventRecord toReturn = pushbackEvent; + pushbackEvent = null; + return toReturn; + } + if (isData()) { return nextRecord(dis, serializationVersion); } else { @@ -275,6 +289,65 @@ public abstract class CompressableRecordReader implements RecordReader { } } + protected Optional getBlockIndex(final long eventId) { + final TocReader tocReader = getTocReader(); + if (tocReader == null) { + return Optional.empty(); + } else { + final Integer blockIndex = tocReader.getBlockIndexForEventId(eventId); + return Optional.ofNullable(blockIndex); + } + } + + @Override + public Optional skipToEvent(final long eventId) throws IOException { + if (pushbackEvent != null) { + final StandardProvenanceEventRecord previousPushBack = pushbackEvent; + if (previousPushBack.getEventId() >= eventId) { + return Optional.of(previousPushBack); + } else { + pushbackEvent = null; + } + } + + final Optional blockIndex = getBlockIndex(eventId); + if (blockIndex.isPresent()) { + // Skip to the appropriate block index and then read until we've found an Event + // that has an ID >= the event id. + skipToBlock(blockIndex.get()); + } + + try { + boolean read = true; + while (read) { + final Optional eventOptional = readToEvent(eventId, dis, serializationVersion); + if (eventOptional.isPresent()) { + pushbackEvent = eventOptional.get(); + return Optional.of(pushbackEvent); + } else { + read = isData(); + } + } + + return Optional.empty(); + } catch (final EOFException eof) { + // This can occur if we run out of data and attempt to read the next event ID. + logger.error("Unexpectedly reached end of File when looking for Provenance Event with ID {} in {}", eventId, filename); + return Optional.empty(); + } + } + + protected Optional readToEvent(final long eventId, final DataInputStream dis, final int serializationVerison) throws IOException { + StandardProvenanceEventRecord event; + while ((event = nextRecord()) != null) { + if (event.getEventId() >= eventId) { + return Optional.of(event); + } + } + + return Optional.empty(); + } + protected abstract StandardProvenanceEventRecord nextRecord(DataInputStream in, int serializationVersion) throws IOException; protected void readHeader(DataInputStream in, int serializationVersion) throws IOException { diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordWriter.java index fa0e390f6e..b5646009b0 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/CompressableRecordWriter.java @@ -17,17 +17,18 @@ package org.apache.nifi.provenance.serialization; +import java.io.BufferedOutputStream; +import java.io.DataOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.util.concurrent.atomic.AtomicLong; import org.apache.nifi.provenance.AbstractRecordWriter; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.toc.TocWriter; -import org.apache.nifi.stream.io.BufferedOutputStream; import org.apache.nifi.stream.io.ByteCountingOutputStream; -import org.apache.nifi.stream.io.DataOutputStream; import org.apache.nifi.stream.io.GZIPOutputStream; import org.apache.nifi.stream.io.NonCloseableOutputStream; import org.slf4j.Logger; @@ -40,14 +41,16 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { private final ByteCountingOutputStream rawOutStream; private final boolean compressed; private final int uncompressedBlockSize; + private final AtomicLong idGenerator; private DataOutputStream out; private ByteCountingOutputStream byteCountingOut; - private long lastBlockOffset = 0L; + private long blockStartOffset = 0L; private int recordCount = 0; - public CompressableRecordWriter(final File file, final TocWriter writer, final boolean compressed, final int uncompressedBlockSize) throws IOException { + public CompressableRecordWriter(final File file, final AtomicLong idGenerator, final TocWriter writer, final boolean compressed, + final int uncompressedBlockSize) throws IOException { super(file, writer); logger.trace("Creating Record Writer for {}", file.getName()); @@ -55,18 +58,25 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { this.fos = new FileOutputStream(file); rawOutStream = new ByteCountingOutputStream(fos); this.uncompressedBlockSize = uncompressedBlockSize; + this.idGenerator = idGenerator; } - public CompressableRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { - super(null, tocWriter); + public CompressableRecordWriter(final OutputStream out, final String storageLocation, final AtomicLong idGenerator, final TocWriter tocWriter, final boolean compressed, + final int uncompressedBlockSize) throws IOException { + super(storageLocation, tocWriter); this.fos = null; this.compressed = compressed; this.uncompressedBlockSize = uncompressedBlockSize; this.rawOutStream = new ByteCountingOutputStream(out); + this.idGenerator = idGenerator; } + protected AtomicLong getIdGenerator() { + return idGenerator; + } + @Override public synchronized void writeHeader(final long firstEventId) throws IOException { if (isDirty()) { @@ -74,13 +84,13 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { } try { - lastBlockOffset = rawOutStream.getBytesWritten(); + blockStartOffset = rawOutStream.getBytesWritten(); resetWriteStream(firstEventId); out.writeUTF(getSerializationName()); out.writeInt(getSerializationVersion()); writeHeader(firstEventId, out); out.flush(); - lastBlockOffset = rawOutStream.getBytesWritten(); + blockStartOffset = getBytesWritten(); } catch (final IOException ioe) { markDirty(); throw ioe; @@ -95,7 +105,7 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { * @param eventId the first id that will be written to the new block * @throws IOException if unable to flush/close the current streams properly */ - private void resetWriteStream(final long eventId) throws IOException { + protected void resetWriteStream(final Long eventId) throws IOException { try { if (out != null) { out.flush(); @@ -114,13 +124,13 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { out.close(); } - if (tocWriter != null) { + if (tocWriter != null && eventId != null) { tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId); } writableStream = new BufferedOutputStream(new GZIPOutputStream(new NonCloseableOutputStream(rawOutStream), 1), 65536); } else { - if (tocWriter != null) { + if (tocWriter != null && eventId != null) { tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId); } @@ -136,33 +146,34 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { } } - + protected synchronized void ensureStreamState(final long recordIdentifier, final long startBytes) throws IOException { + // add a new block to the TOC if needed. + if (getTocWriter() != null && (startBytes - blockStartOffset >= uncompressedBlockSize)) { + blockStartOffset = startBytes; + resetWriteStream(recordIdentifier); + } + } @Override - public long writeRecord(final ProvenanceEventRecord record, final long recordIdentifier) throws IOException { + public synchronized StorageSummary writeRecord(final ProvenanceEventRecord record) throws IOException { if (isDirty()) { throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository"); } try { + final long recordIdentifier = record.getEventId() == -1L ? idGenerator.getAndIncrement() : record.getEventId(); final long startBytes = byteCountingOut.getBytesWritten(); - // add a new block to the TOC if needed. - if (getTocWriter() != null && (startBytes - lastBlockOffset >= uncompressedBlockSize)) { - lastBlockOffset = startBytes; - - if (compressed) { - // because of the way that GZIPOutputStream works, we need to call close() on it in order for it - // to write its trailing bytes. But we don't want to close the underlying OutputStream, so we wrap - // the underlying OutputStream in a NonCloseableOutputStream - resetWriteStream(recordIdentifier); - } - } - + ensureStreamState(recordIdentifier, startBytes); writeRecord(record, recordIdentifier, out); recordCount++; - return byteCountingOut.getBytesWritten() - startBytes; + final long bytesWritten = byteCountingOut.getBytesWritten(); + final long serializedLength = bytesWritten - startBytes; + final TocWriter tocWriter = getTocWriter(); + final Integer blockIndex = tocWriter == null ? null : tocWriter.getCurrentBlockIndex(); + final String storageLocation = getStorageLocation(); + return new StorageSummary(recordIdentifier, storageLocation, blockIndex, serializedLength, bytesWritten); } catch (final IOException ioe) { markDirty(); throw ioe; @@ -170,7 +181,12 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { } @Override - public void flush() throws IOException { + public synchronized long getBytesWritten() { + return byteCountingOut == null ? 0L : byteCountingOut.getBytesWritten(); + } + + @Override + public synchronized void flush() throws IOException { out.flush(); } @@ -180,22 +196,26 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter { } @Override - protected OutputStream getBufferedOutputStream() { + protected synchronized DataOutputStream getBufferedOutputStream() { return out; } @Override - protected OutputStream getUnderlyingOutputStream() { + protected synchronized OutputStream getUnderlyingOutputStream() { return fos; } @Override - protected void syncUnderlyingOutputStream() throws IOException { + protected synchronized void syncUnderlyingOutputStream() throws IOException { if (fos != null) { fos.getFD().sync(); } } + protected boolean isCompressed() { + return compressed; + } + protected abstract void writeRecord(final ProvenanceEventRecord event, final long eventId, final DataOutputStream out) throws IOException; protected abstract void writeHeader(final long firstEventId, final DataOutputStream out) throws IOException; diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/EmptyRecordReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/EmptyRecordReader.java index 38a4cc902a..d4487ab2e5 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/EmptyRecordReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/EmptyRecordReader.java @@ -18,7 +18,9 @@ package org.apache.nifi.provenance.serialization; import java.io.IOException; +import java.util.Optional; +import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.apache.nifi.provenance.toc.TocReader; @@ -69,4 +71,14 @@ public class EmptyRecordReader implements RecordReader { public long getMaxEventId() throws IOException { return 0; } + + @Override + public Optional skipToEvent(long eventId) throws IOException { + return Optional.empty(); + } + + @Override + public boolean isData() { + return false; + } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/EventFileCompressor.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/EventFileCompressor.java new file mode 100644 index 0000000000..4814c95d7b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/EventFileCompressor.java @@ -0,0 +1,188 @@ +/* + * 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.serialization; + +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.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.provenance.store.EventFileManager; +import org.apache.nifi.provenance.toc.StandardTocReader; +import org.apache.nifi.provenance.toc.StandardTocWriter; +import org.apache.nifi.provenance.toc.TocReader; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.toc.TocWriter; +import org.apache.nifi.provenance.util.CloseableUtil; +import org.apache.nifi.stream.io.ByteCountingOutputStream; +import org.apache.nifi.stream.io.GZIPOutputStream; +import org.apache.nifi.stream.io.NonCloseableOutputStream; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.util.FormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + *

+ * This class is responsible for compressing Event Files as a background task. This is done as a background task instead of being + * done inline because if compression is performed inline, whenever NiFi is restarted (especially if done so abruptly), it is very + * possible that the GZIP stream will be corrupt. As a result, we would stand to lose some Provenance Events when NiFi is restarted. + * In order to avoid that, we write data in an uncompressed format and then compress the data in the background. Once the data has + * been compressed, this task will then remove the original, uncompressed file. If the file is being read by another thread, this + * task will wait for the other thread to finish reading the data before deleting the file. This synchronization of the File is handled + * via the {@link EventFileManager Event File Manager}. + *

+ */ +public class EventFileCompressor implements Runnable { + private static final Logger logger = LoggerFactory.getLogger(EventFileCompressor.class); + private final BlockingQueue filesToCompress; + private final EventFileManager eventFileManager; + private volatile boolean shutdown = false; + + public EventFileCompressor(final BlockingQueue filesToCompress, final EventFileManager eventFileManager) { + this.filesToCompress = filesToCompress; + this.eventFileManager = eventFileManager; + } + + public void shutdown() { + shutdown = true; + } + + @Override + public void run() { + while (!shutdown) { + File uncompressedEventFile = null; + + try { + final long start = System.nanoTime(); + uncompressedEventFile = filesToCompress.poll(1, TimeUnit.SECONDS); + if (uncompressedEventFile == null || shutdown) { + continue; + } + + File outputFile = null; + long bytesBefore = 0L; + StandardTocReader tocReader = null; + + File tmpTocFile = null; + eventFileManager.obtainReadLock(uncompressedEventFile); + try { + StandardTocWriter tocWriter = null; + + final File tocFile = TocUtil.getTocFile(uncompressedEventFile); + try { + tocReader = new StandardTocReader(tocFile); + } catch (final IOException e) { + logger.error("Failed to read TOC File {}", tocFile, e); + continue; + } + + bytesBefore = uncompressedEventFile.length(); + + try { + outputFile = new File(uncompressedEventFile.getParentFile(), uncompressedEventFile.getName() + ".gz"); + try { + tmpTocFile = new File(tocFile.getParentFile(), tocFile.getName() + ".tmp"); + tocWriter = new StandardTocWriter(tmpTocFile, true, false); + compress(uncompressedEventFile, tocReader, outputFile, tocWriter); + tocWriter.close(); + } catch (final IOException ioe) { + logger.error("Failed to compress {} on rollover", uncompressedEventFile, ioe); + } + } finally { + CloseableUtil.closeQuietly(tocReader, tocWriter); + } + } finally { + eventFileManager.releaseReadLock(uncompressedEventFile); + } + + eventFileManager.obtainWriteLock(uncompressedEventFile); + try { + // Attempt to delete the input file and associated toc file + if (uncompressedEventFile.delete()) { + if (tocReader != null) { + final File tocFile = tocReader.getFile(); + if (!tocFile.delete()) { + logger.warn("Failed to delete {}; this file should be cleaned up manually", tocFile); + } + + if (tmpTocFile != null) { + tmpTocFile.renameTo(tocFile); + } + } + } else { + logger.warn("Failed to delete {}; this file should be cleaned up manually", uncompressedEventFile); + } + } finally { + eventFileManager.releaseWriteLock(uncompressedEventFile); + } + + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + final long bytesAfter = outputFile.length(); + final double reduction = 100 * (1 - (double) bytesAfter / (double) bytesBefore); + final String reductionTwoDecimals = String.format("%.2f", reduction); + logger.debug("Successfully compressed Provenance Event File {} in {} millis from {} to {}, a reduction of {}%", + uncompressedEventFile, millis, FormatUtils.formatDataSize(bytesBefore), FormatUtils.formatDataSize(bytesAfter), reductionTwoDecimals); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + return; + } catch (final Exception e) { + logger.error("Failed to compress {}", uncompressedEventFile, e); + } + } + } + + public static void compress(final File input, final TocReader tocReader, final File output, final TocWriter tocWriter) throws IOException { + try (final InputStream fis = new FileInputStream(input); + final OutputStream fos = new FileOutputStream(output); + final ByteCountingOutputStream byteCountingOut = new ByteCountingOutputStream(fos)) { + + int blockIndex = 0; + while (true) { + // Determine the min and max byte ranges for the current block. + final long blockStart = tocReader.getBlockOffset(blockIndex); + if (blockStart == -1) { + break; + } + + long blockEnd = tocReader.getBlockOffset(blockIndex + 1); + if (blockEnd < 0) { + blockEnd = input.length(); + } + + final long firstEventId = tocReader.getFirstEventIdForBlock(blockIndex); + final long blockStartOffset = byteCountingOut.getBytesWritten(); + + try (final OutputStream ncos = new NonCloseableOutputStream(byteCountingOut); + final OutputStream gzipOut = new GZIPOutputStream(ncos, 1)) { + StreamUtils.copy(fis, gzipOut, blockEnd - blockStart); + } + + tocWriter.addBlockOffset(blockStartOffset, firstEventId); + blockIndex++; + } + } + + // Close the TOC Reader and TOC Writer + CloseableUtil.closeQuietly(tocReader, tocWriter); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReader.java index 91c8222ea8..9377f2c785 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReader.java @@ -18,7 +18,9 @@ package org.apache.nifi.provenance.serialization; import java.io.Closeable; import java.io.IOException; +import java.util.Optional; +import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.apache.nifi.provenance.toc.TocReader; @@ -50,13 +52,25 @@ public interface RecordReader extends Closeable { /** * Skips to the specified compression block * - * @param blockIndex the byte index to skip to + * @param blockIndex the block index to skip to * @throws IOException if the underlying stream throws IOException, or if the reader has already * read passed the specified compression block index * @throws IllegalStateException if the RecordReader does not have a TableOfContents associated with it */ void skipToBlock(int blockIndex) throws IOException; + /** + * Skips to the first event in the stream with an Event ID >= the given ID. If no event is found with an + * ID >= the given ID an empty Optional is returned. Otherwise, an Optional containing the first event in the stream with an + * ID >= the given ID is returned. Unlike {@link #nextRecord()}, this method does not consume the returned event from the stream. + * I.e., if a record is returned, that same record will be returned again the next time that {@link #nextRecord()} is called. + * + * @param eventId the ID of the event to retrieve + * @return the first event in the stream with an Event ID >= the given ID or an empty Optional if no such event can be found + * @throws IOException if the underlying stream throws IOException + */ + Optional skipToEvent(long eventId) throws IOException; + /** * Returns the block index that the Reader is currently reading from. * Note that the block index is incremented at the beginning of the {@link #nextRecord()} @@ -100,4 +114,11 @@ public interface RecordReader extends Closeable { * @throws IOException if unable to get id of the last event */ long getMaxEventId() throws IOException; + + /** + * Returns true if there is more data for hte Record Reader to read, false otherwise. + * + * @return true if there is more data for hte Record Reader to read, false otherwise. + */ + boolean isData(); } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java index 526a488cdf..8e79ddd7fc 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordReaders.java @@ -30,8 +30,9 @@ import java.util.zip.GZIPInputStream; import org.apache.nifi.provenance.ByteArraySchemaRecordReader; import org.apache.nifi.provenance.ByteArraySchemaRecordWriter; +import org.apache.nifi.provenance.EventIdFirstSchemaRecordReader; +import org.apache.nifi.provenance.EventIdFirstSchemaRecordWriter; import org.apache.nifi.provenance.StandardRecordReader; -import org.apache.nifi.provenance.StandardRecordWriter; import org.apache.nifi.provenance.lucene.LuceneUtil; import org.apache.nifi.provenance.toc.StandardTocReader; import org.apache.nifi.provenance.toc.TocReader; @@ -78,10 +79,10 @@ public class RecordReaders { String filename = file.getName(); openStream: while ( fis == null ) { final File dir = file.getParentFile(); - final String baseName = LuceneUtil.substringBefore(file.getName(), "."); + final String baseName = LuceneUtil.substringBefore(file.getName(), ".prov"); - // depending on which rollover actions have occurred, we could have 3 possibilities for the - // filename that we need. The majority of the time, we will use the extension ".prov.indexed.gz" + // depending on which rollover actions have occurred, we could have 2 possibilities for the + // filename that we need. The majority of the time, we will use the extension ".prov.gz" // because most often we are compressing on rollover and most often we have already finished // compressing by the time that we are querying the data. for ( final String extension : new String[] {".prov.gz", ".prov"} ) { @@ -123,7 +124,7 @@ public class RecordReaders { } switch (serializationName) { - case StandardRecordWriter.SERIALIZATION_NAME: { + case StandardRecordReader.SERIALIZATION_NAME: { if (tocFile.exists()) { final TocReader tocReader = new StandardTocReader(tocFile); return new StandardRecordReader(bufferedInStream, filename, tocReader, maxAttributeChars); @@ -139,6 +140,14 @@ public class RecordReaders { return new ByteArraySchemaRecordReader(bufferedInStream, filename, maxAttributeChars); } } + case EventIdFirstSchemaRecordWriter.SERIALIZATION_NAME: { + if (!tocFile.exists()) { + throw new FileNotFoundException("Cannot create TOC Reader because the file " + tocFile + " does not exist"); + } + + final TocReader tocReader = new StandardTocReader(tocFile); + return new EventIdFirstSchemaRecordReader(bufferedInStream, filename, tocReader, maxAttributeChars); + } default: { throw new IOException("Unable to read data from file " + file + " because the file was written using an unknown Serializer: " + serializationName); } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriter.java index 17dd75c02d..c9d2a22adf 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriter.java @@ -37,11 +37,10 @@ public interface RecordWriter extends Closeable { * Writes the given record out to the underlying stream * * @param record the record to write - * @param recordIdentifier the new identifier of the record * @return the number of bytes written for the given records * @throws IOException if unable to write the record to the stream */ - long writeRecord(ProvenanceEventRecord record, long recordIdentifier) throws IOException; + StorageSummary writeRecord(ProvenanceEventRecord record) throws IOException; /** * Flushes any data that is held in a buffer to the underlying storage mechanism @@ -55,6 +54,11 @@ public interface RecordWriter extends Closeable { */ int getRecordsWritten(); + /** + * @return the number of bytes written to this writer + */ + long getBytesWritten(); + /** * @return the file that this RecordWriter is writing to */ @@ -88,6 +92,11 @@ public interface RecordWriter extends Closeable { */ void markDirty(); + /** + * @return true if {@link #markDirty()} has been called, false otherwise + */ + boolean isDirty(); + /** * Syncs the content written to this writer to disk. * diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriters.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriters.java index be4c9cf7f5..cacaebdff0 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriters.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/RecordWriters.java @@ -18,6 +18,7 @@ package org.apache.nifi.provenance.serialization; import java.io.File; import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; import org.apache.nifi.provenance.ByteArraySchemaRecordWriter; import org.apache.nifi.provenance.toc.StandardTocWriter; @@ -27,13 +28,14 @@ import org.apache.nifi.provenance.toc.TocWriter; public class RecordWriters { private static final int DEFAULT_COMPRESSION_BLOCK_SIZE = 1024 * 1024; // 1 MB - public static RecordWriter newSchemaRecordWriter(final File file, final boolean compressed, final boolean createToc) throws IOException { - return newSchemaRecordWriter(file, compressed, createToc, DEFAULT_COMPRESSION_BLOCK_SIZE); + public static RecordWriter newSchemaRecordWriter(final File file, final AtomicLong idGenerator, final boolean compressed, final boolean createToc) throws IOException { + return newSchemaRecordWriter(file, idGenerator, compressed, createToc, DEFAULT_COMPRESSION_BLOCK_SIZE); } - public static RecordWriter newSchemaRecordWriter(final File file, final boolean compressed, final boolean createToc, final int compressionBlockBytes) throws IOException { + public static RecordWriter newSchemaRecordWriter(final File file, final AtomicLong idGenerator, final boolean compressed, final boolean createToc, + final int compressionBlockBytes) throws IOException { final TocWriter tocWriter = createToc ? new StandardTocWriter(TocUtil.getTocFile(file), false, false) : null; - return new ByteArraySchemaRecordWriter(file, tocWriter, compressed, compressionBlockBytes); + return new ByteArraySchemaRecordWriter(file, idGenerator, tocWriter, compressed, compressionBlockBytes); } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/StorageSummary.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/StorageSummary.java new file mode 100644 index 0000000000..dffcd6d8ed --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/serialization/StorageSummary.java @@ -0,0 +1,72 @@ +/* + * 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.serialization; + +import java.util.Optional; + +public class StorageSummary { + private final long eventId; + private final String storageLocation; + private final String partitionName; + private final Integer blockIndex; + private final long serializedLength; + private final long bytesWritten; + + public StorageSummary(final long eventId, final String storageLocation, final Integer blockIndex, final long serializedLength, final long bytesWritten) { + this(eventId, storageLocation, null, blockIndex, serializedLength, bytesWritten); + } + + public StorageSummary(final long eventId, final String storageLocation, final String partitionName, + final Integer blockIndex, final long serializedLength, final long bytesWritten) { + this.eventId = eventId; + this.storageLocation = storageLocation; + this.partitionName = partitionName; + this.blockIndex = blockIndex; + this.serializedLength = serializedLength; + this.bytesWritten = bytesWritten; + } + + public long getEventId() { + return eventId; + } + + public String getStorageLocation() { + return storageLocation; + } + + public Optional getPartitionName() { + return Optional.ofNullable(partitionName); + } + + public Integer getBlockIndex() { + return blockIndex; + } + + public long getSerializedLength() { + return serializedLength; + } + + public long getBytesWritten() { + return bytesWritten; + } + + @Override + public String toString() { + return "StorageSummary[eventId=" + getEventId() + ", partition=" + getPartitionName().orElse(null) + ", location=" + getStorageLocation() + "]"; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventFileManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventFileManager.java new file mode 100644 index 0000000000..3754113a0b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventFileManager.java @@ -0,0 +1,109 @@ +/* + * 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.store; + +import java.io.File; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Function; + +import org.apache.nifi.provenance.lucene.LuceneUtil; +import org.apache.nifi.util.Tuple; + +/** + * The EventFileManager is responsible for maintaining locks on Event Files so that we can ensure that no thread deletes + * an Event File while it is still being read. Without this manager, this could happen, for instance, if the Compression Thread + * were to compress an Event File, and then delete the original/uncompressed version while a Provenance Query was reading the + * uncompressed version of the file. + */ +public class EventFileManager { + + private final ConcurrentMap> lockMap = new ConcurrentHashMap<>(); + + private String getMapKey(final File file) { + return LuceneUtil.substringBefore(file.getName(), ".prov"); + } + + private ReadWriteLock updateCount(final File file, final Function update) { + final String key = getMapKey(file); + boolean updated = false; + + Tuple updatedTuple = null; + while (!updated) { + final Tuple tuple = lockMap.computeIfAbsent(key, k -> new Tuple<>(new ReentrantReadWriteLock(), 0)); + final Integer updatedCount = update.apply(tuple.getValue()); + updatedTuple = new Tuple<>(tuple.getKey(), updatedCount); + updated = lockMap.replace(key, tuple, updatedTuple); + } + + return updatedTuple.getKey(); + } + + private ReadWriteLock incrementCount(final File file) { + return updateCount(file, val -> val + 1); + } + + private ReadWriteLock decrementCount(final File file) { + return updateCount(file, val -> val - 1); + } + + + public void obtainReadLock(final File file) { + final ReadWriteLock rwLock = incrementCount(file); + rwLock.readLock().lock(); + } + + public void releaseReadLock(final File file) { + final ReadWriteLock rwLock = decrementCount(file); + rwLock.readLock().unlock(); + } + + public void obtainWriteLock(final File file) { + final ReadWriteLock rwLock = incrementCount(file); + rwLock.writeLock().lock(); + } + + public void releaseWriteLock(final File file) { + final String key = getMapKey(file); + + boolean updated = false; + while (!updated) { + final Tuple tuple = lockMap.get(key); + if (tuple == null) { + throw new IllegalMonitorStateException("Lock is not owned"); + } + + // If this is the only reference to the lock, remove it from the map and then unlock. + if (tuple.getValue() <= 1) { + updated = lockMap.remove(key, tuple); + if (updated) { + tuple.getKey().writeLock().unlock(); + } + } else { + final Tuple updatedTuple = new Tuple<>(tuple.getKey(), tuple.getValue() - 1); + updated = lockMap.replace(key, tuple, updatedTuple); + if (updated) { + tuple.getKey().writeLock().unlock(); + } + } + } + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStore.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStore.java new file mode 100644 index 0000000000..ba4acea974 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStore.java @@ -0,0 +1,123 @@ +/* + * 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.store; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.EventTransformer; +import org.apache.nifi.provenance.index.EventIndex; +import org.apache.nifi.provenance.serialization.StorageSummary; + +/** + *

+ * An Event Store is responsible for storing Provenance Events and retrieving them at a later time. + *

+ */ +public interface EventStore extends Closeable { + + /** + * Performs any initialization routines that need to happen before the store is used + * + * @throws IOException if unable to perform initialization + */ + void initialize() throws IOException; + + /** + * Adds the given events to the store. All events will be written to the same Storage Location. + * I.e., all of the {@link StorageSummary} objects that are provided when calling the {@link StorageResult#getStorageLocations()} + * method will return the same value for the {@link StorageSummary#getStorageLocation()}. Each one, however, will + * have a different Event ID and potentially a different Block Index. + * + * @param events the events to add + * @return a mapping of event to the location where it was stored + * @throws IOException if unable to add the events + */ + StorageResult addEvents(Iterable events) throws IOException; + + /** + * @return the number of bytes occupied by the events in the store + * @throws IOException if unable to determine the size of the store + */ + long getSize() throws IOException; + + /** + * @return the largest Event ID that has been written to this store, or -1 if no events have yet been stored. + */ + long getMaxEventId(); + + /** + * Retrieves the event with the given ID + * + * @param id the ID of the event to retrieve + * @return an Optional containing the Event with the given ID, or an empty optional if the event cannot be found + * @throws IOException if unable to read the event from storage + */ + Optional getEvent(long id) throws IOException; + + /** + * Retrieves up to maxRecords events from the store, starting with the event whose ID is equal to firstRecordId. If that + * event cannot be found, then the first event will be the oldest event in the store whose ID is greater than firstRecordId. + * All events will be returned in the order that they were written to the store. I.e., all events will have monotonically + * increasing Event ID's. No events will be filtered out, since there is no EventAuthorizer provided. + * + * @param firstRecordId the ID of the first event to retrieve + * @param maxRecords the maximum number of records to retrieve. The actual number of results returned may be less than this. + * @return a List of ProvenanceEventRecord's + * @throws IOException if unable to retrieve records from the store + */ + List getEvents(long firstRecordId, int maxRecords) throws IOException; + + /** + * Retrieves up to maxRecords events from the store, starting with the event whose ID is equal to firstRecordId. If that + * event cannot be found, then the first event will be the oldest event in the store whose ID is greater than firstRecordId. + * All events will be returned in the order that they were written to the store. I.e., all events will have monotonically + * increasing Event ID's. + * + * @param firstRecordId the ID of the first event to retrieve + * @param maxRecords the maximum number of records to retrieve. The actual number of results returned may be less than this. + * @param authorizer the authorizer that should be used to filter out any events that the user doesn't have access to + * @param unauthorizedTransformer the transformer to apply to unauthorized events + * @return a List of ProvenanceEventRecord's + * @throws IOException if unable to retrieve records from the store + */ + List getEvents(long firstRecordId, int maxRecords, EventAuthorizer authorizer, EventTransformer unauthorizedTransformer) throws IOException; + + /** + * Given a List of Event ID's, returns a List of Provenance Events that contain the events that have those corresponding + * Event ID's. If any events cannot be found, a warning will be logged but no Exception will be thrown. + * + * @param eventIds a Stream of Event ID's + * @param authorizer the authorizer that should be used to filter out any events that the user doesn't have access to + * @param unauthorizedTransformer the transformer to apply to unauthorized events + * @return a List of events that correspond to the given ID's + * @throws IOException if unable to retrieve records from the store + */ + List getEvents(List eventIds, EventAuthorizer authorizer, EventTransformer unauthorizedTransformer) throws IOException; + + /** + * Causes the latest events in this store to be re-indexed by the given Event Index + * + * @param eventIndex the EventIndex to use for indexing events + */ + void reindexLatestEvents(EventIndex eventIndex); +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStorePartition.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStorePartition.java new file mode 100644 index 0000000000..ccb94f821b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/EventStorePartition.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.store; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.store.iterator.EventIterator; + +public interface EventStorePartition extends Closeable { + /** + * Performs any initialization routines that need to happen before the store is used + * + * @throws IOException if unable to perform initialization + */ + void initialize() throws IOException; + + /** + * Adds the given events to the store + * + * @param events the events to add + * @return a mapping of event to the location where it was stored + * @throws IOException if unable to add the events + */ + StorageResult addEvents(Iterable events) throws IOException; + + /** + * @return the number of bytes occupied by the events in the store + */ + long getSize(); + + /** + * @return the largest Event ID that has been written to this store, or -1 if no events have yet been stored. + */ + long getMaxEventId(); + + /** + * Retrieves the event with the given ID + * + * @param id the ID of the event to retrieve + * @return the Event with the given ID, or null if the event cannot be found + * @throws IOException if unable to read the event from storage + */ + Optional getEvent(long id) throws IOException; + + /** + * Retrieves up to maxRecords events from the store, starting with the event whose ID is equal to firstRecordId. If that + * event cannot be found, then the first event will be the oldest event in the store whose ID is greater than firstRecordId. + * All events will be returned in the order that they were written to the store. I.e., all events will have monotonically + * increasing Event ID's. + * + * @param firstRecordId the ID of the first event to retrieve + * @param maxEvents the maximum number of events to retrieve. The actual number of results returned may be less than this. + * @param authorizer the authorizer that should be used to filter out any events that the user doesn't have access to + * @return a List of ProvenanceEventRecord's + * @throws IOException if unable to retrieve records from the store + */ + List getEvents(long firstRecordId, int maxEvents, EventAuthorizer authorizer) throws IOException; + + /** + * Returns an {@link EventIterator} that is capable of iterating over the events in the store beginning with the given + * record id. The events returned by the EventIterator will be provided in the order in which they were stored in the + * partition. All events retrieved from this EventIterator will have monotonically increasing Event ID's. + * + * @param minimumEventId the minimum value of any Event ID that should be returned + * @return an EventIterator that is capable of iterating over events in the store + */ + EventIterator createEventIterator(long minimumEventId); + + /** + * Returns an {@link EventIterator} that iterates over the given event ID's and returns one ProvenanceEventRecord for + * each given, if the ID given can be found. If a given ID cannot be found, it will be skipped and no error will be reported. + * + * @param eventIds the ID's of the events to retrieve + * @return an EventIterator that iterates over the given event ID's + */ + EventIterator createEventIterator(List eventIds); + + /** + * Purges any events from the partition that are older than the given amount of time + * + * @param olderThan the amount of time for which any event older than this should be removed + * @param timeUnit the unit of time that applies to the first argument + */ + void purgeOldEvents(long olderThan, TimeUnit timeUnit); + + /** + * Purges some number of events from the partition. The oldest events will be purged. + * + * @return the number of bytes purged from the partition + */ + long purgeOldestEvents(); +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/PartitionedEventStore.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/PartitionedEventStore.java new file mode 100644 index 0000000000..5f922dd43b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/PartitionedEventStore.java @@ -0,0 +1,284 @@ +/* + * 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.store; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; + +import org.apache.lucene.util.NamedThreadFactory; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.EventTransformer; +import org.apache.nifi.provenance.store.iterator.AuthorizingEventIterator; +import org.apache.nifi.provenance.store.iterator.EventIterator; +import org.apache.nifi.provenance.util.DirectoryUtils; +import org.apache.nifi.reporting.Severity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class PartitionedEventStore implements EventStore { + private static final Logger logger = LoggerFactory.getLogger(PartitionedEventStore.class); + private static final String EVENT_CATEGORY = "Provenance Repository"; + + private final AtomicLong partitionIndex = new AtomicLong(0L); + private final RepositoryConfiguration repoConfig; + private final EventReporter eventReporter; + private ScheduledExecutorService maintenanceExecutor; + + public PartitionedEventStore(final RepositoryConfiguration config, final EventReporter eventReporter) { + this.repoConfig = config; + this.eventReporter = eventReporter; + } + + + @Override + public void initialize() throws IOException { + maintenanceExecutor = Executors.newScheduledThreadPool(1, new NamedThreadFactory("Provenance Repository Maintenance")); + maintenanceExecutor.scheduleWithFixedDelay(() -> performMaintenance(), 1, 1, TimeUnit.MINUTES); + + for (final EventStorePartition partition : getPartitions()) { + partition.initialize(); + } + } + + @Override + public void close() throws IOException { + if (maintenanceExecutor != null) { + maintenanceExecutor.shutdownNow(); + } + + IOException thrown = null; + + for (final EventStorePartition partition : getPartitions()) { + try { + partition.close(); + } catch (final IOException ioe) { + if (thrown == null) { + thrown = ioe; + } else { + thrown.addSuppressed(ioe); + } + } + } + + if (thrown != null) { + throw thrown; + } + } + + + @Override + public StorageResult addEvents(final Iterable events) throws IOException { + final List partitions = getPartitions(); + final int index = (int) (partitionIndex.getAndIncrement() % partitions.size()); + final EventStorePartition partition = partitions.get(index); + return partition.addEvents(events); + } + + @Override + public long getSize() { + long size = 0; + for (final EventStorePartition partition : getPartitions()) { + size += partition.getSize(); + } + + return size; + } + + private long getRepoSize() { + long total = 0L; + + for (final File storageDir : repoConfig.getStorageDirectories().values()) { + total += DirectoryUtils.getSize(storageDir); + } + + return total; + } + + @Override + public long getMaxEventId() { + return getPartitions().stream() + .mapToLong(part -> part.getMaxEventId()) + .max() + .orElse(-1L); + } + + @Override + public Optional getEvent(final long id) throws IOException { + for (final EventStorePartition partition : getPartitions()) { + final Optional option = partition.getEvent(id); + if (option.isPresent()) { + return option; + } + } + + return Optional.empty(); + } + + @Override + public List getEvents(final long firstRecordId, final int maxRecords) throws IOException { + return getEvents(firstRecordId, maxRecords, EventAuthorizer.GRANT_ALL, EventTransformer.EMPTY_TRANSFORMER); + } + + @Override + public List getEvents(final long firstRecordId, final int maxRecords, final EventAuthorizer authorizer, + final EventTransformer transformer) throws IOException { + if (firstRecordId + maxRecords < 1 || maxRecords < 1 || firstRecordId > getMaxEventId()) { + return Collections.emptyList(); + } + + return getEvents(maxRecords, authorizer, part -> part.createEventIterator(firstRecordId), transformer); + } + + @Override + public List getEvents(final List eventIds, final EventAuthorizer authorizer, final EventTransformer transformer) throws IOException { + if (eventIds == null || eventIds.isEmpty()) { + return Collections.emptyList(); + } + + return getEvents(eventIds.size(), authorizer, part -> part.createEventIterator(eventIds), transformer); + } + + private List getEvents(final int maxRecords, final EventAuthorizer authorizer, + final Function eventIteratorFactory, final EventTransformer transformer) throws IOException { + + if (maxRecords < 1) { + return Collections.emptyList(); + } + + final List selectedEvents = new ArrayList<>(); + + // Create a Map so that the key is the next record available from a partition and the value is the EventIterator from which + // the record came. This sorted map is then used so that we are able to always get the first entry, which is the next + // lowest record id among all partitions. + final SortedMap recordToIteratorMap = new TreeMap<>( + (o1, o2) -> Long.compare(o1.getEventId(), o2.getEventId())); + + try { + // Seed our map with the first event in each Partition. + for (final EventStorePartition partition : getPartitions()) { + final EventAuthorizer nonNullAuthorizer = authorizer == null ? EventAuthorizer.GRANT_ALL : authorizer; + final EventIterator partitionIterator = eventIteratorFactory.apply(partition); + final EventIterator iterator = new AuthorizingEventIterator(partitionIterator, nonNullAuthorizer, transformer); + + final Optional option = iterator.nextEvent(); + if (option.isPresent()) { + recordToIteratorMap.put(option.get(), iterator); + } + } + + // If no records found, just return the empty list. + if (recordToIteratorMap.isEmpty()) { + return selectedEvents; + } + + // Get the event with the next-lowest ID. Add it to the list of selected events, + // then read the next event from the same EventIterator that this event came from. + // This ensures that our map is always populated with the next event for each + // EventIterator, which also ensures that the first key in our map is the event + // with the lowest ID (since all events from a given EventIterator have monotonically + // increasing Event ID's). + ProvenanceEventRecord nextEvent = recordToIteratorMap.firstKey(); + while (nextEvent != null && selectedEvents.size() < maxRecords) { + selectedEvents.add(nextEvent); + + final EventIterator iterator = recordToIteratorMap.remove(nextEvent); + final Optional nextRecordFromIterator = iterator.nextEvent(); + if (nextRecordFromIterator.isPresent()) { + recordToIteratorMap.put(nextRecordFromIterator.get(), iterator); + } + + nextEvent = recordToIteratorMap.isEmpty() ? null : recordToIteratorMap.firstKey(); + } + + return selectedEvents; + } finally { + // Ensure that we close all record readers that have been created + for (final EventIterator iterator : recordToIteratorMap.values()) { + try { + iterator.close(); + } catch (final Exception e) { + if (logger.isDebugEnabled()) { + logger.warn("Failed to close Record Reader {}", iterator, e); + } else { + logger.warn("Failed to close Record Reader {}", iterator); + } + } + } + } + } + + + void performMaintenance() { + try { + final long maxFileLife = repoConfig.getMaxRecordLife(TimeUnit.MILLISECONDS); + for (final EventStorePartition partition : getPartitions()) { + try { + partition.purgeOldEvents(maxFileLife, TimeUnit.MILLISECONDS); + } catch (final Exception e) { + logger.error("Failed to purge expired events from " + partition, e); + eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, + "Failed to purge expired events from Provenance Repository. See logs for more information."); + } + } + + final long maxStorageCapacity = repoConfig.getMaxStorageCapacity(); + long currentSize; + try { + currentSize = getRepoSize(); + } catch (final Exception e) { + logger.error("Could not determine size of Provenance Repository. Will not expire any data due to storage limits", e); + eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Failed to determine size of Provenance Repository. " + + "No data will be expired due to storage limits at this time. See logs for more information."); + return; + } + + while (currentSize > maxStorageCapacity) { + for (final EventStorePartition partition : getPartitions()) { + try { + final long removed = partition.purgeOldestEvents(); + currentSize -= removed; + } catch (final Exception e) { + logger.error("Failed to purge oldest events from " + partition, e); + eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, + "Failed to purge oldest events from Provenance Repository. See logs for more information."); + } + } + } + } catch (final Exception e) { + logger.error("Failed to perform periodic maintenance", e); + eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, + "Failed to perform periodic maintenace for Provenance Repository. See logs for more information."); + } + } + + protected abstract List getPartitions(); +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/PartitionedWriteAheadEventStore.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/PartitionedWriteAheadEventStore.java new file mode 100644 index 0000000000..14de80eae0 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/PartitionedWriteAheadEventStore.java @@ -0,0 +1,142 @@ +/* + * 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.store; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.lucene.util.NamedThreadFactory; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.index.EventIndex; +import org.apache.nifi.provenance.serialization.EventFileCompressor; + +public class PartitionedWriteAheadEventStore extends PartitionedEventStore { + private final BlockingQueue filesToCompress; + private final List partitions; + private final RepositoryConfiguration repoConfig; + + private final ExecutorService compressionExecutor; + private final List fileCompressors = Collections.synchronizedList(new ArrayList<>()); + private final EventReporter eventReporter; + private final EventFileManager fileManager; + + public PartitionedWriteAheadEventStore(final RepositoryConfiguration repoConfig, final RecordWriterFactory recordWriterFactory, + final RecordReaderFactory recordReaderFactory, final EventReporter eventReporter, final EventFileManager fileManager) { + super(repoConfig, eventReporter); + this.repoConfig = repoConfig; + this.eventReporter = eventReporter; + this.filesToCompress = new LinkedBlockingQueue<>(100); + final AtomicLong idGenerator = new AtomicLong(0L); + this.partitions = createPartitions(repoConfig, recordWriterFactory, recordReaderFactory, idGenerator); + this.fileManager = fileManager; + + // Creates tasks to compress data on rollover + if (repoConfig.isCompressOnRollover()) { + compressionExecutor = Executors.newFixedThreadPool(repoConfig.getIndexThreadPoolSize(), new NamedThreadFactory("Compress Provenance Logs")); + } else { + compressionExecutor = null; + } + } + + private List createPartitions(final RepositoryConfiguration repoConfig, final RecordWriterFactory recordWriterFactory, + final RecordReaderFactory recordReaderFactory, final AtomicLong idGenerator) { + final Map storageDirectories = repoConfig.getStorageDirectories(); + final List partitions = new ArrayList<>(storageDirectories.size()); + + for (final Map.Entry entry : storageDirectories.entrySet()) { + // Need to ensure that the same partition directory always gets the same partition index. + // If we don't, then we will end up re-indexing the events from 1 index into another index, and + // this will result in a lot of duplicates (up to a million per index per restart). This is the reason + // that we use a partition name here based on the properties file. + final String partitionName = entry.getKey(); + final File storageDirectory = entry.getValue(); + partitions.add(new WriteAheadStorePartition(storageDirectory, partitionName, repoConfig, + recordWriterFactory, recordReaderFactory, filesToCompress, idGenerator, eventReporter)); + } + + return partitions; + } + + @Override + public void initialize() throws IOException { + if (repoConfig.isCompressOnRollover()) { + for (int i = 0; i < repoConfig.getIndexThreadPoolSize(); i++) { + final EventFileCompressor compressor = new EventFileCompressor(filesToCompress, fileManager); + compressionExecutor.submit(compressor); + fileCompressors.add(compressor); + } + } + + super.initialize(); + } + + @Override + public void close() throws IOException { + super.close(); + + for (final EventFileCompressor compressor : fileCompressors) { + compressor.shutdown(); + } + + if (compressionExecutor != null) { + compressionExecutor.shutdown(); + } + } + + @Override + public void reindexLatestEvents(final EventIndex eventIndex) { + final List partitions = getPartitions(); + final int numPartitions = partitions.size(); + + final List> futures = new ArrayList<>(numPartitions); + final ExecutorService executor = Executors.newFixedThreadPool(numPartitions); + + for (final WriteAheadStorePartition partition : partitions) { + futures.add(executor.submit(() -> partition.reindexLatestEvents(eventIndex))); + } + + executor.shutdown(); + for (final Future future : futures) { + try { + future.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Failed to re-index events because Thread was interrupted", e); + } catch (ExecutionException e) { + throw new RuntimeException("Failed to re-index events", e); + } + } + } + + @Override + protected List getPartitions() { + return partitions; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordReaderFactory.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordReaderFactory.java new file mode 100644 index 0000000000..ddb8165f8f --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordReaderFactory.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.store; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Collection; + +import org.apache.nifi.provenance.serialization.RecordReader; + +public interface RecordReaderFactory { + RecordReader newRecordReader(File file, Collection provenanceLogFiles, int maxAttributeChars) throws IOException; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordWriterFactory.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordWriterFactory.java new file mode 100644 index 0000000000..89da603516 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordWriterFactory.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.store; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.provenance.serialization.RecordWriter; + +public interface RecordWriterFactory { + RecordWriter createWriter(final File file, final AtomicLong idGenerator, final boolean compressed, final boolean createToc) throws IOException; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordWriterLease.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordWriterLease.java new file mode 100644 index 0000000000..8543d2b4db --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/RecordWriterLease.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.provenance.store; + +import org.apache.nifi.provenance.serialization.RecordWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RecordWriterLease { + private final Logger logger = LoggerFactory.getLogger(RecordWriterLease.class); + + private final RecordWriter writer; + private final long maxBytes; + private final int maxEvents; + private long usageCounter; + private boolean markedRollable = false; + private boolean closed = false; + + public RecordWriterLease(final RecordWriter writer, final long maxBytes) { + this(writer, maxBytes, Integer.MAX_VALUE); + } + + public RecordWriterLease(final RecordWriter writer, final long maxBytes, final int maxEvents) { + this.writer = writer; + this.maxBytes = maxBytes; + this.maxEvents = maxEvents; + } + + public RecordWriter getWriter() { + return writer; + } + + public synchronized boolean tryClaim() { + if (markedRollable || writer.isClosed() || writer.isDirty() || writer.getBytesWritten() >= maxBytes || writer.getRecordsWritten() >= maxEvents) { + return false; + } + + usageCounter++; + return true; + } + + public synchronized void relinquishClaim() { + usageCounter--; + + if (closed && usageCounter < 1) { + try { + writer.close(); + } catch (final Exception e) { + logger.warn("Failed to close " + writer, e); + } + } + } + + public synchronized boolean shouldRoll() { + if (markedRollable) { + return true; + } + + if (usageCounter < 1 && (writer.isClosed() || writer.isDirty() || writer.getBytesWritten() >= maxBytes || writer.getRecordsWritten() >= maxEvents)) { + markedRollable = true; + return true; + } + + return false; + } + + public synchronized void close() { + closed = true; + + if (usageCounter < 1) { + try { + writer.close(); + } catch (final Exception e) { + logger.warn("Failed to close " + writer, e); + } + } + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/StorageResult.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/StorageResult.java new file mode 100644 index 0000000000..94b1ece8d2 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/StorageResult.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.store; + +import java.util.Collections; +import java.util.Map; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.serialization.StorageSummary; + +public interface StorageResult { + /** + * @return a map of each Provenance Event Record to the location where it was stored + */ + Map getStorageLocations(); + + /** + * Indicates whether or not the storage of events triggered the store to roll over + * the storage location that it is storing data to + * + * @return true if the store rolled over to a new storage location, false otherwise + */ + boolean triggeredRollover(); + + /** + * @return the number of events that were stored in the storage location that was rolled over, or + * null if no storage locations were rolled over. + */ + Integer getEventsRolledOver(); + + public static StorageResult EMPTY = new StorageResult() { + @Override + public Map getStorageLocations() { + return Collections.emptyMap(); + } + + @Override + public boolean triggeredRollover() { + return false; + } + + @Override + public Integer getEventsRolledOver() { + return null; + } + + @Override + public String toString() { + return "StorageResult.EMPTY"; + } + }; +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/WriteAheadStorePartition.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/WriteAheadStorePartition.java new file mode 100644 index 0000000000..a25043a50f --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/WriteAheadStorePartition.java @@ -0,0 +1,637 @@ +/* + * 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.store; + +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.BlockingQueue; +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 java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.StandardProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.index.EventIndex; +import org.apache.nifi.provenance.serialization.RecordReader; +import org.apache.nifi.provenance.serialization.RecordWriter; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.provenance.store.iterator.EventIterator; +import org.apache.nifi.provenance.store.iterator.SelectiveRecordReaderEventIterator; +import org.apache.nifi.provenance.store.iterator.SequentialRecordReaderEventIterator; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.util.DirectoryUtils; +import org.apache.nifi.provenance.util.NamedThreadFactory; +import org.apache.nifi.util.FormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class WriteAheadStorePartition implements EventStorePartition { + private static final Logger logger = LoggerFactory.getLogger(WriteAheadStorePartition.class); + + + private final RepositoryConfiguration config; + private final File partitionDirectory; + private final String partitionName; + private final RecordWriterFactory recordWriterFactory; + private final RecordReaderFactory recordReaderFactory; + private final BlockingQueue filesToCompress; + private final AtomicLong idGenerator; + private final AtomicLong maxEventId = new AtomicLong(-1L); + private volatile boolean closed = false; + + private AtomicReference eventWriterLeaseRef = new AtomicReference<>(); + + private final SortedMap minEventIdToPathMap = new TreeMap<>(); // guarded by synchronizing on object + + public WriteAheadStorePartition(final File storageDirectory, final String partitionName, final RepositoryConfiguration repoConfig, final RecordWriterFactory recordWriterFactory, + final RecordReaderFactory recordReaderFactory, final BlockingQueue filesToCompress, final AtomicLong idGenerator, final EventReporter eventReporter) { + + this.partitionName = partitionName; + this.config = repoConfig; + this.idGenerator = idGenerator; + this.partitionDirectory = storageDirectory; + this.recordWriterFactory = recordWriterFactory; + this.recordReaderFactory = recordReaderFactory; + this.filesToCompress = filesToCompress; + } + + @Override + public void close() throws IOException { + closed = true; + + final RecordWriterLease lease = eventWriterLeaseRef.get(); + if (lease != null) { + lease.close(); + } + } + + @Override + public synchronized void initialize() throws IOException { + if (!partitionDirectory.exists()) { + Files.createDirectories(partitionDirectory.toPath()); + } + + final File[] files = partitionDirectory.listFiles(DirectoryUtils.EVENT_FILE_FILTER); + if (files == null) { + throw new IOException("Could not access files in the " + partitionDirectory + " directory"); + } + + // We need to determine what the largest Event ID is in this partition. To do this, we + // iterate over all files starting with the file that has the greatest ID, and try to find + // the largest Event ID in that file. Once we successfully determine the greatest Event ID + // in any one of the files, we are done, since we are iterating over the files in order of + // the Largest Event ID to the smallest. + long maxEventId = -1L; + final List fileList = Arrays.asList(files); + Collections.sort(fileList, DirectoryUtils.LARGEST_ID_FIRST); + for (final File file : fileList) { + try { + final RecordReader reader = recordReaderFactory.newRecordReader(file, Collections.emptyList(), Integer.MAX_VALUE); + final long eventId = reader.getMaxEventId(); + if (eventId > maxEventId) { + maxEventId = eventId; + break; + } + } catch (final IOException ioe) { + logger.warn("Could not read file {}; if this file contains Provenance Events, new events may be created with the same event identifiers", file, ioe); + } + } + + synchronized (minEventIdToPathMap) { + for (final File file : fileList) { + final long minEventId = DirectoryUtils.getMinId(file); + minEventIdToPathMap.put(minEventId, file); + } + } + + this.maxEventId.set(maxEventId); + + // If configured to compress, compress any files that are not yet compressed. + if (config.isCompressOnRollover()) { + final File[] uncompressedFiles = partitionDirectory.listFiles(f -> f.getName().endsWith(".prov")); + if (uncompressedFiles != null) { + for (final File file : uncompressedFiles) { + // If we have both a compressed file and an uncompressed file for the same .prov file, then + // we must have been in the process of compressing it when NiFi was restarted. Delete the partial + // .gz file and we will start compressing it again. + final File compressed = new File(file.getParentFile(), file.getName() + ".gz"); + if (compressed.exists()) { + compressed.delete(); + } + } + } + } + + // Update the ID Generator to the max of the ID Generator or maxEventId + final long nextPartitionId = maxEventId + 1; + final long updatedId = idGenerator.updateAndGet(curVal -> Math.max(curVal, nextPartitionId)); + logger.info("After recovering {}, next Event ID to be generated will be {}", partitionDirectory, updatedId); + } + + + @Override + public StorageResult addEvents(final Iterable events) throws IOException { + if (closed) { + throw new IOException(this + " is closed"); + } + + // Claim a Record Writer Lease so that we have a writer to persist the events to + boolean claimed = false; + RecordWriterLease lease = null; + while (!claimed) { + lease = getLease(); + claimed = lease.tryClaim(); + + if (claimed) { + break; + } + + if (lease.shouldRoll()) { + tryRollover(lease); + } + } + + // Add the events to the writer and ensure that we always + // relinquish the claim that we've obtained on the writer + Map storageMap; + final RecordWriter writer = lease.getWriter(); + try { + storageMap = addEvents(events, writer); + } finally { + lease.relinquishClaim(); + } + + // Roll over the writer if necessary + Integer eventsRolledOver = null; + final boolean shouldRoll = lease.shouldRoll(); + try { + if (shouldRoll && tryRollover(lease)) { + eventsRolledOver = writer.getRecordsWritten(); + } + } catch (final IOException ioe) { + logger.error("Updated {} but failed to rollover to a new Event File", this, ioe); + } + + final Integer rolloverCount = eventsRolledOver; + return new StorageResult() { + @Override + public Map getStorageLocations() { + return storageMap; + } + + @Override + public boolean triggeredRollover() { + return rolloverCount != null; + } + + @Override + public Integer getEventsRolledOver() { + return rolloverCount; + } + + @Override + public String toString() { + return getStorageLocations().toString(); + } + }; + } + + private RecordWriterLease getLease() throws IOException { + while (true) { + final RecordWriterLease lease = eventWriterLeaseRef.get(); + if (lease != null) { + return lease; + } + + if (tryRollover(null)) { + return eventWriterLeaseRef.get(); + } + } + } + + private synchronized boolean tryRollover(final RecordWriterLease lease) throws IOException { + if (!Objects.equals(lease, eventWriterLeaseRef.get())) { + return false; + } + + final long nextEventId = idGenerator.get(); + final File updatedEventFile = new File(partitionDirectory, nextEventId + ".prov"); + final RecordWriter updatedWriter = recordWriterFactory.createWriter(updatedEventFile, idGenerator, false, true); + final RecordWriterLease updatedLease = new RecordWriterLease(updatedWriter, config.getMaxEventFileCapacity(), config.getMaxEventFileCount()); + final boolean updated = eventWriterLeaseRef.compareAndSet(lease, updatedLease); + + if (updated) { + updatedWriter.writeHeader(nextEventId); + + synchronized (minEventIdToPathMap) { + minEventIdToPathMap.put(nextEventId, updatedEventFile); + } + + if (config.isCompressOnRollover() && lease != null && lease.getWriter() != null) { + boolean offered = false; + while (!offered && !closed) { + try { + offered = filesToCompress.offer(lease.getWriter().getFile(), 1, TimeUnit.SECONDS); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while waiting to enqueue " + lease.getWriter().getFile() + " for compression"); + } + } + } + + return true; + } else { + try { + updatedWriter.close(); + } catch (final Exception e) { + logger.warn("Failed to close Record Writer {}; some resources may not be cleaned up properly.", updatedWriter, e); + } + + updatedEventFile.delete(); + return false; + } + } + + private Map addEvents(final Iterable events, final RecordWriter writer) throws IOException { + final Map locationMap = new HashMap<>(); + + try { + long maxId = -1L; + int numEvents = 0; + for (final ProvenanceEventRecord nextEvent : events) { + final StorageSummary writerSummary = writer.writeRecord(nextEvent); + final StorageSummary summaryWithIndex = new StorageSummary(writerSummary.getEventId(), writerSummary.getStorageLocation(), this.partitionName, + writerSummary.getBlockIndex(), writerSummary.getSerializedLength(), writerSummary.getBytesWritten()); + locationMap.put(nextEvent, summaryWithIndex); + maxId = summaryWithIndex.getEventId(); + numEvents++; + } + + if (numEvents == 0) { + return locationMap; + } + + writer.flush(); + + // Update max event id to be equal to be the greater of the current value or the + // max value just written. + final long maxIdWritten = maxId; + this.maxEventId.getAndUpdate(cur -> maxIdWritten > cur ? maxIdWritten : cur); + + if (config.isAlwaysSync()) { + writer.sync(); + } + } catch (final Exception e) { + // We need to set the repoDirty flag before we release the lock for this journal. + // Otherwise, another thread may write to this journal -- this is a problem because + // the journal contains part of our record but not all of it. Writing to the end of this + // journal will result in corruption! + writer.markDirty(); + throw e; + } + + return locationMap; + } + + + @Override + public long getSize() { + return getEventFilesFromDisk() + .collect(Collectors.summarizingLong(file -> file.length())) + .getSum(); + } + + private Stream getEventFilesFromDisk() { + final File[] files = partitionDirectory.listFiles(DirectoryUtils.EVENT_FILE_FILTER); + return files == null ? Stream.empty() : Arrays.stream(files); + } + + @Override + public long getMaxEventId() { + return maxEventId.get(); + } + + @Override + public Optional getEvent(final long id) throws IOException { + final Optional option = getPathForEventId(id); + if (!option.isPresent()) { + return Optional.empty(); + } + + try (final RecordReader reader = recordReaderFactory.newRecordReader(option.get(), Collections.emptyList(), config.getMaxAttributeChars())) { + final Optional eventOption = reader.skipToEvent(id); + if (!eventOption.isPresent()) { + return eventOption; + } + + // If an event is returned, the event may be the one we want, or it may be an event with a + // higher event ID, if the desired event is not in the record reader. So we need to get the + // event and check the Event ID to know whether to return the empty optional or the Optional + // that was returned. + final ProvenanceEventRecord event = eventOption.get(); + if (event.getEventId() == id) { + return eventOption; + } else { + return Optional.empty(); + } + } + } + + @Override + public List getEvents(final long firstRecordId, final int maxEvents, final EventAuthorizer authorizer) throws IOException { + final List events = new ArrayList<>(Math.min(maxEvents, 1000)); + try (final EventIterator iterator = createEventIterator(firstRecordId)) { + Optional eventOption; + while ((eventOption = iterator.nextEvent()).isPresent() && events.size() < maxEvents) { + final ProvenanceEventRecord event = eventOption.get(); + if (authorizer.isAuthorized(event)) { + events.add(event); + } + } + } + + return events; + } + + @Override + public EventIterator createEventIterator(final long minDesiredId) { + final List filesOfInterest = new ArrayList<>(); + synchronized (minEventIdToPathMap) { + File lastFile = null; + + for (final Map.Entry entry : minEventIdToPathMap.entrySet()) { + final long minFileId = entry.getKey(); + + // If the minimum ID for the file is greater than the minDesiredId, then + // that means that we will want to iterate over this file. + if (minFileId > minDesiredId) { + // The minimum ID for this file is greater than the desired ID, so + // that means that the last file we saw may have the minimum desired + // ID and any number of more events before we get to this file. So + // if we've not already added the lastFile, add it now. + if (filesOfInterest.isEmpty() && lastFile != null) { + filesOfInterest.add(lastFile); + } + + filesOfInterest.add(entry.getValue()); + } + + lastFile = entry.getValue(); + } + + // We don't know the max ID of the last file, so we always want to include it, since it may contain + // an event with an ID greater than minDesiredId. + if (lastFile != null && !filesOfInterest.contains(lastFile)) { + filesOfInterest.add(lastFile); + } + } + + if (filesOfInterest.isEmpty()) { + return EventIterator.EMPTY; + } + + return new SequentialRecordReaderEventIterator(filesOfInterest, recordReaderFactory, minDesiredId, config.getMaxAttributeChars()); + } + + + @Override + public EventIterator createEventIterator(final List eventIds) { + final List allFiles; + synchronized (minEventIdToPathMap) { + allFiles = new ArrayList<>(minEventIdToPathMap.values()); + } + + if (allFiles.isEmpty()) { + return EventIterator.EMPTY; + } + + return new SelectiveRecordReaderEventIterator(allFiles, recordReaderFactory, eventIds, config.getMaxAttributeChars()); + } + + private Optional getPathForEventId(final long id) { + File lastFile = null; + + synchronized (minEventIdToPathMap) { + for (final Map.Entry entry : minEventIdToPathMap.entrySet()) { + final long minId = entry.getKey(); + if (minId > id) { + break; + } + + lastFile = entry.getValue(); + } + } + + return Optional.ofNullable(lastFile); + } + + + @Override + public void purgeOldEvents(final long olderThan, final TimeUnit unit) { + final long timeCutoff = System.currentTimeMillis() - unit.toMillis(olderThan); + + getEventFilesFromDisk().filter(file -> file.lastModified() < timeCutoff) + .sorted(DirectoryUtils.SMALLEST_ID_FIRST) + .forEach(file -> delete(file)); + } + + + @Override + public long purgeOldestEvents() { + final List eventFiles = getEventFilesFromDisk().sorted(DirectoryUtils.SMALLEST_ID_FIRST).collect(Collectors.toList()); + if (eventFiles.isEmpty()) { + return 0L; + } + + for (final File eventFile : eventFiles) { + final long fileSize = eventFile.length(); + + if (delete(eventFile)) { + logger.debug("{} Deleted {} event file ({}) due to storage limits", this, eventFile, FormatUtils.formatDataSize(fileSize)); + return fileSize; + } else { + logger.warn("{} Failed to delete oldest event file {}. This file should be cleaned up manually.", this, eventFile); + continue; + } + } + + return 0L; + } + + private boolean delete(final File file) { + final long firstEventId = DirectoryUtils.getMinId(file); + synchronized (minEventIdToPathMap) { + minEventIdToPathMap.remove(firstEventId); + } + + if (!file.delete()) { + logger.warn("Failed to remove Provenance Event file {}; this file should be cleaned up manually", file); + return false; + } + + final File tocFile = TocUtil.getTocFile(file); + if (tocFile.exists() && !tocFile.delete()) { + logger.warn("Failed to remove Provenance Table-of-Contents file {}; this file should be cleaned up manually", tocFile); + } + + return true; + } + + void reindexLatestEvents(final EventIndex eventIndex) { + final List eventFiles = getEventFilesFromDisk().sorted(DirectoryUtils.SMALLEST_ID_FIRST).collect(Collectors.toList()); + if (eventFiles.isEmpty()) { + return; + } + + final long minEventIdToReindex = eventIndex.getMinimumEventIdToReindex(partitionName); + final long maxEventId = getMaxEventId(); + final long eventsToReindex = maxEventId - minEventIdToReindex; + + logger.info("The last Provenance Event indexed for partition {} is {}, but the last event written to partition has ID {}. " + + "Re-indexing up to the last {} events to ensure that the Event Index is accurate and up-to-date", + partitionName, minEventIdToReindex, maxEventId, eventsToReindex, partitionDirectory); + + // Find the first event file that we care about. + int firstEventFileIndex = 0; + for (int i = eventFiles.size() - 1; i >= 0; i--) { + final File eventFile = eventFiles.get(i); + final long minIdInFile = DirectoryUtils.getMinId(eventFile); + if (minIdInFile <= minEventIdToReindex) { + firstEventFileIndex = i; + break; + } + } + + // Create a subList that contains the files of interest + final List eventFilesToReindex = eventFiles.subList(firstEventFileIndex, eventFiles.size()); + + final ExecutorService executor = Executors.newFixedThreadPool(Math.min(4, eventFilesToReindex.size()), new NamedThreadFactory("Re-Index Provenance Events", true)); + final List> futures = new ArrayList<>(eventFilesToReindex.size()); + final AtomicLong reindexedCount = new AtomicLong(0L); + + // Re-Index the last bunch of events. + // We don't use an Event Iterator here because it's possible that one of the event files could be corrupt (for example, if NiFi does while + // writing to the file, a record may be incomplete). We don't want to prevent us from moving on and continuing to index the rest of the + // un-indexed events. So we just use a List of files and create a reader for each one. + final long start = System.nanoTime(); + int fileCount = 0; + for (final File eventFile : eventFilesToReindex) { + final boolean skipToEvent; + if (fileCount++ == 0) { + skipToEvent = true; + } else { + skipToEvent = false; + } + + final Runnable reindexTask = new Runnable() { + @Override + public void run() { + final Map storageMap = new HashMap<>(1000); + + try (final RecordReader recordReader = recordReaderFactory.newRecordReader(eventFile, Collections.emptyList(), Integer.MAX_VALUE)) { + if (skipToEvent) { + final Optional eventOption = recordReader.skipToEvent(minEventIdToReindex); + if (!eventOption.isPresent()) { + return; + } + } + + StandardProvenanceEventRecord event = null; + while (true) { + final long startBytesConsumed = recordReader.getBytesConsumed(); + + event = recordReader.nextRecord(); + if (event == null) { + eventIndex.reindexEvents(storageMap); + reindexedCount.addAndGet(storageMap.size()); + storageMap.clear(); + break; // stop reading from this file + } else { + final long eventSize = recordReader.getBytesConsumed() - startBytesConsumed; + storageMap.put(event, new StorageSummary(event.getEventId(), eventFile.getName(), partitionName, recordReader.getBlockIndex(), eventSize, 0L)); + + if (storageMap.size() == 1000) { + eventIndex.reindexEvents(storageMap); + reindexedCount.addAndGet(storageMap.size()); + storageMap.clear(); + } + } + } + } catch (final EOFException eof) { + // Ran out of data. Continue on. + logger.warn("Failed to find event with ID {} in Event File {} due to {}", minEventIdToReindex, eventFile, eof.toString()); + } catch (final Exception e) { + logger.error("Failed to index Provenance Events found in {}", eventFile, e); + } + } + }; + + futures.add(executor.submit(reindexTask)); + } + + for (final Future future : futures) { + try { + future.get(); + } catch (final ExecutionException ee) { + logger.error("Failed to re-index some Provenance events. These events may not be query-able via the Provenance interface", ee.getCause()); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("Interrupted while waiting for Provenance events to be re-indexed", e); + break; + } + } + + try { + eventIndex.commitChanges(partitionName); + } catch (final IOException e) { + logger.error("Failed to re-index Provenance Events for partition " + partitionName, e); + } + + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + final long seconds = millis / 1000L; + final long millisRemainder = millis % 1000L; + logger.info("Finished re-indexing {} events across {} files for {} in {}.{} seconds", + reindexedCount.get(), eventFilesToReindex.size(), partitionDirectory, seconds, millisRemainder); + } + + @Override + public String toString() { + return "Provenance Event Store Partition[directory=" + partitionDirectory + "]"; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/AuthorizingEventIterator.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/AuthorizingEventIterator.java new file mode 100644 index 0000000000..7ff2be79c8 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/AuthorizingEventIterator.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.store.iterator; + +import java.io.IOException; +import java.util.Optional; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.EventTransformer; + +public class AuthorizingEventIterator implements EventIterator { + private final EventIterator iterator; + private final EventAuthorizer authorizer; + private final EventTransformer transformer; + + public AuthorizingEventIterator(final EventIterator iterator, final EventAuthorizer authorizer, + final EventTransformer unauthorizedTransformer) { + this.iterator = iterator; + this.authorizer = authorizer; + this.transformer = unauthorizedTransformer; + } + + @Override + public void close() throws IOException { + iterator.close(); + } + + @Override + public Optional nextEvent() throws IOException { + while (true) { + final Optional next = iterator.nextEvent(); + if (!next.isPresent()) { + return next; + } + + if (authorizer.isAuthorized(next.get())) { + return next; + } + + final Optional eventOption = transformer.transform(next.get()); + if (eventOption.isPresent()) { + return eventOption; + } + } + } + +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/EventIterator.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/EventIterator.java new file mode 100644 index 0000000000..79acda3d1b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/EventIterator.java @@ -0,0 +1,56 @@ +/* + * 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.store.iterator; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Optional; + +import org.apache.nifi.provenance.ProvenanceEventRecord; + +public interface EventIterator extends Closeable { + + Optional nextEvent() throws IOException; + + public static EventIterator EMPTY = new EventIterator() { + @Override + public void close() throws IOException { + } + + @Override + public Optional nextEvent() { + return Optional.empty(); + } + }; + + public static EventIterator of(final ProvenanceEventRecord... events) { + final Iterator itr = Arrays.asList(events).iterator(); + return new EventIterator() { + @Override + public void close() throws IOException { + } + + @Override + public Optional nextEvent() { + return itr.hasNext() ? Optional.empty() : Optional.of(itr.next()); + } + }; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SelectiveRecordReaderEventIterator.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SelectiveRecordReaderEventIterator.java new file mode 100644 index 0000000000..c4a130ba01 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SelectiveRecordReaderEventIterator.java @@ -0,0 +1,174 @@ +/* + * 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.store.iterator; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.serialization.RecordReader; +import org.apache.nifi.provenance.store.RecordReaderFactory; +import org.apache.nifi.provenance.util.DirectoryUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SelectiveRecordReaderEventIterator implements EventIterator { + private static final Logger logger = LoggerFactory.getLogger(SelectiveRecordReaderEventIterator.class); + private final List files; + private final RecordReaderFactory readerFactory; + private final List eventIds; + private final Iterator idIterator; + private final int maxAttributeChars; + + private boolean closed = false; + private RecordReader reader; + private File currentFile; + + public SelectiveRecordReaderEventIterator(final List filesToRead, final RecordReaderFactory readerFactory, final List eventIds, final int maxAttributeChars) { + this.readerFactory = readerFactory; + + this.eventIds = new ArrayList<>(eventIds); + Collections.sort(this.eventIds); + idIterator = this.eventIds.iterator(); + + // Make a copy of the list of files and prune out any Files that are not relevant to the Event ID's that we were given. + if (eventIds.isEmpty() || filesToRead.isEmpty()) { + this.files = Collections.emptyList(); + } else { + this.files = filterUnneededFiles(filesToRead, this.eventIds); + } + + this.maxAttributeChars = maxAttributeChars; + } + + protected static List filterUnneededFiles(final List filesToRead, final List eventIds) { + final List files = new ArrayList<>(); + final Long firstEventId = eventIds.get(0); + final Long lastEventId = eventIds.get(eventIds.size() - 1); + + final List sortedFileList = new ArrayList<>(filesToRead); + Collections.sort(sortedFileList, DirectoryUtils.SMALLEST_ID_FIRST); + + File lastFile = null; + for (final File file : filesToRead) { + final long firstIdInFile = DirectoryUtils.getMinId(file); + if (firstIdInFile > lastEventId) { + continue; + } + + if (firstIdInFile > firstEventId) { + if (files.isEmpty() && lastFile != null) { + files.add(lastFile); + } + + files.add(file); + } + + lastFile = file; + } + + if (files.isEmpty() && lastFile != null) { + files.add(lastFile); + } + + return files; + } + + @Override + public void close() throws IOException { + closed = true; + + if (reader != null) { + reader.close(); + } + } + + + @Override + public Optional nextEvent() throws IOException { + if (closed) { + throw new IOException("EventIterator is already closed"); + } + + final long start = System.nanoTime(); + try { + while (idIterator.hasNext()) { + // Determine the next event ID to fetch + final long eventId = idIterator.next(); + + // Determine which file the event should be in. + final File fileForEvent = getFileForEventId(eventId); + if (fileForEvent == null) { + continue; + } + + // If we determined which file the event should be in, and that's not the file that + // we are currently reading from, rotate the reader to the appropriate one. + if (!fileForEvent.equals(currentFile)) { + if (reader != null) { + try { + reader.close(); + } catch (final Exception e) { + logger.warn("Failed to close {}; some resources may not be cleaned up appropriately", reader); + } + } + + reader = readerFactory.newRecordReader(fileForEvent, Collections.emptyList(), maxAttributeChars); + this.currentFile = fileForEvent; + } + + final Optional eventOption = reader.skipToEvent(eventId); + if (eventOption.isPresent() && eventOption.get().getEventId() == eventId) { + reader.nextRecord(); // consume the event from the stream. + return eventOption; + } + + continue; + } + + return Optional.empty(); + } finally { + final long ms = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + logger.trace("Took {} ms to read next event", ms); + } + } + + private File getFileForEventId(final long eventId) { + File lastFile = null; + for (final File file : files) { + final long firstEventId = DirectoryUtils.getMinId(file); + if (firstEventId == eventId) { + return file; + } + + if (firstEventId > eventId) { + return lastFile; + } + + lastFile = file; + } + + return lastFile; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SequentialRecordReaderEventIterator.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SequentialRecordReaderEventIterator.java new file mode 100644 index 0000000000..869febfb6b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/store/iterator/SequentialRecordReaderEventIterator.java @@ -0,0 +1,115 @@ +/* + * 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.store.iterator; + +import java.io.EOFException; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.serialization.RecordReader; +import org.apache.nifi.provenance.store.RecordReaderFactory; + +public class SequentialRecordReaderEventIterator implements EventIterator { + private final Iterator fileIterator; + private final RecordReaderFactory readerFactory; + private final long minimumEventId; + private final int maxAttributeChars; + + private boolean closed = false; + private RecordReader reader; + + public SequentialRecordReaderEventIterator(final List filesToRead, final RecordReaderFactory readerFactory, final long minimumEventId, final int maxAttributeChars) { + this.fileIterator = filesToRead.iterator(); + this.readerFactory = readerFactory; + this.minimumEventId = minimumEventId; + this.maxAttributeChars = maxAttributeChars; + } + + @Override + public void close() throws IOException { + closed = true; + + if (reader != null) { + reader.close(); + } + } + + @Override + public Optional nextEvent() throws IOException { + if (closed) { + throw new IOException("EventIterator is already closed"); + } + + if (reader == null) { + if (!rotateReader()) { + return Optional.empty(); + } + } + + while (true) { + final ProvenanceEventRecord event = reader.nextRecord(); + if (event == null) { + if (rotateReader()) { + continue; + } else { + return Optional.empty(); + } + } else { + return Optional.of(event); + } + } + } + + private boolean rotateReader() throws IOException { + final boolean readerExists = (reader != null); + if (readerExists) { + reader.close(); + } + + boolean multipleReadersOpened = false; + while (true) { + if (!fileIterator.hasNext()) { + return false; + } + + final File eventFile = fileIterator.next(); + try { + reader = readerFactory.newRecordReader(eventFile, Collections.emptyList(), maxAttributeChars); + break; + } catch (final FileNotFoundException | EOFException e) { + multipleReadersOpened = true; + // File may have aged off or was not fully written. Move to next file + continue; + } + } + + // If this is the first file in our list, the event of interest may not be the first event, + // so skip to the event that we want. + if (!readerExists && !multipleReadersOpened) { + reader.skipToEvent(minimumEventId); + } + + return true; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/StandardTocReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/StandardTocReader.java index 60328fa95f..a9c0f20e5e 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/StandardTocReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/StandardTocReader.java @@ -16,12 +16,13 @@ */ package org.apache.nifi.provenance.toc; -import java.io.DataInputStream; import java.io.EOFException; import java.io.File; import java.io.FileInputStream; import java.io.IOException; +import org.apache.nifi.stream.io.StreamUtils; + /** * Standard implementation of TocReader. * @@ -38,27 +39,29 @@ public class StandardTocReader implements TocReader { private final boolean compressed; private final long[] offsets; private final long[] firstEventIds; + private final File file; public StandardTocReader(final File file) throws IOException { - try (final FileInputStream fis = new FileInputStream(file); - final DataInputStream dis = new DataInputStream(fis)) { + this.file = file; + final long fileLength = file.length(); + if (fileLength < 2) { + throw new EOFException(); + } - final int version = dis.read(); - if ( version < 0 ) { - throw new EOFException(); - } + try (final FileInputStream fis = new FileInputStream(file)) { + final byte[] buffer = new byte[(int) fileLength]; + StreamUtils.fillBuffer(fis, buffer); - final int compressionFlag = dis.read(); - if ( compressionFlag < 0 ) { - throw new EOFException(); - } + final int version = buffer[0]; + final int compressionFlag = buffer[1]; 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); + throw new IOException("Table of Contents file " + file + " appears to be corrupt: could not read 'compression flag' from header; " + + "expected value of 0 or 1 but got " + compressionFlag); } final int blockInfoBytes; @@ -72,7 +75,7 @@ public class StandardTocReader implements TocReader { break; } - final int numBlocks = (int) ((file.length() - 2) / blockInfoBytes); + final int numBlocks = (buffer.length - 2) / blockInfoBytes; offsets = new long[numBlocks]; if ( version > 1 ) { @@ -81,21 +84,40 @@ public class StandardTocReader implements TocReader { firstEventIds = new long[0]; } + int index = 2; for (int i=0; i < numBlocks; i++) { - offsets[i] = dis.readLong(); + offsets[i] = readLong(buffer, index); + index += 8; if ( version > 1 ) { - firstEventIds[i] = dis.readLong(); + firstEventIds[i] = readLong(buffer, index); + index += 8; } } } } + private long readLong(final byte[] buffer, final int offset) { + return ((long) buffer[offset] << 56) + + ((long) (buffer[offset + 1] & 0xFF) << 48) + + ((long) (buffer[offset + 2] & 0xFF) << 40) + + ((long) (buffer[offset + 3] & 0xFF) << 32) + + ((long) (buffer[offset + 4] & 0xFF) << 24) + + ((long) (buffer[offset + 5] & 0xFF) << 16) + + ((long) (buffer[offset + 6] & 0xFF) << 8) + + (buffer[offset + 7] & 0xFF); + } + @Override public boolean isCompressed() { return compressed; } + @Override + public File getFile() { + return file; + } + @Override public long getBlockOffset(final int blockIndex) { if ( blockIndex >= offsets.length ) { @@ -104,6 +126,15 @@ public class StandardTocReader implements TocReader { return offsets[blockIndex]; } + @Override + public long getFirstEventIdForBlock(final int blockIndex) { + if (blockIndex >= firstEventIds.length) { + return -1L; + } + + return firstEventIds[blockIndex]; + } + @Override public long getLastBlockOffset() { if ( offsets.length == 0 ) { @@ -113,7 +144,7 @@ public class StandardTocReader implements TocReader { } @Override - public void close() throws IOException { + public void close() { } @Override @@ -152,4 +183,9 @@ public class StandardTocReader implements TocReader { // Therefore, if the event is present, it must be in the last block. return firstEventIds.length - 1; } + + @Override + public String toString() { + return "StandardTocReader[file=" + file + ", compressed=" + compressed + "]"; + } } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocReader.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocReader.java index be6a165af9..0bb630eea7 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocReader.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocReader.java @@ -17,6 +17,7 @@ package org.apache.nifi.provenance.toc; import java.io.Closeable; +import java.io.File; /** *

@@ -36,15 +37,30 @@ public interface TocReader extends Closeable { */ boolean isCompressed(); + /** + * @return the file that holds the TOC information + */ + File getFile(); + /** * Returns the byte offset into the Journal File for the Block with the given index. * * @param blockIndex the block index to get the byte offset for * @return the byte offset for the given block index, or -1 if the given block index - * does not exist + * does not exist */ long getBlockOffset(int blockIndex); + /** + * Returns the ID of the first event that is found in the block with the given index, or -1 if + * the given block index does not exist. + * + * @param blockIndex the block index to get the first event id for + * @return the ID of the first event that is found in the block with the given index, or -1 if + * the given block index does not exist + */ + long getFirstEventIdForBlock(int blockIndex); + /** * Returns the byte offset into the Journal File of the last Block in the given index * @return the byte offset into the Journal File of the last Block in the given index diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocUtil.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocUtil.java index 28cecd82ca..91f70db653 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocUtil.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/toc/TocUtil.java @@ -32,7 +32,7 @@ public class TocUtil { */ public static File getTocFile(final File journalFile) { final File tocDir = new File(journalFile.getParentFile(), "toc"); - final String basename = LuceneUtil.substringBefore(journalFile.getName(), "."); + final String basename = LuceneUtil.substringBefore(journalFile.getName(), ".prov"); final File tocFile = new File(tocDir, basename + ".toc"); return tocFile; } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/CloseableUtil.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/CloseableUtil.java new file mode 100644 index 0000000000..26caa10ce3 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/CloseableUtil.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.provenance.util; + +import java.io.Closeable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CloseableUtil { + private static final Logger logger = LoggerFactory.getLogger(CloseableUtil.class); + + public static void closeQuietly(final Closeable... closeables) { + for (final Closeable closeable : closeables) { + if (closeable == null) { + continue; + } + + try { + closeable.close(); + } catch (final Exception e) { + if (logger.isDebugEnabled()) { + logger.warn("Failed to close {}; sources resources may not be cleaned up appropriately.", closeable, e); + } else { + logger.warn("Failed to close {}; sources resources may not be cleaned up appropriately.", closeable); + } + } + } + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/DirectoryUtils.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/DirectoryUtils.java new file mode 100644 index 0000000000..a90500d90d --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/DirectoryUtils.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.util; + +import java.io.File; +import java.io.FileFilter; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.nifi.provenance.RepositoryConfiguration; + +public class DirectoryUtils { + + public static final FileFilter EVENT_FILE_FILTER = f -> f.getName().endsWith(".prov") || f.getName().endsWith(".prov.gz"); + public static final FileFilter INDEX_FILE_FILTER = f -> f.getName().startsWith("index-"); + public static final Comparator SMALLEST_ID_FIRST = (a, b) -> Long.compare(getMinId(a), getMinId(b)); + public static final Comparator LARGEST_ID_FIRST = SMALLEST_ID_FIRST.reversed(); + public static final Comparator OLDEST_INDEX_FIRST = (a, b) -> Long.compare(getIndexTimestamp(a), getIndexTimestamp(b)); + public static final Comparator NEWEST_INDEX_FIRST = OLDEST_INDEX_FIRST.reversed(); + + public static List getProvenanceEventFiles(final RepositoryConfiguration repoConfig) { + return repoConfig.getStorageDirectories().values().stream() + .flatMap(f -> { + final File[] eventFiles = f.listFiles(EVENT_FILE_FILTER); + return eventFiles == null ? Stream.empty() : Arrays.stream(eventFiles); + }) + .map(f -> f.toPath()) + .collect(Collectors.toList()); + } + + public static long getMinId(final File file) { + final String filename = file.getName(); + final int firstDotIndex = filename.indexOf("."); + if (firstDotIndex < 1) { + return -1L; + } + + final String firstEventId = filename.substring(0, firstDotIndex); + try { + return Long.parseLong(firstEventId); + } catch (final NumberFormatException nfe) { + return -1L; + } + } + + public static long getIndexTimestamp(final File file) { + final String filename = file.getName(); + if (!filename.startsWith("index-") && filename.length() > 6) { + return -1L; + } + + final String suffix = filename.substring(6); + try { + return Long.parseLong(suffix); + } catch (final NumberFormatException nfe) { + return -1L; + } + } + + public static long getSize(final File file) { + if (file.isFile()) { + return file.length(); + } + + final File[] children = file.listFiles(); + if (children == null || children.length == 0) { + return 0L; + } + + long total = 0L; + for (final File child : children) { + total += getSize(child); + } + + return total; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/DumpEventFile.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/DumpEventFile.java new file mode 100644 index 0000000000..df16356828 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/DumpEventFile.java @@ -0,0 +1,79 @@ +/* + * 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.util; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.Date; + +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; + +public class DumpEventFile { + + private static void printUsage() { + System.out.println("Usage:"); + System.out.println(); + System.out.println("java " + DumpEventFile.class.getName() + " "); + System.out.println(); + } + + public static void main(final String[] args) throws IOException { + if (args.length != 1) { + printUsage(); + return; + } + + final File file = new File(args[0]); + if (!file.exists()) { + System.out.println("Cannot find file " + file.getAbsolutePath()); + return; + } + + try (final RecordReader reader = RecordReaders.newRecordReader(file, Collections.emptyList(), 65535)) { + StandardProvenanceEventRecord event; + int index = 0; + while ((event = reader.nextRecord()) != null) { + final long byteOffset = reader.getBytesConsumed(); + final String string = stringify(event, index++, byteOffset); + System.out.println(string); + } + } + } + + private static String stringify(final ProvenanceEventRecord event, final int index, final long byteOffset) { + final StringBuilder sb = new StringBuilder(); + sb.append("Event Index in File = ").append(index).append(", Byte Offset = ").append(byteOffset); + sb.append("\n\t").append("Event ID = ").append(event.getEventId()); + sb.append("\n\t").append("Event Type = ").append(event.getEventType()); + sb.append("\n\t").append("Event Time = ").append(new Date(event.getEventTime())); + sb.append("\n\t").append("Event UUID = ").append(event.getFlowFileUuid()); + sb.append("\n\t").append("Component ID = ").append(event.getComponentId()); + sb.append("\n\t").append("Event ID = ").append(event.getComponentType()); + sb.append("\n\t").append("Transit URI = ").append(event.getTransitUri()); + sb.append("\n\t").append("Parent IDs = ").append(event.getParentUuids()); + sb.append("\n\t").append("Child IDs = ").append(event.getChildUuids()); + sb.append("\n\t").append("Previous Attributes = ").append(event.getPreviousAttributes()); + sb.append("\n\t").append("Updated Attributes = ").append(event.getUpdatedAttributes()); + + return sb.toString(); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/NamedThreadFactory.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/NamedThreadFactory.java new file mode 100644 index 0000000000..2ee6ed6fa2 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/NamedThreadFactory.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.util; + +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +public class NamedThreadFactory implements ThreadFactory { + + private final AtomicInteger counter = new AtomicInteger(0); + private final ThreadFactory defaultThreadFactory = Executors.defaultThreadFactory(); + private final String namePrefix; + private final boolean daemon; + + public NamedThreadFactory(final String namePrefix) { + this(namePrefix, false); + } + + public NamedThreadFactory(final String namePrefix, final boolean daemon) { + this.namePrefix = namePrefix; + this.daemon = daemon; + } + + @Override + public Thread newThread(final Runnable r) { + final Thread thread = defaultThreadFactory.newThread(r); + thread.setName(namePrefix + "-" + counter.incrementAndGet()); + thread.setDaemon(daemon); + return thread; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/StorageSummaryEvent.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/StorageSummaryEvent.java new file mode 100644 index 0000000000..41d5ade7b6 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/util/StorageSummaryEvent.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.util; + +import java.util.List; +import java.util.Map; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.serialization.StorageSummary; + +public class StorageSummaryEvent implements ProvenanceEventRecord { + private final ProvenanceEventRecord event; + private final StorageSummary storageSummary; + + public StorageSummaryEvent(final ProvenanceEventRecord event, final StorageSummary storageSummary) { + this.event = event; + this.storageSummary = storageSummary; + } + + @Override + public long getEventId() { + return storageSummary.getEventId(); + } + + @Override + public long getEventTime() { + return event.getEventTime(); + } + + @Override + public long getFlowFileEntryDate() { + return event.getFlowFileEntryDate(); + } + + @Override + public long getLineageStartDate() { + return event.getLineageStartDate(); + } + + @Override + public long getFileSize() { + return event.getFileSize(); + } + + @Override + public Long getPreviousFileSize() { + return event.getPreviousFileSize(); + } + + @Override + public long getEventDuration() { + return event.getEventDuration(); + } + + @Override + public ProvenanceEventType getEventType() { + return event.getEventType(); + } + + @Override + public Map getAttributes() { + return event.getAttributes(); + } + + @Override + public Map getPreviousAttributes() { + return event.getPreviousAttributes(); + } + + @Override + public Map getUpdatedAttributes() { + return event.getUpdatedAttributes(); + } + + @Override + public String getComponentId() { + return event.getComponentId(); + } + + @Override + public String getComponentType() { + return event.getComponentType(); + } + + @Override + public String getTransitUri() { + return event.getTransitUri(); + } + + @Override + public String getSourceSystemFlowFileIdentifier() { + return event.getSourceSystemFlowFileIdentifier(); + } + + @Override + public String getFlowFileUuid() { + return event.getFlowFileUuid(); + } + + @Override + public List getParentUuids() { + return event.getParentUuids(); + } + + @Override + public List getChildUuids() { + return event.getChildUuids(); + } + + @Override + public String getAlternateIdentifierUri() { + return event.getAlternateIdentifierUri(); + } + + @Override + public String getDetails() { + return event.getDetails(); + } + + @Override + public String getRelationship() { + return event.getRelationship(); + } + + @Override + public String getSourceQueueIdentifier() { + return event.getSourceQueueIdentifier(); + } + + @Override + public String getContentClaimSection() { + return event.getContentClaimSection(); + } + + @Override + public String getPreviousContentClaimSection() { + return event.getPreviousContentClaimSection(); + } + + @Override + public String getContentClaimContainer() { + return event.getContentClaimContainer(); + } + + @Override + public String getPreviousContentClaimContainer() { + return event.getPreviousContentClaimContainer(); + } + + @Override + public String getContentClaimIdentifier() { + return event.getContentClaimIdentifier(); + } + + @Override + public String getPreviousContentClaimIdentifier() { + return event.getPreviousContentClaimIdentifier(); + } + + @Override + public Long getContentClaimOffset() { + return event.getContentClaimOffset(); + } + + @Override + public Long getPreviousContentClaimOffset() { + return event.getPreviousContentClaimOffset(); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceRepository b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceRepository index 78da70e31f..6a353d2af8 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceRepository +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/resources/META-INF/services/org.apache.nifi.provenance.ProvenanceRepository @@ -12,4 +12,5 @@ # 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.PersistentProvenanceRepository \ No newline at end of file +org.apache.nifi.provenance.PersistentProvenanceRepository +org.apache.nifi.provenance.WriteAheadProvenanceRepository \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java index bae2364f21..36397c4744 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/AbstractTestRecordReaderWriter.java @@ -17,8 +17,8 @@ package org.apache.nifi.provenance; -import static org.apache.nifi.provenance.TestUtil.createFlowFile; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -27,8 +27,10 @@ import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.util.HashMap; +import java.util.ArrayList; +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import org.apache.nifi.provenance.serialization.RecordReader; @@ -42,6 +44,7 @@ import org.apache.nifi.util.file.FileUtils; import org.junit.BeforeClass; import org.junit.Test; + public abstract class AbstractTestRecordReaderWriter { @BeforeClass public static void setLogLevel() { @@ -49,20 +52,7 @@ public abstract class AbstractTestRecordReaderWriter { } protected ProvenanceEventRecord createEvent() { - final Map attributes = new HashMap<>(); - attributes.put("filename", "1.txt"); - attributes.put("uuid", UUID.randomUUID().toString()); - - final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); - builder.setEventTime(System.currentTimeMillis()); - builder.setEventType(ProvenanceEventType.RECEIVE); - builder.setTransitUri("nifi://unit-test"); - builder.fromFlowFile(createFlowFile(3L, 3000L, attributes)); - builder.setComponentId("1234"); - builder.setComponentType("dummy processor"); - final ProvenanceEventRecord record = builder.build(); - - return record; + return TestUtil.createEvent(); } @Test @@ -73,7 +63,7 @@ public abstract class AbstractTestRecordReaderWriter { final RecordWriter writer = createWriter(journalFile, tocWriter, false, 1024 * 1024); writer.writeHeader(1L); - writer.writeRecord(createEvent(), 1L); + writer.writeRecord(createEvent()); writer.close(); final TocReader tocReader = new StandardTocReader(tocFile); @@ -101,7 +91,7 @@ public abstract class AbstractTestRecordReaderWriter { final RecordWriter writer = createWriter(journalFile, tocWriter, true, 8192); writer.writeHeader(1L); - writer.writeRecord(createEvent(), 1L); + writer.writeRecord(createEvent()); writer.close(); final TocReader tocReader = new StandardTocReader(tocFile); @@ -131,7 +121,7 @@ public abstract class AbstractTestRecordReaderWriter { writer.writeHeader(1L); for (int i = 0; i < 10; i++) { - writer.writeRecord(createEvent(), i); + writer.writeRecord(createEvent()); } writer.close(); @@ -170,7 +160,7 @@ public abstract class AbstractTestRecordReaderWriter { writer.writeHeader(1L); for (int i = 0; i < 10; i++) { - writer.writeRecord(createEvent(), i); + writer.writeRecord(createEvent()); } writer.close(); @@ -198,6 +188,56 @@ public abstract class AbstractTestRecordReaderWriter { FileUtils.deleteFile(journalFile.getParentFile(), true); } + @Test + public void testSkipToEvent() throws IOException { + final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz"); + final File tocFile = TocUtil.getTocFile(journalFile); + final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false); + // new block each 10 bytes + final RecordWriter writer = createWriter(journalFile, tocWriter, true, 100); + + writer.writeHeader(0L); + final int numEvents = 10; + final List events = new ArrayList<>(); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + events.add(event); + writer.writeRecord(event); + } + writer.close(); + + final TocReader tocReader = new StandardTocReader(tocFile); + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + + for (int i = 0; i < numEvents; i++) { + final Optional eventOption = reader.skipToEvent(i); + assertTrue(eventOption.isPresent()); + assertEquals(i, eventOption.get().getEventId()); + assertEquals(events.get(i), eventOption.get()); + + final StandardProvenanceEventRecord consumedEvent = reader.nextRecord(); + assertEquals(eventOption.get(), consumedEvent); + } + + assertFalse(reader.skipToEvent(numEvents + 1).isPresent()); + } + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + + for (int i = 0; i < 3; i++) { + final Optional eventOption = reader.skipToEvent(8); + assertTrue(eventOption.isPresent()); + assertEquals(events.get(8), eventOption.get()); + } + + final StandardProvenanceEventRecord consumedEvent = reader.nextRecord(); + assertEquals(events.get(8), consumedEvent); + } + } + protected abstract RecordWriter createWriter(File file, TocWriter tocWriter, boolean compressed, int uncompressedBlockSize) throws IOException; protected abstract RecordReader createReader(InputStream in, String journalFilename, TocReader tocReader, int maxAttributeSize) throws IOException; diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestEventIdFirstSchemaRecordReaderWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestEventIdFirstSchemaRecordReaderWriter.java new file mode 100644 index 0000000000..9c89ab30e2 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestEventIdFirstSchemaRecordReaderWriter.java @@ -0,0 +1,477 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.provenance.serialization.RecordReader; +import org.apache.nifi.provenance.serialization.RecordWriter; +import org.apache.nifi.provenance.toc.StandardTocReader; +import org.apache.nifi.provenance.toc.StandardTocWriter; +import org.apache.nifi.provenance.toc.TocReader; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.toc.TocWriter; +import org.apache.nifi.util.file.FileUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +public class TestEventIdFirstSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter { + private final AtomicLong idGenerator = new AtomicLong(0L); + private File journalFile; + private File tocFile; + + @BeforeClass + public static void setupLogger() { + System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi", "DEBUG"); + } + + @Before + public void setup() { + journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testEventIdFirstSchemaRecordReaderWriter"); + tocFile = TocUtil.getTocFile(journalFile); + idGenerator.set(0L); + }; + + @Test + public void testContentClaimUnchanged() throws IOException { + final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz"); + final File tocFile = TocUtil.getTocFile(journalFile); + final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false); + final RecordWriter writer = createWriter(journalFile, tocWriter, true, 8192); + + final Map attributes = new HashMap<>(); + attributes.put("filename", "1.txt"); + attributes.put("uuid", UUID.randomUUID().toString()); + + final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventTime(System.currentTimeMillis()); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + builder.fromFlowFile(TestUtil.createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("dummy processor"); + builder.setPreviousContentClaim("container-1", "section-1", "identifier-1", 1L, 1L); + builder.setCurrentContentClaim("container-1", "section-1", "identifier-1", 1L, 1L); + final ProvenanceEventRecord record = builder.build(); + + writer.writeHeader(1L); + writer.writeRecord(record); + writer.close(); + + final TocReader tocReader = new StandardTocReader(tocFile); + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + assertEquals(0, reader.getBlockIndex()); + reader.skipToBlock(0); + final StandardProvenanceEventRecord recovered = reader.nextRecord(); + assertNotNull(recovered); + + assertEquals("nifi://unit-test", recovered.getTransitUri()); + + assertEquals("container-1", recovered.getPreviousContentClaimContainer()); + assertEquals("container-1", recovered.getContentClaimContainer()); + + assertEquals("section-1", recovered.getPreviousContentClaimSection()); + assertEquals("section-1", recovered.getContentClaimSection()); + + assertEquals("identifier-1", recovered.getPreviousContentClaimIdentifier()); + assertEquals("identifier-1", recovered.getContentClaimIdentifier()); + + assertEquals(1L, recovered.getPreviousContentClaimOffset().longValue()); + assertEquals(1L, recovered.getContentClaimOffset().longValue()); + + assertEquals(1L, recovered.getPreviousFileSize().longValue()); + assertEquals(1L, recovered.getContentClaimOffset().longValue()); + + assertNull(reader.nextRecord()); + } + + FileUtils.deleteFile(journalFile.getParentFile(), true); + } + + @Test + public void testContentClaimRemoved() throws IOException { + final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz"); + final File tocFile = TocUtil.getTocFile(journalFile); + final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false); + final RecordWriter writer = createWriter(journalFile, tocWriter, true, 8192); + + final Map attributes = new HashMap<>(); + attributes.put("filename", "1.txt"); + attributes.put("uuid", UUID.randomUUID().toString()); + + final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventTime(System.currentTimeMillis()); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + builder.fromFlowFile(TestUtil.createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("dummy processor"); + builder.setPreviousContentClaim("container-1", "section-1", "identifier-1", 1L, 1L); + builder.setCurrentContentClaim(null, null, null, 0L, 0L); + final ProvenanceEventRecord record = builder.build(); + + writer.writeHeader(1L); + writer.writeRecord(record); + writer.close(); + + final TocReader tocReader = new StandardTocReader(tocFile); + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + assertEquals(0, reader.getBlockIndex()); + reader.skipToBlock(0); + final StandardProvenanceEventRecord recovered = reader.nextRecord(); + assertNotNull(recovered); + + assertEquals("nifi://unit-test", recovered.getTransitUri()); + + assertEquals("container-1", recovered.getPreviousContentClaimContainer()); + assertNull(recovered.getContentClaimContainer()); + + assertEquals("section-1", recovered.getPreviousContentClaimSection()); + assertNull(recovered.getContentClaimSection()); + + assertEquals("identifier-1", recovered.getPreviousContentClaimIdentifier()); + assertNull(recovered.getContentClaimIdentifier()); + + assertEquals(1L, recovered.getPreviousContentClaimOffset().longValue()); + assertNull(recovered.getContentClaimOffset()); + + assertEquals(1L, recovered.getPreviousFileSize().longValue()); + assertEquals(0L, recovered.getFileSize()); + + assertNull(reader.nextRecord()); + } + + FileUtils.deleteFile(journalFile.getParentFile(), true); + } + + @Test + public void testContentClaimAdded() throws IOException { + final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz"); + final File tocFile = TocUtil.getTocFile(journalFile); + final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false); + final RecordWriter writer = createWriter(journalFile, tocWriter, true, 8192); + + final Map attributes = new HashMap<>(); + attributes.put("filename", "1.txt"); + attributes.put("uuid", UUID.randomUUID().toString()); + + final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventTime(System.currentTimeMillis()); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + builder.fromFlowFile(TestUtil.createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("dummy processor"); + builder.setCurrentContentClaim("container-1", "section-1", "identifier-1", 1L, 1L); + final ProvenanceEventRecord record = builder.build(); + + writer.writeHeader(1L); + writer.writeRecord(record); + writer.close(); + + final TocReader tocReader = new StandardTocReader(tocFile); + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + assertEquals(0, reader.getBlockIndex()); + reader.skipToBlock(0); + final StandardProvenanceEventRecord recovered = reader.nextRecord(); + assertNotNull(recovered); + + assertEquals("nifi://unit-test", recovered.getTransitUri()); + + assertEquals("container-1", recovered.getContentClaimContainer()); + assertNull(recovered.getPreviousContentClaimContainer()); + + assertEquals("section-1", recovered.getContentClaimSection()); + assertNull(recovered.getPreviousContentClaimSection()); + + assertEquals("identifier-1", recovered.getContentClaimIdentifier()); + assertNull(recovered.getPreviousContentClaimIdentifier()); + + assertEquals(1L, recovered.getContentClaimOffset().longValue()); + assertNull(recovered.getPreviousContentClaimOffset()); + + assertEquals(1L, recovered.getFileSize()); + assertNull(recovered.getPreviousContentClaimOffset()); + + assertNull(reader.nextRecord()); + } + + FileUtils.deleteFile(journalFile.getParentFile(), true); + } + + @Test + public void testContentClaimChanged() throws IOException { + final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz"); + final File tocFile = TocUtil.getTocFile(journalFile); + final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false); + final RecordWriter writer = createWriter(journalFile, tocWriter, true, 8192); + + final Map attributes = new HashMap<>(); + attributes.put("filename", "1.txt"); + attributes.put("uuid", UUID.randomUUID().toString()); + + final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventTime(System.currentTimeMillis()); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + builder.fromFlowFile(TestUtil.createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("dummy processor"); + builder.setPreviousContentClaim("container-1", "section-1", "identifier-1", 1L, 1L); + builder.setCurrentContentClaim("container-2", "section-2", "identifier-2", 2L, 2L); + final ProvenanceEventRecord record = builder.build(); + + writer.writeHeader(1L); + writer.writeRecord(record); + writer.close(); + + final TocReader tocReader = new StandardTocReader(tocFile); + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + assertEquals(0, reader.getBlockIndex()); + reader.skipToBlock(0); + final StandardProvenanceEventRecord recovered = reader.nextRecord(); + assertNotNull(recovered); + + assertEquals("nifi://unit-test", recovered.getTransitUri()); + + assertEquals("container-1", recovered.getPreviousContentClaimContainer()); + assertEquals("container-2", recovered.getContentClaimContainer()); + + assertEquals("section-1", recovered.getPreviousContentClaimSection()); + assertEquals("section-2", recovered.getContentClaimSection()); + + assertEquals("identifier-1", recovered.getPreviousContentClaimIdentifier()); + assertEquals("identifier-2", recovered.getContentClaimIdentifier()); + + assertEquals(1L, recovered.getPreviousContentClaimOffset().longValue()); + assertEquals(2L, recovered.getContentClaimOffset().longValue()); + + assertEquals(1L, recovered.getPreviousFileSize().longValue()); + assertEquals(2L, recovered.getContentClaimOffset().longValue()); + + assertNull(reader.nextRecord()); + } + + FileUtils.deleteFile(journalFile.getParentFile(), true); + } + + @Test + public void testEventIdAndTimestampCorrect() throws IOException { + final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.gz"); + final File tocFile = TocUtil.getTocFile(journalFile); + final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false); + final RecordWriter writer = createWriter(journalFile, tocWriter, true, 8192); + + final Map attributes = new HashMap<>(); + attributes.put("filename", "1.txt"); + attributes.put("uuid", UUID.randomUUID().toString()); + + final long timestamp = System.currentTimeMillis() - 10000L; + + final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventId(1_000_000); + builder.setEventTime(timestamp); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + builder.fromFlowFile(TestUtil.createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("dummy processor"); + builder.setPreviousContentClaim("container-1", "section-1", "identifier-1", 1L, 1L); + builder.setCurrentContentClaim("container-2", "section-2", "identifier-2", 2L, 2L); + final ProvenanceEventRecord record = builder.build(); + + writer.writeHeader(500_000L); + writer.writeRecord(record); + writer.close(); + + final TocReader tocReader = new StandardTocReader(tocFile); + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + + final ProvenanceEventRecord event = reader.nextRecord(); + assertNotNull(event); + assertEquals(1_000_000L, event.getEventId()); + assertEquals(timestamp, event.getEventTime()); + assertNull(reader.nextRecord()); + } + + FileUtils.deleteFile(journalFile.getParentFile(), true); + } + + + @Test + public void testComponentIdInlineAndLookup() throws IOException { + final File journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testSimpleWrite.prov"); + final File tocFile = TocUtil.getTocFile(journalFile); + final TocWriter tocWriter = new StandardTocWriter(tocFile, false, false); + + final IdentifierLookup lookup = new IdentifierLookup() { + @Override + public List getQueueIdentifiers() { + return Collections.emptyList(); + } + + @Override + public List getComponentTypes() { + return Collections.singletonList("unit-test-component-1"); + } + + @Override + public List getComponentIdentifiers() { + return Collections.singletonList("1234"); + } + }; + + final RecordWriter writer = new EventIdFirstSchemaRecordWriter(journalFile, idGenerator, tocWriter, false, 1024 * 32, lookup); + + final Map attributes = new HashMap<>(); + attributes.put("filename", "1.txt"); + attributes.put("uuid", UUID.randomUUID().toString()); + + final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventId(1_000_000); + builder.setEventTime(System.currentTimeMillis()); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + builder.fromFlowFile(TestUtil.createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("unit-test-component-2"); + builder.setPreviousContentClaim("container-1", "section-1", "identifier-1", 1L, 1L); + builder.setCurrentContentClaim("container-2", "section-2", "identifier-2", 2L, 2L); + + writer.writeHeader(500_000L); + writer.writeRecord(builder.build()); + + builder.setEventId(1_000_001L); + builder.setComponentId("4444"); + builder.setComponentType("unit-test-component-1"); + writer.writeRecord(builder.build()); + + writer.close(); + + final TocReader tocReader = new StandardTocReader(tocFile); + + try (final FileInputStream fis = new FileInputStream(journalFile); + final RecordReader reader = createReader(fis, journalFile.getName(), tocReader, 2048)) { + + ProvenanceEventRecord event = reader.nextRecord(); + assertNotNull(event); + assertEquals(1_000_000L, event.getEventId()); + assertEquals("1234", event.getComponentId()); + assertEquals("unit-test-component-2", event.getComponentType()); + + event = reader.nextRecord(); + assertNotNull(event); + assertEquals(1_000_001L, event.getEventId()); + assertEquals("4444", event.getComponentId()); + assertEquals("unit-test-component-1", event.getComponentType()); + + assertNull(reader.nextRecord()); + } + + FileUtils.deleteFile(journalFile.getParentFile(), true); + } + + @Override + protected RecordWriter createWriter(final File file, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { + return new EventIdFirstSchemaRecordWriter(file, idGenerator, tocWriter, compressed, uncompressedBlockSize, IdentifierLookup.EMPTY); + } + + @Override + protected RecordReader createReader(final InputStream in, final String journalFilename, final TocReader tocReader, final int maxAttributeSize) throws IOException { + return new EventIdFirstSchemaRecordReader(in, journalFilename, tocReader, maxAttributeSize); + } + + @Test + @Ignore + public void testPerformanceOfRandomAccessReads() throws Exception { + journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testPerformanceOfRandomAccessReads.gz"); + tocFile = TocUtil.getTocFile(journalFile); + + final int blockSize = 1024 * 32; + try (final RecordWriter writer = createWriter(journalFile, new StandardTocWriter(tocFile, true, false), true, blockSize)) { + writer.writeHeader(0L); + + for (int i = 0; i < 100_000; i++) { + writer.writeRecord(createEvent()); + } + } + + final long[] eventIds = new long[] { + 4, 80, 1024, 1025, 1026, 1027, 1028, 1029, 1030, 40_000, 80_000, 99_000 + }; + + boolean loopForever = true; + while (loopForever) { + final long start = System.nanoTime(); + for (int i = 0; i < 1000; i++) { + try (final InputStream in = new FileInputStream(journalFile); + final RecordReader reader = createReader(in, journalFile.getName(), new StandardTocReader(tocFile), 32 * 1024)) { + + for (final long id : eventIds) { + time(() -> { + reader.skipToEvent(id); + return reader.nextRecord(); + }, id); + } + } + } + + final long ms = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + System.out.println(ms + " ms total"); + } + } + + private void time(final Callable task, final long id) throws Exception { + final long start = System.nanoTime(); + final StandardProvenanceEventRecord event = task.call(); + Assert.assertNotNull(event); + Assert.assertEquals(id, event.getEventId()); + // System.out.println(event); + final long nanos = System.nanoTime() - start; + final long millis = TimeUnit.NANOSECONDS.toMillis(nanos); + // System.out.println("Took " + millis + " ms to " + taskDescription); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java index 00f46174d6..48d8e09aff 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; +import java.io.DataOutputStream; import java.io.File; import java.io.FileFilter; import java.io.FileInputStream; @@ -65,6 +66,8 @@ import org.apache.nifi.authorization.AccessDeniedException; import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.events.EventReporter; import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.EventIndexWriter; import org.apache.nifi.provenance.lineage.EventNode; import org.apache.nifi.provenance.lineage.Lineage; import org.apache.nifi.provenance.lineage.LineageEdge; @@ -83,7 +86,6 @@ import org.apache.nifi.provenance.serialization.RecordReaders; import org.apache.nifi.provenance.serialization.RecordWriter; import org.apache.nifi.provenance.serialization.RecordWriters; import org.apache.nifi.reporting.Severity; -import org.apache.nifi.stream.io.DataOutputStream; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.file.FileUtils; import org.junit.After; @@ -120,7 +122,7 @@ public class TestPersistentProvenanceRepository { private static RepositoryConfiguration createConfiguration() { config = new RepositoryConfiguration(); - config.addStorageDirectory(new File("target/storage/" + UUID.randomUUID().toString())); + config.addStorageDirectory("1", new File("target/storage/" + UUID.randomUUID().toString())); config.setCompressOnRollover(true); config.setMaxEventFileLife(2000L, TimeUnit.SECONDS); config.setCompressionBlockBytes(100); @@ -152,14 +154,15 @@ public class TestPersistentProvenanceRepository { final File tempRecordFile = tempFolder.newFile("record.tmp"); System.out.println("findJournalSizes position 0 = " + tempRecordFile.length()); - final RecordWriter writer = RecordWriters.newSchemaRecordWriter(tempRecordFile, false, false); + final AtomicLong idGenerator = new AtomicLong(0L); + final RecordWriter writer = RecordWriters.newSchemaRecordWriter(tempRecordFile, idGenerator, false, false); writer.writeHeader(12345L); writer.flush(); headerSize = Long.valueOf(tempRecordFile.length()).intValue(); - writer.writeRecord(record, 12345L); + writer.writeRecord(record); writer.flush(); recordSize = Long.valueOf(tempRecordFile.length()).intValue() - headerSize; - writer.writeRecord(record2, 23456L); + writer.writeRecord(record2); writer.flush(); recordSize2 = Long.valueOf(tempRecordFile.length()).intValue() - headerSize - recordSize; writer.close(); @@ -187,34 +190,45 @@ public class TestPersistentProvenanceRepository { @After public void closeRepo() throws IOException { - if (repo != null) { - try { - repo.close(); - } catch (final IOException ioe) { - } + if (repo == null) { + return; } + try { + repo.close(); + } catch (final IOException ioe) { + } + + // Delete all of the storage files. We do this in order to clean up the tons of files that + // we create but also to ensure that we have closed all of the file handles. If we leave any + // streams open, for instance, this will throw an IOException, causing our unit test to fail. if (config != null) { - // Delete all of the storage files. We do this in order to clean up the tons of files that - // we create but also to ensure that we have closed all of the file handles. If we leave any - // streams open, for instance, this will throw an IOException, causing our unit test to fail. - for (final File storageDir : config.getStorageDirectories()) { - if (storageDir.exists()) { - int i; - for (i = 0; i < 3; i++) { - try { - System.out.println("file: " + storageDir.toString() + " exists=" + storageDir.exists()); - FileUtils.deleteFile(storageDir, true); - break; - } catch (final IOException ioe) { - // if there is a virus scanner, etc. running in the background we may not be able to - // delete the file. Wait a sec and try again. - if (i == 2) { - throw ioe; - } else { - try { - Thread.sleep(1000L); - } catch (final InterruptedException ie) { + for (final File storageDir : config.getStorageDirectories().values()) { + int i; + for (i = 0; i < 3; i++) { + try { + FileUtils.deleteFile(storageDir, true); + break; + } catch (final IOException ioe) { + // if there is a virus scanner, etc. running in the background we may not be able to + // delete the file. Wait a sec and try again. + if (i == 2) { + throw ioe; + } else { + try { + System.out.println("file: " + storageDir.toString() + " exists=" + storageDir.exists()); + FileUtils.deleteFile(storageDir, true); + break; + } catch (final IOException ioe2) { + // if there is a virus scanner, etc. running in the background we may not be able to + // delete the file. Wait a sec and try again. + if (i == 2) { + throw ioe2; + } else { + try { + Thread.sleep(1000L); + } catch (final InterruptedException ie) { + } } } } @@ -240,7 +254,7 @@ public class TestPersistentProvenanceRepository { config.setJournalCount(10); config.setQueryThreadPoolSize(10); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final Map attributes = new HashMap<>(); attributes.put("abc", "xyz"); @@ -288,7 +302,7 @@ public class TestPersistentProvenanceRepository { System.out.println("Closing and re-initializing"); repo.close(); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); System.out.println("Re-initialized"); final long fetchStart = System.nanoTime(); @@ -311,7 +325,7 @@ public class TestPersistentProvenanceRepository { return "2000 millis"; } else if (key.equals(NiFiProperties.PROVENANCE_REPO_DIRECTORY_PREFIX + ".default")) { createConfiguration(); - return config.getStorageDirectories().get(0).getAbsolutePath(); + return config.getStorageDirectories().values().iterator().next().getAbsolutePath(); } else { return null; } @@ -340,8 +354,8 @@ public class TestPersistentProvenanceRepository { @Test public void constructorConfig() throws IOException { - RepositoryConfiguration configuration = createTestableRepositoryConfiguration(properties); - TestablePersistentProvenanceRepository tppr = new TestablePersistentProvenanceRepository(configuration, 20000); + RepositoryConfiguration configuration = RepositoryConfiguration.create(properties); + new TestablePersistentProvenanceRepository(configuration, 20000); } @Test @@ -350,7 +364,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1L); config.setMaxEventFileLife(1, TimeUnit.SECONDS); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final Map attributes = new HashMap<>(); attributes.put("abc", "xyz"); @@ -376,7 +390,7 @@ public class TestPersistentProvenanceRepository { Thread.sleep(500L); // Give the repo time to shutdown (i.e., close all file handles, etc.) repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final List recoveredRecords = repo.getEvents(0L, 12); assertEquals(10, recoveredRecords.size()); @@ -399,7 +413,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileLife(2, TimeUnit.SECONDS); config.setSearchableFields(searchableFields); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final Map attributes = new HashMap<>(); attributes.put("abc", "xyz"); @@ -454,7 +468,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); config.setSearchableAttributes(SearchableFieldParser.extractSearchableFields("immense", false)); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); int immenseAttrSize = 33000; // must be greater than 32766 for a meaningful test StringBuilder immenseBldr = new StringBuilder(immenseAttrSize); @@ -498,7 +512,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -542,7 +556,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setCompressOnRollover(true); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -565,7 +579,7 @@ public class TestPersistentProvenanceRepository { } repo.waitForRollover(); - final File storageDir = config.getStorageDirectories().get(0); + final File storageDir = config.getStorageDirectories().values().iterator().next(); final File compressedLogFile = new File(storageDir, "0.prov.gz"); assertTrue(compressedLogFile.exists()); } @@ -580,7 +594,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "10000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -653,8 +667,8 @@ public class TestPersistentProvenanceRepository { final AtomicInteger indexSearcherCount = new AtomicInteger(0); @Override - public IndexSearcher borrowIndexSearcher(File indexDir) throws IOException { - final IndexSearcher searcher = mgr.borrowIndexSearcher(indexDir); + public EventIndexSearcher borrowIndexSearcher(File indexDir) throws IOException { + final EventIndexSearcher searcher = mgr.borrowIndexSearcher(indexDir); final int idx = indexSearcherCount.incrementAndGet(); obtainIndexSearcherLatch.countDown(); @@ -677,7 +691,7 @@ public class TestPersistentProvenanceRepository { } @Override - public IndexWriter borrowIndexWriter(File indexingDirectory) throws IOException { + public EventIndexWriter borrowIndexWriter(File indexingDirectory) throws IOException { return mgr.borrowIndexWriter(indexingDirectory); } @@ -687,18 +701,19 @@ public class TestPersistentProvenanceRepository { } @Override - public void removeIndex(File indexDirectory) { + public boolean removeIndex(File indexDirectory) { mgr.removeIndex(indexDirectory); + return true; } @Override - public void returnIndexSearcher(File indexDirectory, IndexSearcher searcher) { - mgr.returnIndexSearcher(indexDirectory, searcher); + public void returnIndexSearcher(EventIndexSearcher searcher) { + mgr.returnIndexSearcher(searcher); } @Override - public void returnIndexWriter(File indexingDirectory, IndexWriter writer) { - mgr.returnIndexWriter(indexingDirectory, writer); + public void returnIndexWriter(EventIndexWriter writer) { + mgr.returnIndexWriter(writer); } }; } @@ -707,7 +722,7 @@ public class TestPersistentProvenanceRepository { } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "10000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -790,7 +805,7 @@ public class TestPersistentProvenanceRepository { @Test public void testIndexAndCompressOnRolloverAndSubsequentSearchMultipleStorageDirs() throws IOException, InterruptedException, ParseException { final RepositoryConfiguration config = createConfiguration(); - config.addStorageDirectory(new File("target/storage/" + UUID.randomUUID().toString())); + config.addStorageDirectory("2", new File("target/storage/" + UUID.randomUUID().toString())); config.setMaxRecordLife(30, TimeUnit.SECONDS); config.setMaxStorageCapacity(1024L * 1024L); config.setMaxEventFileLife(1, TimeUnit.SECONDS); @@ -798,7 +813,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -885,7 +900,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -941,7 +956,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000001"; final Map attributes = new HashMap<>(); @@ -996,7 +1011,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000001"; final Map attributes = new HashMap<>(); @@ -1055,7 +1070,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String childId = "00000000-0000-0000-0000-000000000000"; @@ -1105,7 +1120,7 @@ public class TestPersistentProvenanceRepository { config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String childId = "00000000-0000-0000-0000-000000000000"; @@ -1152,7 +1167,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(1, TimeUnit.SECONDS); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -1178,7 +1193,7 @@ public class TestPersistentProvenanceRepository { repo.close(); final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - secondRepo.initialize(getEventReporter(), null, null); + secondRepo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); try { final ProvenanceEventRecord event11 = builder.build(); @@ -1239,7 +1254,7 @@ public class TestPersistentProvenanceRepository { config.setDesiredIndexSize(10); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); String uuid = UUID.randomUUID().toString(); for (int i = 0; i < 20; i++) { @@ -1253,7 +1268,7 @@ public class TestPersistentProvenanceRepository { } } repo.waitForRollover(); - File eventFile = new File(config.getStorageDirectories().get(0), "10.prov.gz"); + File eventFile = new File(config.getStorageDirectories().values().iterator().next(), "10.prov.gz"); assertTrue(eventFile.delete()); return eventFile; } @@ -1270,7 +1285,7 @@ public class TestPersistentProvenanceRepository { config.setDesiredIndexSize(10); // force new index to be created for each rollover repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -1298,7 +1313,7 @@ public class TestPersistentProvenanceRepository { Thread.sleep(2000L); final FileFilter indexFileFilter = file -> file.getName().startsWith("index"); - final int numIndexDirs = config.getStorageDirectories().get(0).listFiles(indexFileFilter).length; + final int numIndexDirs = config.getStorageDirectories().values().iterator().next().listFiles(indexFileFilter).length; assertEquals(1, numIndexDirs); // add more records so that we will create a new index @@ -1324,7 +1339,7 @@ public class TestPersistentProvenanceRepository { assertEquals(20, result.getMatchingEvents().size()); // Ensure index directories exists - File[] indexDirs = config.getStorageDirectories().get(0).listFiles(indexFileFilter); + File[] indexDirs = config.getStorageDirectories().values().iterator().next().listFiles(indexFileFilter); assertEquals(2, indexDirs.length); // expire old events and indexes @@ -1337,7 +1352,7 @@ public class TestPersistentProvenanceRepository { assertEquals(10, newRecordSet.getMatchingEvents().size()); // Ensure that one index directory is gone - indexDirs = config.getStorageDirectories().get(0).listFiles(indexFileFilter); + indexDirs = config.getStorageDirectories().values().iterator().next().listFiles(indexFileFilter); assertEquals(1, indexDirs.length); } @@ -1354,12 +1369,12 @@ public class TestPersistentProvenanceRepository { final AccessDeniedException expectedException = new AccessDeniedException("Unit Test - Intentionally Thrown"); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) { @Override - protected void authorize(ProvenanceEventRecord event, NiFiUser user) { + public void authorize(ProvenanceEventRecord event, NiFiUser user) { throw expectedException; } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -1409,7 +1424,7 @@ public class TestPersistentProvenanceRepository { } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -1461,7 +1476,7 @@ public class TestPersistentProvenanceRepository { } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -1522,7 +1537,7 @@ public class TestPersistentProvenanceRepository { } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -1641,7 +1656,7 @@ public class TestPersistentProvenanceRepository { return journalCountRef.get(); } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final Map attributes = new HashMap<>(); final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); @@ -1697,7 +1712,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -1732,7 +1747,7 @@ public class TestPersistentProvenanceRepository { final List indexDirs = indexConfig.getIndexDirectories(); final String query = "uuid:00000000-0000-0000-0000-0000000000* AND NOT filename:file-?"; - final List results = runQuery(indexDirs.get(0), config.getStorageDirectories(), query); + final List results = runQuery(indexDirs.get(0), new ArrayList<>(config.getStorageDirectories().values()), query); assertEquals(6, results.size()); } @@ -1786,7 +1801,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(3, TimeUnit.SECONDS); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final Map attributes = new HashMap<>(); @@ -1813,7 +1828,7 @@ public class TestPersistentProvenanceRepository { repo.waitForRollover(); - final File storageDir = config.getStorageDirectories().get(0); + final File storageDir = config.getStorageDirectories().values().iterator().next(); long counter = 0; for (final File file : storageDir.listFiles()) { if (file.isFile()) { @@ -1853,7 +1868,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(3, TimeUnit.SECONDS); TestablePersistentProvenanceRepository testRepo = new TestablePersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - testRepo.initialize(getEventReporter(), null, null); + testRepo.initialize(getEventReporter(), null, null, null); final Map attributes = new HashMap<>(); @@ -1897,7 +1912,7 @@ public class TestPersistentProvenanceRepository { + "that the record wasn't completely written to the file. This journal will be skipped.", reportedEvents.get(reportedEvents.size() - 1).getMessage()); - final File storageDir = config.getStorageDirectories().get(0); + final File storageDir = config.getStorageDirectories().values().iterator().next(); assertTrue(checkJournalRecords(storageDir, false) < 10000); } @@ -1906,7 +1921,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(3, TimeUnit.SECONDS); TestablePersistentProvenanceRepository testRepo = new TestablePersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - testRepo.initialize(getEventReporter(), null, null); + testRepo.initialize(getEventReporter(), null, null, null); final Map attributes = new HashMap<>(); @@ -1951,7 +1966,7 @@ public class TestPersistentProvenanceRepository { + "be skipped.", reportedEvents.get(reportedEvents.size() - 1).getMessage()); - final File storageDir = config.getStorageDirectories().get(0); + final File storageDir = config.getStorageDirectories().values().iterator().next(); assertTrue(checkJournalRecords(storageDir, false) < 10000); } @@ -1960,7 +1975,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(3, TimeUnit.SECONDS); TestablePersistentProvenanceRepository testRepo = new TestablePersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - testRepo.initialize(getEventReporter(), null, null); + testRepo.initialize(getEventReporter(), null, null, null); final Map attributes = new HashMap<>(); @@ -1997,7 +2012,7 @@ public class TestPersistentProvenanceRepository { assertEquals("mergeJournals() should not error on empty journal", 0, reportedEvents.size()); - final File storageDir = config.getStorageDirectories().get(0); + final File storageDir = config.getStorageDirectories().values().iterator().next(); assertEquals(config.getJournalCount() - 1, checkJournalRecords(storageDir, true)); } @@ -2025,7 +2040,7 @@ public class TestPersistentProvenanceRepository { return 10L; // retry quickly. } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final Map attributes = new HashMap<>(); @@ -2062,7 +2077,7 @@ public class TestPersistentProvenanceRepository { config.setMaxAttributeChars(50); config.setMaxEventFileLife(3, TimeUnit.SECONDS); repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final String maxLengthChars = "12345678901234567890123456789012345678901234567890"; final Map attributes = new HashMap<>(); @@ -2108,7 +2123,7 @@ public class TestPersistentProvenanceRepository { repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS) { @Override protected synchronized IndexingAction createIndexingAction() { - return new IndexingAction(repo) { + return new IndexingAction(config.getSearchableFields(), config.getSearchableAttributes()) { @Override public void index(StandardProvenanceEventRecord record, IndexWriter indexWriter, Integer blockIndex) throws IOException { final int count = indexedEventCount.incrementAndGet(); @@ -2121,7 +2136,7 @@ public class TestPersistentProvenanceRepository { }; } }; - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); final Map attributes = new HashMap<>(); attributes.put("uuid", "12345678-0000-0000-0000-012345678912"); @@ -2169,7 +2184,7 @@ public class TestPersistentProvenanceRepository { }; // initialize with our event reporter - repo.initialize(getEventReporter(), null, null); + repo.initialize(getEventReporter(), null, null, IdentifierLookup.EMPTY); // create some events in the journal files. final Map attributes = new HashMap<>(); @@ -2219,10 +2234,12 @@ public class TestPersistentProvenanceRepository { this.message = message; } + @SuppressWarnings("unused") public String getCategory() { return category; } + @SuppressWarnings("unused") public String getMessage() { return message; } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestSchemaRecordReaderWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestSchemaRecordReaderWriter.java index aed690b4d6..2eb353e0ea 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestSchemaRecordReaderWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestSchemaRecordReaderWriter.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -34,11 +35,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import org.apache.nifi.provenance.schema.EventFieldNames; import org.apache.nifi.provenance.schema.EventRecord; -import org.apache.nifi.provenance.schema.EventRecordFields; import org.apache.nifi.provenance.schema.ProvenanceEventSchema; import org.apache.nifi.provenance.serialization.RecordReader; import org.apache.nifi.provenance.serialization.RecordWriter; @@ -55,14 +58,14 @@ import org.apache.nifi.repository.schema.RecordField; import org.apache.nifi.repository.schema.RecordSchema; import org.apache.nifi.repository.schema.Repetition; import org.apache.nifi.repository.schema.SimpleRecordField; -import org.apache.nifi.stream.io.DataOutputStream; import org.apache.nifi.stream.io.NullOutputStream; +import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter { - + private final AtomicLong idGenerator = new AtomicLong(0L); private File journalFile; private File tocFile; @@ -70,6 +73,58 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter public void setup() { journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testFieldAddedToSchema"); tocFile = TocUtil.getTocFile(journalFile); + idGenerator.set(0L); + } + + + @Test + @Ignore("runs forever for performance analysis/profiling") + public void testPerformanceOfRandomAccessReads() throws Exception { + journalFile = new File("target/storage/" + UUID.randomUUID().toString() + "/testPerformanceOfRandomAccessReads.gz"); + tocFile = TocUtil.getTocFile(journalFile); + + try (final RecordWriter writer = createWriter(journalFile, new StandardTocWriter(tocFile, true, false), true, 1024 * 32)) { + writer.writeHeader(0L); + + for (int i = 0; i < 100_000; i++) { + writer.writeRecord(createEvent()); + } + } + + final long[] eventIds = new long[] { + 4, 80, 1024, 1025, 1026, 1027, 1028, 1029, 1030, 40_000, 80_000, 99_000 + }; + + boolean loopForever = true; + while (loopForever) { + final long start = System.nanoTime(); + for (int i = 0; i < 1000; i++) { + try (final InputStream in = new FileInputStream(journalFile); + final RecordReader reader = createReader(in, journalFile.getName(), new StandardTocReader(tocFile), 32 * 1024)) { + + for (final long id : eventIds) { + time(() -> { + reader.skipToEvent(id); + return reader.nextRecord(); + }, id); + } + } + } + + final long ms = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + System.out.println(ms + " ms total"); + } + } + + private void time(final Callable task, final long id) throws Exception { + final long start = System.nanoTime(); + final StandardProvenanceEventRecord event = task.call(); + Assert.assertNotNull(event); + Assert.assertEquals(id, event.getEventId()); + // System.out.println(event); + final long nanos = System.nanoTime() - start; + final long millis = TimeUnit.NANOSECONDS.toMillis(nanos); + // System.out.println("Took " + millis + " ms to " + taskDescription); } @@ -83,8 +138,8 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter try (final ByteArraySchemaRecordWriter writer = createSchemaWriter(schemaModifier, toAdd)) { writer.writeHeader(1L); - writer.writeRecord(createEvent(), 3L); - writer.writeRecord(createEvent(), 3L); + writer.writeRecord(createEvent()); + writer.writeRecord(createEvent()); } try (final InputStream in = new FileInputStream(journalFile); @@ -94,7 +149,6 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter for (int i = 0; i < 2; i++) { final StandardProvenanceEventRecord event = reader.nextRecord(); assertNotNull(event); - assertEquals(3L, event.getEventId()); assertEquals("1234", event.getComponentId()); assertEquals(ProvenanceEventType.RECEIVE, event.getEventType()); @@ -111,14 +165,14 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter // Create a schema that has the fields modified final RecordSchema schemaV1 = ProvenanceEventSchema.PROVENANCE_EVENT_SCHEMA_V1; final List fields = new ArrayList<>(schemaV1.getFields()); - fields.remove(new SimpleRecordField(EventRecordFields.Names.UPDATED_ATTRIBUTES, FieldType.STRING, Repetition.EXACTLY_ONE)); - fields.remove(new SimpleRecordField(EventRecordFields.Names.PREVIOUS_ATTRIBUTES, FieldType.STRING, Repetition.EXACTLY_ONE)); + fields.remove(new SimpleRecordField(EventFieldNames.UPDATED_ATTRIBUTES, FieldType.STRING, Repetition.EXACTLY_ONE)); + fields.remove(new SimpleRecordField(EventFieldNames.PREVIOUS_ATTRIBUTES, FieldType.STRING, Repetition.EXACTLY_ONE)); final RecordSchema recordSchema = new RecordSchema(fields); // Create a record writer whose schema does not contain updated attributes or previous attributes. // This means that we must also override the method that writes out attributes so that we are able // to avoid actually writing them out. - final ByteArraySchemaRecordWriter writer = new ByteArraySchemaRecordWriter(journalFile, tocWriter, false, 0) { + final ByteArraySchemaRecordWriter writer = new ByteArraySchemaRecordWriter(journalFile, idGenerator, tocWriter, false, 0) { @Override public void writeHeader(long firstEventId, DataOutputStream out) throws IOException { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -130,15 +184,15 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter @Override protected Record createRecord(final ProvenanceEventRecord event, final long eventId) { - final RecordSchema contentClaimSchema = new RecordSchema(recordSchema.getField(EventRecordFields.Names.CONTENT_CLAIM).getSubFields()); + final RecordSchema contentClaimSchema = new RecordSchema(recordSchema.getField(EventFieldNames.CONTENT_CLAIM).getSubFields()); return new EventRecord(event, eventId, recordSchema, contentClaimSchema); } }; try { writer.writeHeader(1L); - writer.writeRecord(createEvent(), 3L); - writer.writeRecord(createEvent(), 3L); + writer.writeRecord(createEvent()); + writer.writeRecord(createEvent()); } finally { writer.close(); } @@ -154,7 +208,6 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter for (int i = 0; i < 2; i++) { final StandardProvenanceEventRecord event = reader.nextRecord(); assertNotNull(event); - assertEquals(3L, event.getEventId()); assertEquals(ProvenanceEventType.RECEIVE, event.getEventType()); // We will still have a Map for updated attributes because the @@ -175,7 +228,7 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter try (final ByteArraySchemaRecordWriter writer = createSchemaWriter(schemaModifier, toAdd)) { writer.writeHeader(1L); - writer.writeRecord(createEvent(), 3L); + writer.writeRecord(createEvent()); } try (final InputStream in = new FileInputStream(journalFile); @@ -207,9 +260,9 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter fieldModifier.accept(fields); final RecordSchema recordSchema = new RecordSchema(fields); - final RecordSchema contentClaimSchema = new RecordSchema(recordSchema.getField(EventRecordFields.Names.CONTENT_CLAIM).getSubFields()); + final RecordSchema contentClaimSchema = new RecordSchema(recordSchema.getField(EventFieldNames.CONTENT_CLAIM).getSubFields()); - final ByteArraySchemaRecordWriter writer = new ByteArraySchemaRecordWriter(journalFile, tocWriter, false, 0) { + final ByteArraySchemaRecordWriter writer = new ByteArraySchemaRecordWriter(journalFile, idGenerator, tocWriter, false, 0) { @Override public void writeHeader(long firstEventId, DataOutputStream out) throws IOException { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -250,14 +303,13 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter final int numEvents = 10_000_000; final long startNanos = System.nanoTime(); try (final OutputStream nullOut = new NullOutputStream(); - final RecordWriter writer = new ByteArraySchemaRecordWriter(nullOut, tocWriter, false, 0)) { + final RecordWriter writer = new ByteArraySchemaRecordWriter(nullOut, "out", idGenerator, tocWriter, false, 0)) { writer.writeHeader(0L); for (int i = 0; i < numEvents; i++) { - writer.writeRecord(event, i); + writer.writeRecord(event); } - } final long nanos = System.nanoTime() - startNanos; @@ -280,7 +332,7 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter try (final ByteArrayOutputStream headerOut = new ByteArrayOutputStream(); final DataOutputStream out = new DataOutputStream(headerOut)) { - final RecordWriter schemaWriter = new ByteArraySchemaRecordWriter(out, null, false, 0); + final RecordWriter schemaWriter = new ByteArraySchemaRecordWriter(out, "out", idGenerator, null, false, 0); schemaWriter.writeHeader(1L); header = headerOut.toByteArray(); @@ -288,12 +340,12 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter final byte[] serializedRecord; try (final ByteArrayOutputStream headerOut = new ByteArrayOutputStream(); - final RecordWriter writer = new ByteArraySchemaRecordWriter(headerOut, null, false, 0)) { + final RecordWriter writer = new ByteArraySchemaRecordWriter(headerOut, "out", idGenerator, null, false, 0)) { writer.writeHeader(1L); headerOut.reset(); - writer.writeRecord(event, 1L); + writer.writeRecord(event); writer.flush(); serializedRecord = headerOut.toByteArray(); } @@ -322,7 +374,7 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter @Override protected RecordWriter createWriter(File file, TocWriter tocWriter, boolean compressed, int uncompressedBlockSize) throws IOException { - return new ByteArraySchemaRecordWriter(file, tocWriter, compressed, uncompressedBlockSize); + return new ByteArraySchemaRecordWriter(file, idGenerator, tocWriter, compressed, uncompressedBlockSize); } @@ -331,11 +383,4 @@ public class TestSchemaRecordReaderWriter extends AbstractTestRecordReaderWriter final ByteArraySchemaRecordReader reader = new ByteArraySchemaRecordReader(in, journalFilename, tocReader, maxAttributeSize); return reader; } - - private static interface WriteRecordInterceptor { - void writeRawRecord(ProvenanceEventRecord event, long recordIdentifier, DataOutputStream out) throws IOException; - } - - private static WriteRecordInterceptor NOP_INTERCEPTOR = (event, id, out) -> {}; - private static WriteRecordInterceptor WRITE_DUMMY_STRING_INTERCEPTOR = (event, id, out) -> out.writeUTF("hello"); } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java index dfa37e4798..27002c8740 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestStandardRecordReaderWriter.java @@ -16,7 +16,11 @@ */ package org.apache.nifi.provenance; +import static org.apache.nifi.provenance.TestUtil.createFlowFile; +import static org.junit.Assert.assertTrue; + import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -25,6 +29,7 @@ import java.util.HashMap; import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.provenance.serialization.RecordReader; @@ -32,16 +37,18 @@ import org.apache.nifi.provenance.serialization.RecordWriter; import org.apache.nifi.provenance.toc.NopTocWriter; import org.apache.nifi.provenance.toc.TocReader; import org.apache.nifi.provenance.toc.TocWriter; -import org.apache.nifi.stream.io.DataOutputStream; import org.apache.nifi.stream.io.NullOutputStream; +import org.junit.Before; import org.junit.Ignore; import org.junit.Test; -import static org.apache.nifi.provenance.TestUtil.createFlowFile; -import static org.junit.Assert.assertTrue; - public class TestStandardRecordReaderWriter extends AbstractTestRecordReaderWriter { + private AtomicLong idGenerator = new AtomicLong(0L); + @Before + public void resetIds() { + idGenerator.set(0L); + } @Test @Ignore("For local testing only") @@ -56,12 +63,12 @@ public class TestStandardRecordReaderWriter extends AbstractTestRecordReaderWrit final int numEvents = 10_000_000; final long startNanos = System.nanoTime(); try (final OutputStream nullOut = new NullOutputStream(); - final RecordWriter writer = new StandardRecordWriter(nullOut, tocWriter, false, 100000)) { + final RecordWriter writer = new StandardRecordWriter(nullOut, "devnull", idGenerator, tocWriter, false, 100000)) { writer.writeHeader(0L); for (int i = 0; i < numEvents; i++) { - writer.writeRecord(event, i); + writer.writeRecord(event); } } @@ -90,12 +97,12 @@ public class TestStandardRecordReaderWriter extends AbstractTestRecordReaderWrit final byte[] serializedRecord; try (final ByteArrayOutputStream headerOut = new ByteArrayOutputStream(); - final StandardRecordWriter writer = new StandardRecordWriter(headerOut, null, false, 0)) { + final StandardRecordWriter writer = new StandardRecordWriter(headerOut, "devnull", idGenerator, null, false, 0)) { writer.writeHeader(1L); headerOut.reset(); - writer.writeRecord(event, 1L); + writer.writeRecord(event); writer.flush(); serializedRecord = headerOut.toByteArray(); } @@ -142,18 +149,18 @@ public class TestStandardRecordReaderWriter extends AbstractTestRecordReaderWrit } try (final ByteArrayOutputStream recordOut = new ByteArrayOutputStream(); - final StandardRecordWriter writer = new StandardRecordWriter(recordOut, null, false, 0)) { + final StandardRecordWriter writer = new StandardRecordWriter(recordOut, "devnull", idGenerator, null, false, 0)) { writer.writeHeader(1L); recordOut.reset(); - writer.writeRecord(record, 1L); + writer.writeRecord(record); } } @Override protected RecordWriter createWriter(File file, TocWriter tocWriter, boolean compressed, int uncompressedBlockSize) throws IOException { - return new StandardRecordWriter(file, tocWriter, compressed, uncompressedBlockSize); + return new StandardRecordWriter(file, idGenerator, tocWriter, compressed, uncompressedBlockSize); } @Override diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java index 514a43eaf6..224ee713f1 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestUtil.java @@ -18,6 +18,7 @@ package org.apache.nifi.provenance; import java.util.HashMap; import java.util.Map; +import java.util.UUID; import org.apache.nifi.flowfile.FlowFile; @@ -82,4 +83,22 @@ public class TestUtil { } }; } + + public static ProvenanceEventRecord createEvent() { + final Map attributes = new HashMap<>(); + attributes.put("filename", "1.txt"); + attributes.put("uuid", UUID.randomUUID().toString()); + + final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder(); + builder.setEventTime(System.currentTimeMillis()); + builder.setEventType(ProvenanceEventType.RECEIVE); + builder.setTransitUri("nifi://unit-test"); + builder.fromFlowFile(createFlowFile(3L, 3000L, attributes)); + builder.setComponentId("1234"); + builder.setComponentType("dummy processor"); + final ProvenanceEventRecord record = builder.build(); + + return record; + } + } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestEventIndexTask.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestEventIndexTask.java new file mode 100644 index 0000000000..4c58b13269 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestEventIndexTask.java @@ -0,0 +1,142 @@ +/* + * 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.index.lucene; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field.Store; +import org.apache.lucene.document.LongField; +import org.apache.lucene.index.IndexWriter; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.index.EventIndexWriter; +import org.apache.nifi.provenance.lucene.IndexManager; +import org.apache.nifi.provenance.lucene.LuceneEventIndexWriter; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mockito; + +public class TestEventIndexTask { + + @BeforeClass + public static void setupClass() { + System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi", "DEBUG"); + } + + @Test(timeout = 5000) + public void testIndexWriterCommittedWhenAppropriate() throws IOException, InterruptedException { + final BlockingQueue docQueue = new LinkedBlockingQueue<>(); + final RepositoryConfiguration repoConfig = new RepositoryConfiguration(); + final File storageDir = new File("target/storage/TestEventIndexTask/1"); + repoConfig.addStorageDirectory("1", storageDir); + + final AtomicInteger commitCount = new AtomicInteger(0); + + // Mock out an IndexWriter and keep track of the number of events that are indexed. + final IndexWriter indexWriter = Mockito.mock(IndexWriter.class); + final EventIndexWriter eventIndexWriter = new LuceneEventIndexWriter(indexWriter, storageDir); + + final IndexManager indexManager = Mockito.mock(IndexManager.class); + Mockito.when(indexManager.borrowIndexWriter(Mockito.any(File.class))).thenReturn(eventIndexWriter); + + final IndexDirectoryManager directoryManager = new IndexDirectoryManager(repoConfig); + + // Create an EventIndexTask and override the commit(IndexWriter) method so that we can keep track of how + // many times the index writer gets committed. + final EventIndexTask task = new EventIndexTask(docQueue, repoConfig, indexManager, directoryManager, 201, EventReporter.NO_OP) { + @Override + protected void commit(EventIndexWriter indexWriter) throws IOException { + commitCount.incrementAndGet(); + } + }; + + // Create 4 threads, each one a daemon thread running the EventIndexTask + for (int i = 0; i < 4; i++) { + final Thread t = new Thread(task); + t.setDaemon(true); + t.start(); + } + + assertEquals(0, commitCount.get()); + + // Index 100 documents with a storage filename of "0.0.prov" + for (int i = 0; i < 100; i++) { + final Document document = new Document(); + document.add(new LongField(SearchableFields.EventTime.getSearchableFieldName(), System.currentTimeMillis(), Store.NO)); + + final StorageSummary location = new StorageSummary(1L, "0.0.prov", "1", 0, 1000L, 1000L); + final StoredDocument storedDoc = new StoredDocument(document, location); + docQueue.add(storedDoc); + } + assertEquals(0, commitCount.get()); + + // Index 100 documents + for (int i = 0; i < 100; i++) { + final Document document = new Document(); + document.add(new LongField(SearchableFields.EventTime.getSearchableFieldName(), System.currentTimeMillis(), Store.NO)); + + final StorageSummary location = new StorageSummary(1L, "0.0.prov", "1", 0, 1000L, 1000L); + final StoredDocument storedDoc = new StoredDocument(document, location); + docQueue.add(storedDoc); + } + + // Wait until we've indexed all 200 events + while (eventIndexWriter.getEventsIndexed() < 200) { + Thread.sleep(10L); + } + + // Wait a bit and make sure that we still haven't committed the index writer. + Thread.sleep(100L); + assertEquals(0, commitCount.get()); + + // Add another document. + final Document document = new Document(); + document.add(new LongField(SearchableFields.EventTime.getSearchableFieldName(), System.currentTimeMillis(), Store.NO)); + final StorageSummary location = new StorageSummary(1L, "0.0.prov", "1", 0, 1000L, 1000L); + + StoredDocument storedDoc = new StoredDocument(document, location); + docQueue.add(storedDoc); + + // Wait until index writer is committed. + while (commitCount.get() == 0) { + Thread.sleep(10L); + } + assertEquals(1, commitCount.get()); + + // Add a new IndexableDocument with a count of 1 to ensure that the writer is committed again. + storedDoc = new StoredDocument(document, location); + docQueue.add(storedDoc); + Thread.sleep(100L); + assertEquals(1, commitCount.get()); + + // Add a new IndexableDocument with a count of 3. Index writer should not be committed again. + storedDoc = new StoredDocument(document, location); + docQueue.add(storedDoc); + Thread.sleep(100L); + assertEquals(1, commitCount.get()); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestIndexDirectoryManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestIndexDirectoryManager.java new file mode 100644 index 0000000000..3f3c42296c --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestIndexDirectoryManager.java @@ -0,0 +1,100 @@ +/* + * 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.index.lucene; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.junit.Test; + +public class TestIndexDirectoryManager { + + @Test + public void testGetDirectoriesIncludesMatchingTimestampPlusOne() { + final List locations = new ArrayList<>(); + locations.add(createLocation(999L)); + locations.add(createLocation(1002L)); + locations.add(createLocation(1005L)); + + final List directories = IndexDirectoryManager.getDirectories(1000L, 1001L, locations); + assertEquals(2, directories.size()); + assertTrue(directories.contains(new File("index-999"))); + assertTrue(directories.contains(new File("index-1002"))); + } + + @Test + public void testGetDirectoriesOnlyObtainsDirectoriesForDesiredPartition() { + final RepositoryConfiguration config = createConfig(2); + + final File storageDir1 = config.getStorageDirectories().get("1"); + final File storageDir2 = config.getStorageDirectories().get("2"); + + final File index1 = new File(storageDir1, "index-1"); + final File index2 = new File(storageDir1, "index-2"); + final File index3 = new File(storageDir2, "index-3"); + final File index4 = new File(storageDir2, "index-4"); + + final File[] allIndices = new File[] {index1, index2, index3, index4}; + for (final File file : allIndices) { + assertTrue(file.mkdirs() || file.exists()); + } + + try { + final IndexDirectoryManager mgr = new IndexDirectoryManager(config); + mgr.initialize(); + + final List indexes1 = mgr.getDirectories(0L, Long.MAX_VALUE, "1"); + final List indexes2 = mgr.getDirectories(0L, Long.MAX_VALUE, "2"); + + assertEquals(2, indexes1.size()); + assertTrue(indexes1.contains(index1)); + assertTrue(indexes1.contains(index2)); + + assertEquals(2, indexes2.size()); + assertTrue(indexes2.contains(index3)); + assertTrue(indexes2.contains(index4)); + } finally { + for (final File file : allIndices) { + file.delete(); + } + } + } + + + private IndexLocation createLocation(final long timestamp) { + return createLocation(timestamp, "1"); + } + + private IndexLocation createLocation(final long timestamp, final String partitionName) { + return new IndexLocation(new File("index-" + timestamp), timestamp, partitionName, 1024 * 1024L); + } + + private RepositoryConfiguration createConfig(final int partitions) { + final RepositoryConfiguration repoConfig = new RepositoryConfiguration(); + for (int i = 1; i <= partitions; i++) { + repoConfig.addStorageDirectory(String.valueOf(i), new File("target/storage/testIndexDirectoryManager/" + UUID.randomUUID() + "/" + i)); + } + return repoConfig; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java new file mode 100644 index 0000000000..c892376367 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/index/lucene/TestLuceneEventIndex.java @@ -0,0 +1,538 @@ +/* + * 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.index.lucene; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.SearchableFields; +import org.apache.nifi.provenance.StandardProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +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.lucene.IndexManager; +import org.apache.nifi.provenance.lucene.SimpleIndexManager; +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.serialization.StorageSummary; +import org.apache.nifi.provenance.store.ArrayListEventStore; +import org.apache.nifi.provenance.store.EventStore; +import org.apache.nifi.provenance.store.StorageResult; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class TestLuceneEventIndex { + + private final AtomicLong idGenerator = new AtomicLong(0L); + + @Rule + public TestName testName = new TestName(); + + @BeforeClass + public static void setLogger() { + System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi", "DEBUG"); + } + + + @Test(timeout = 5000) + public void testGetMinimumIdToReindex() throws InterruptedException { + final RepositoryConfiguration repoConfig = createConfig(1); + repoConfig.setDesiredIndexSize(1L); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final ArrayListEventStore eventStore = new ArrayListEventStore(); + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 20_000, EventReporter.NO_OP); + index.initialize(eventStore); + + for (int i = 0; i < 50_000; i++) { + final ProvenanceEventRecord event = createEvent("1234"); + final StorageResult storageResult = eventStore.addEvent(event); + index.addEvents(storageResult.getStorageLocations()); + } + + while (index.getMaxEventId("1") < 40_000L) { + Thread.sleep(25); + } + + final long id = index.getMinimumEventIdToReindex("1"); + assertTrue(id >= 30000L); + } + + @Test(timeout = 5000) + public void testUnauthorizedEventsGetPlaceholdersForLineage() throws InterruptedException { + final RepositoryConfiguration repoConfig = createConfig(1); + repoConfig.setDesiredIndexSize(1L); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final ArrayListEventStore eventStore = new ArrayListEventStore(); + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 3, EventReporter.NO_OP); + index.initialize(eventStore); + + for (int i = 0; i < 3; i++) { + final ProvenanceEventRecord event = createEvent("1234"); + final StorageResult storageResult = eventStore.addEvent(event); + index.addEvents(storageResult.getStorageLocations()); + } + + final NiFiUser user = createUser(); + + List nodes = Collections.emptyList(); + while (nodes.size() < 3) { + final ComputeLineageSubmission submission = index.submitLineageComputation(1L, user, EventAuthorizer.DENY_ALL); + assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS)); + + nodes = submission.getResult().getNodes(); + Thread.sleep(25L); + } + + assertEquals(3, nodes.size()); + + for (final LineageNode node : nodes) { + assertEquals(LineageNodeType.PROVENANCE_EVENT_NODE, node.getNodeType()); + final ProvenanceEventLineageNode eventNode = (ProvenanceEventLineageNode) node; + assertEquals(ProvenanceEventType.UNKNOWN, eventNode.getEventType()); + } + } + + @Test(timeout = 5000) + public void testUnauthorizedEventsGetPlaceholdersForExpandChildren() throws InterruptedException { + final RepositoryConfiguration repoConfig = createConfig(1); + repoConfig.setDesiredIndexSize(1L); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final ArrayListEventStore eventStore = new ArrayListEventStore(); + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 3, EventReporter.NO_OP); + index.initialize(eventStore); + + final ProvenanceEventRecord firstEvent = createEvent("4444"); + + final Map previousAttributes = new HashMap<>(); + previousAttributes.put("uuid", "4444"); + final Map updatedAttributes = new HashMap<>(); + updatedAttributes.put("updated", "true"); + final ProvenanceEventRecord fork = new StandardProvenanceEventRecord.Builder() + .setEventType(ProvenanceEventType.FORK) + .setAttributes(previousAttributes, updatedAttributes) + .addChildFlowFile("1234") + .setComponentId("component-1") + .setComponentType("unit test") + .setEventId(idGenerator.getAndIncrement()) + .setEventTime(System.currentTimeMillis()) + .setFlowFileEntryDate(System.currentTimeMillis()) + .setFlowFileUUID("4444") + .setLineageStartDate(System.currentTimeMillis()) + .setCurrentContentClaim("container", "section", "unit-test-id", 0L, 1024L) + .build(); + + index.addEvents(eventStore.addEvent(firstEvent).getStorageLocations()); + index.addEvents(eventStore.addEvent(fork).getStorageLocations()); + + for (int i = 0; i < 3; i++) { + final ProvenanceEventRecord event = createEvent("1234"); + final StorageResult storageResult = eventStore.addEvent(event); + index.addEvents(storageResult.getStorageLocations()); + } + + final NiFiUser user = createUser(); + + final EventAuthorizer allowForkEvents = new EventAuthorizer() { + @Override + public boolean isAuthorized(ProvenanceEventRecord event) { + return event.getEventType() == ProvenanceEventType.FORK; + } + + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + } + }; + + List nodes = Collections.emptyList(); + while (nodes.size() < 5) { + final ComputeLineageSubmission submission = index.submitExpandChildren(1L, user, allowForkEvents); + assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS)); + + nodes = submission.getResult().getNodes(); + Thread.sleep(25L); + } + + assertEquals(5, nodes.size()); + + assertEquals(1L, nodes.stream().filter(n -> n.getNodeType() == LineageNodeType.FLOWFILE_NODE).count()); + assertEquals(4L, nodes.stream().filter(n -> n.getNodeType() == LineageNodeType.PROVENANCE_EVENT_NODE).count()); + + final Map> eventMap = nodes.stream() + .filter(n -> n.getNodeType() == LineageNodeType.PROVENANCE_EVENT_NODE) + .collect(Collectors.groupingBy(n -> ((ProvenanceEventLineageNode) n).getEventType())); + + assertEquals(2, eventMap.size()); + assertEquals(1, eventMap.get(ProvenanceEventType.FORK).size()); + assertEquals(3, eventMap.get(ProvenanceEventType.UNKNOWN).size()); + } + + @Test(timeout = 5000) + public void testUnauthorizedEventsGetPlaceholdersForFindParents() throws InterruptedException { + final RepositoryConfiguration repoConfig = createConfig(1); + repoConfig.setDesiredIndexSize(1L); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final ArrayListEventStore eventStore = new ArrayListEventStore(); + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 3, EventReporter.NO_OP); + index.initialize(eventStore); + + final ProvenanceEventRecord firstEvent = createEvent("4444"); + + final Map previousAttributes = new HashMap<>(); + previousAttributes.put("uuid", "4444"); + final Map updatedAttributes = new HashMap<>(); + updatedAttributes.put("updated", "true"); + final ProvenanceEventRecord join = new StandardProvenanceEventRecord.Builder() + .setEventType(ProvenanceEventType.JOIN) + .setAttributes(previousAttributes, updatedAttributes) + .addParentUuid("4444") + .addChildFlowFile("1234") + .setComponentId("component-1") + .setComponentType("unit test") + .setEventId(idGenerator.getAndIncrement()) + .setEventTime(System.currentTimeMillis()) + .setFlowFileEntryDate(System.currentTimeMillis()) + .setFlowFileUUID("1234") + .setLineageStartDate(System.currentTimeMillis()) + .setCurrentContentClaim("container", "section", "unit-test-id", 0L, 1024L) + .build(); + + index.addEvents(eventStore.addEvent(firstEvent).getStorageLocations()); + index.addEvents(eventStore.addEvent(join).getStorageLocations()); + + for (int i = 0; i < 3; i++) { + final ProvenanceEventRecord event = createEvent("1234"); + final StorageResult storageResult = eventStore.addEvent(event); + index.addEvents(storageResult.getStorageLocations()); + } + + final NiFiUser user = createUser(); + + final EventAuthorizer allowJoinEvents = new EventAuthorizer() { + @Override + public boolean isAuthorized(ProvenanceEventRecord event) { + return event.getEventType() == ProvenanceEventType.JOIN; + } + + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + } + }; + + List nodes = Collections.emptyList(); + while (nodes.size() < 2) { + final ComputeLineageSubmission submission = index.submitExpandParents(1L, user, allowJoinEvents); + assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS)); + + nodes = submission.getResult().getNodes(); + Thread.sleep(25L); + } + + assertEquals(2, nodes.size()); + + final Map> eventMap = nodes.stream() + .filter(n -> n.getNodeType() == LineageNodeType.PROVENANCE_EVENT_NODE) + .collect(Collectors.groupingBy(n -> ((ProvenanceEventLineageNode) n).getEventType())); + + assertEquals(2, eventMap.size()); + assertEquals(1, eventMap.get(ProvenanceEventType.JOIN).size()); + assertEquals(1, eventMap.get(ProvenanceEventType.UNKNOWN).size()); + + assertEquals("4444", eventMap.get(ProvenanceEventType.UNKNOWN).get(0).getFlowFileUuid()); + } + + @Test(timeout = 5000) + public void testUnauthorizedEventsGetFilteredForQuery() throws InterruptedException { + final RepositoryConfiguration repoConfig = createConfig(1); + repoConfig.setDesiredIndexSize(1L); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final ArrayListEventStore eventStore = new ArrayListEventStore(); + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 3, EventReporter.NO_OP); + index.initialize(eventStore); + + for (int i = 0; i < 3; i++) { + final ProvenanceEventRecord event = createEvent("1234"); + final StorageResult storageResult = eventStore.addEvent(event); + index.addEvents(storageResult.getStorageLocations()); + } + + final Query query = new Query(UUID.randomUUID().toString()); + final EventAuthorizer authorizer = new EventAuthorizer() { + @Override + public boolean isAuthorized(ProvenanceEventRecord event) { + return event.getEventId() % 2 == 0; + } + + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + throw new AccessDeniedException(); + } + }; + + List events = Collections.emptyList(); + while (events.size() < 2) { + final QuerySubmission submission = index.submitQuery(query, authorizer, "unit test"); + assertTrue(submission.getResult().awaitCompletion(5, TimeUnit.SECONDS)); + events = submission.getResult().getMatchingEvents(); + Thread.sleep(25L); + } + + assertEquals(2, events.size()); + } + + + private NiFiUser createUser() { + return new NiFiUser() { + @Override + public String getIdentity() { + return "unit test"; + } + + @Override + public NiFiUser getChain() { + return null; + } + + @Override + public boolean isAnonymous() { + return false; + } + + @Override + public String getClientAddress() { + return "127.0.0.1"; + } + }; + } + + + @Test(timeout = 5000) + public void testExpiration() throws InterruptedException, IOException { + final RepositoryConfiguration repoConfig = createConfig(1); + repoConfig.setDesiredIndexSize(1L); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 1, EventReporter.NO_OP); + + final List events = new ArrayList<>(); + events.add(createEvent(500000L)); + events.add(createEvent()); + + final EventStore eventStore = Mockito.mock(EventStore.class); + Mockito.doAnswer(new Answer>() { + @Override + public List answer(final InvocationOnMock invocation) throws Throwable { + final Long eventId = invocation.getArgumentAt(0, Long.class); + assertEquals(0, eventId.longValue()); + assertEquals(1, invocation.getArgumentAt(1, Integer.class).intValue()); + return Collections.singletonList(events.get(0)); + } + }).when(eventStore).getEvents(Mockito.anyLong(), Mockito.anyInt()); + + index.initialize(eventStore); + index.addEvent(events.get(0), createStorageSummary(events.get(0).getEventId())); + + // Add the first event to the index and wait for it to be indexed, since indexing is asynchronous. + List allDirectories = Collections.emptyList(); + while (allDirectories.isEmpty()) { + allDirectories = index.getDirectoryManager().getDirectories(null, null); + } + + events.remove(0); // Remove the first event from the store + index.performMaintenance(); + assertEquals(1, index.getDirectoryManager().getDirectories(null, null).size()); + } + + private StorageSummary createStorageSummary(final long eventId) { + return new StorageSummary(eventId, "1.prov", "1", 1, 2L, 2L); + } + + + @Test(timeout = 5000) + public void addThenQueryWithEmptyQuery() throws InterruptedException { + final RepositoryConfiguration repoConfig = createConfig(); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 1, EventReporter.NO_OP); + + final ProvenanceEventRecord event = createEvent(); + + index.addEvent(event, new StorageSummary(event.getEventId(), "1.prov", "1", 1, 2L, 2L)); + + final Query query = new Query(UUID.randomUUID().toString()); + + final ArrayListEventStore eventStore = new ArrayListEventStore(); + eventStore.addEvent(event); + index.initialize(eventStore); + + // We don't know how long it will take for the event to be indexed, so keep querying until + // we get a result. The test will timeout after 5 seconds if we've still not succeeded. + List matchingEvents = Collections.emptyList(); + while (matchingEvents.isEmpty()) { + final QuerySubmission submission = index.submitQuery(query, EventAuthorizer.GRANT_ALL, "unit test user"); + assertNotNull(submission); + + final QueryResult result = submission.getResult(); + assertNotNull(result); + result.awaitCompletion(100, TimeUnit.MILLISECONDS); + + assertTrue(result.isFinished()); + assertNull(result.getError()); + + matchingEvents = result.getMatchingEvents(); + assertNotNull(matchingEvents); + Thread.sleep(100L); // avoid crushing the CPU + } + + assertEquals(1, matchingEvents.size()); + assertEquals(event, matchingEvents.get(0)); + } + + @Test(timeout = 50000) + public void testQuerySpecificField() throws InterruptedException { + final RepositoryConfiguration repoConfig = createConfig(); + final IndexManager indexManager = new SimpleIndexManager(repoConfig); + + final LuceneEventIndex index = new LuceneEventIndex(repoConfig, indexManager, 2, EventReporter.NO_OP); + + // add 2 events, one of which we will query for. + final ProvenanceEventRecord event = createEvent(); + index.addEvent(event, new StorageSummary(event.getEventId(), "1.prov", "1", 1, 2L, 2L)); + index.addEvent(createEvent(), new StorageSummary(2L, "1.prov", "1", 1, 2L, 2L)); + + // Create a query that searches for the event with the FlowFile UUID equal to the first event's. + final Query query = new Query(UUID.randomUUID().toString()); + query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.FlowFileUUID, event.getFlowFileUuid())); + + final ArrayListEventStore eventStore = new ArrayListEventStore(); + eventStore.addEvent(event); + index.initialize(eventStore); + + // We don't know how long it will take for the event to be indexed, so keep querying until + // we get a result. The test will timeout after 5 seconds if we've still not succeeded. + List matchingEvents = Collections.emptyList(); + while (matchingEvents.isEmpty()) { + final QuerySubmission submission = index.submitQuery(query, EventAuthorizer.GRANT_ALL, "unit test user"); + assertNotNull(submission); + + final QueryResult result = submission.getResult(); + assertNotNull(result); + result.awaitCompletion(100, TimeUnit.MILLISECONDS); + + assertTrue(result.isFinished()); + assertNull(result.getError()); + + matchingEvents = result.getMatchingEvents(); + assertNotNull(matchingEvents); + Thread.sleep(100L); // avoid crushing the CPU + } + + assertEquals(1, matchingEvents.size()); + assertEquals(event, matchingEvents.get(0)); + } + + private RepositoryConfiguration createConfig() { + return createConfig(1); + } + + private RepositoryConfiguration createConfig(final int storageDirectoryCount) { + final RepositoryConfiguration config = new RepositoryConfiguration(); + final String unitTestName = testName.getMethodName(); + final File storageDir = new File("target/storage/" + unitTestName + "/" + UUID.randomUUID().toString()); + + for (int i = 0; i < storageDirectoryCount; i++) { + config.addStorageDirectory(String.valueOf(i + 1), new File(storageDir, String.valueOf(i))); + } + + config.setSearchableFields(Collections.singletonList(SearchableFields.FlowFileUUID)); + config.setSearchableAttributes(Collections.singletonList(SearchableFields.newSearchableAttribute("updated"))); + + for (final File file : config.getStorageDirectories().values()) { + assertTrue(file.exists() || file.mkdirs()); + } + + return config; + } + + private ProvenanceEventRecord createEvent() { + return createEvent(System.currentTimeMillis()); + } + + private ProvenanceEventRecord createEvent(final String uuid) { + return createEvent(System.currentTimeMillis(), uuid); + } + + private ProvenanceEventRecord createEvent(final long timestamp) { + return createEvent(timestamp, UUID.randomUUID().toString()); + } + + private ProvenanceEventRecord createEvent(final long timestamp, final String uuid) { + final Map previousAttributes = new HashMap<>(); + previousAttributes.put("uuid", uuid); + final Map updatedAttributes = new HashMap<>(); + updatedAttributes.put("updated", "true"); + + final ProvenanceEventRecord event = new StandardProvenanceEventRecord.Builder() + .setEventType(ProvenanceEventType.CONTENT_MODIFIED) + .setAttributes(previousAttributes, updatedAttributes) + .setComponentId("component-1") + .setComponentType("unit test") + .setEventId(idGenerator.getAndIncrement()) + .setEventTime(timestamp) + .setFlowFileEntryDate(timestamp) + .setFlowFileUUID(uuid) + .setLineageStartDate(timestamp) + .setCurrentContentClaim("container", "section", "unit-test-id", 0L, 1024L) + .build(); + + return event; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestCachingIndexManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestCachingIndexManager.java index 36f0b00704..a42b73a9f6 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestCachingIndexManager.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestCachingIndexManager.java @@ -29,14 +29,14 @@ import java.util.UUID; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field.Store; import org.apache.lucene.document.StringField; -import org.apache.lucene.index.IndexWriter; 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.IndexSearcher; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.EventIndexWriter; import org.apache.nifi.util.file.FileUtils; import org.junit.After; import org.junit.Before; @@ -67,47 +67,47 @@ public class TestCachingIndexManager { public void test() throws IOException { // Create and IndexWriter and add a document to the index, then close the writer. // This gives us something that we can query. - final IndexWriter writer = manager.borrowIndexWriter(indexDir); + final EventIndexWriter writer = manager.borrowIndexWriter(indexDir); final Document doc = new Document(); doc.add(new StringField("unit test", "true", Store.YES)); - writer.addDocument(doc); - manager.returnIndexWriter(indexDir, writer); + writer.index(doc, 1000); + manager.returnIndexWriter(writer); // Get an Index Searcher that we can use to query the index. - final IndexSearcher cachedSearcher = manager.borrowIndexSearcher(indexDir); + final EventIndexSearcher cachedSearcher = manager.borrowIndexSearcher(indexDir); // Ensure that we get the expected results. assertCount(cachedSearcher, 1); // While we already have an Index Searcher, get a writer for the same index. // This will cause the Index Searcher to be marked as poisoned. - final IndexWriter writer2 = manager.borrowIndexWriter(indexDir); + final EventIndexWriter writer2 = manager.borrowIndexWriter(indexDir); // Obtain a new Index Searcher with the writer open. This Index Searcher should *NOT* // be the same as the previous searcher because the new one will be a Near-Real-Time Index Searcher // while the other is not. - final IndexSearcher nrtSearcher = manager.borrowIndexSearcher(indexDir); + final EventIndexSearcher nrtSearcher = manager.borrowIndexSearcher(indexDir); assertNotSame(cachedSearcher, nrtSearcher); // Ensure that we get the expected query results. assertCount(nrtSearcher, 1); // Return the writer, so that there is no longer an active writer for the index. - manager.returnIndexWriter(indexDir, writer2); + manager.returnIndexWriter(writer2); // Ensure that we still get the same result. assertCount(cachedSearcher, 1); - manager.returnIndexSearcher(indexDir, cachedSearcher); + manager.returnIndexSearcher(cachedSearcher); // Ensure that our near-real-time index searcher still gets the same result. assertCount(nrtSearcher, 1); - manager.returnIndexSearcher(indexDir, nrtSearcher); + manager.returnIndexSearcher(nrtSearcher); } - private void assertCount(final IndexSearcher searcher, final int count) throws IOException { + private void assertCount(final EventIndexSearcher searcher, final int count) throws IOException { final BooleanQuery query = new BooleanQuery(); query.add(new BooleanClause(new TermQuery(new Term("unit test", "true")), Occur.MUST)); - final TopDocs topDocs = searcher.search(query, count * 10); + final TopDocs topDocs = searcher.getIndexSearcher().search(query, count * 10); assertNotNull(topDocs); assertEquals(1, topDocs.totalHits); } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestSimpleIndexManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestSimpleIndexManager.java index 834177fdd1..05369ca39b 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestSimpleIndexManager.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/lucene/TestSimpleIndexManager.java @@ -18,18 +18,21 @@ package org.apache.nifi.provenance.lucene; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field.Store; import org.apache.lucene.document.StringField; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.TopDocs; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.index.EventIndexSearcher; +import org.apache.nifi.provenance.index.EventIndexWriter; import org.apache.nifi.util.file.FileUtils; import org.junit.BeforeClass; import org.junit.Test; @@ -40,14 +43,13 @@ public class TestSimpleIndexManager { System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi.provenance", "DEBUG"); } - @Test public void testMultipleWritersSimultaneouslySameIndex() throws IOException { - final SimpleIndexManager mgr = new SimpleIndexManager(); + final SimpleIndexManager mgr = new SimpleIndexManager(new RepositoryConfiguration()); final File dir = new File("target/" + UUID.randomUUID().toString()); try { - final IndexWriter writer1 = mgr.borrowIndexWriter(dir); - final IndexWriter writer2 = mgr.borrowIndexWriter(dir); + final EventIndexWriter writer1 = mgr.borrowIndexWriter(dir); + final EventIndexWriter writer2 = mgr.borrowIndexWriter(dir); final Document doc1 = new Document(); doc1.add(new StringField("id", "1", Store.YES)); @@ -55,18 +57,94 @@ public class TestSimpleIndexManager { final Document doc2 = new Document(); doc2.add(new StringField("id", "2", Store.YES)); - writer1.addDocument(doc1); - writer2.addDocument(doc2); - mgr.returnIndexWriter(dir, writer2); - mgr.returnIndexWriter(dir, writer1); + writer1.index(doc1, 1000); + writer2.index(doc2, 1000); + mgr.returnIndexWriter(writer2); + mgr.returnIndexWriter(writer1); - final IndexSearcher searcher = mgr.borrowIndexSearcher(dir); - final TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), 2); + final EventIndexSearcher searcher = mgr.borrowIndexSearcher(dir); + final TopDocs topDocs = searcher.getIndexSearcher().search(new MatchAllDocsQuery(), 2); assertEquals(2, topDocs.totalHits); - mgr.returnIndexSearcher(dir, searcher); + mgr.returnIndexSearcher(searcher); } finally { FileUtils.deleteFile(dir, true); } } + @Test + public void testWriterCloseIfPreviouslyMarkedCloseable() throws IOException { + final AtomicInteger closeCount = new AtomicInteger(0); + + final SimpleIndexManager mgr = new SimpleIndexManager(new RepositoryConfiguration()) { + @Override + protected void close(IndexWriterCount count) throws IOException { + closeCount.incrementAndGet(); + } + }; + + final File dir = new File("target/" + UUID.randomUUID().toString()); + + final EventIndexWriter writer1 = mgr.borrowIndexWriter(dir); + final EventIndexWriter writer2 = mgr.borrowIndexWriter(dir); + assertTrue(writer1 == writer2); + + mgr.returnIndexWriter(writer1, true, true); + assertEquals(0, closeCount.get()); + + final EventIndexWriter[] writers = new EventIndexWriter[10]; + for (int i = 0; i < writers.length; i++) { + writers[i] = mgr.borrowIndexWriter(dir); + assertTrue(writers[i] == writer1); + } + + for (int i = 0; i < writers.length; i++) { + mgr.returnIndexWriter(writers[i], true, false); + assertEquals(0, closeCount.get()); + assertEquals(1, mgr.getWriterCount()); + } + + // this should close the index writer even though 'false' is passed in + // because the previous call marked the writer as closeable and this is + // the last reference to the writer. + mgr.returnIndexWriter(writer2, false, false); + assertEquals(1, closeCount.get()); + assertEquals(0, mgr.getWriterCount()); + } + + @Test + public void testWriterCloseIfOnlyUser() throws IOException { + final AtomicInteger closeCount = new AtomicInteger(0); + + final SimpleIndexManager mgr = new SimpleIndexManager(new RepositoryConfiguration()) { + @Override + protected void close(IndexWriterCount count) throws IOException { + closeCount.incrementAndGet(); + } + }; + + final File dir = new File("target/" + UUID.randomUUID().toString()); + + final EventIndexWriter writer = mgr.borrowIndexWriter(dir); + mgr.returnIndexWriter(writer, true, true); + assertEquals(1, closeCount.get()); + } + + @Test + public void testWriterLeftOpenIfNotCloseable() throws IOException { + final AtomicInteger closeCount = new AtomicInteger(0); + + final SimpleIndexManager mgr = new SimpleIndexManager(new RepositoryConfiguration()) { + @Override + protected void close(IndexWriterCount count) throws IOException { + closeCount.incrementAndGet(); + } + }; + + final File dir = new File("target/" + UUID.randomUUID().toString()); + + final EventIndexWriter writer = mgr.borrowIndexWriter(dir); + mgr.returnIndexWriter(writer, true, false); + assertEquals(0, closeCount.get()); + } + } diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/ArrayListEventStore.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/ArrayListEventStore.java new file mode 100644 index 0000000000..94a3699616 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/ArrayListEventStore.java @@ -0,0 +1,155 @@ +/* + * 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.store; + +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 java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.EventTransformer; +import org.apache.nifi.provenance.index.EventIndex; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ArrayListEventStore implements EventStore { + private static final Logger logger = LoggerFactory.getLogger(ArrayListEventStore.class); + + private final List events = new ArrayList<>(); + private final AtomicLong idGenerator = new AtomicLong(0L); + + @Override + public void close() throws IOException { + } + + @Override + public void initialize() throws IOException { + } + + public StorageResult addEvent(final ProvenanceEventRecord event) { + return addEvents(Collections.singleton(event)); + } + + @Override + public synchronized StorageResult addEvents(Iterable events) { + final Map storageLocations = new HashMap<>(); + + for (final ProvenanceEventRecord event : events) { + this.events.add(event); + + final StorageSummary storageSummary = new StorageSummary(idGenerator.getAndIncrement(), "location", "1", 1, 0L, 0L); + storageLocations.put(event, storageSummary); + } + + return new StorageResult() { + @Override + public Map getStorageLocations() { + return storageLocations; + } + + @Override + public boolean triggeredRollover() { + return false; + } + + @Override + public Integer getEventsRolledOver() { + return null; + } + }; + } + + @Override + public long getSize() throws IOException { + return 0; + } + + @Override + public long getMaxEventId() { + return idGenerator.get() - 1; + } + + @Override + public synchronized Optional getEvent(long id) throws IOException { + if (events.size() <= id) { + return Optional.empty(); + } + + return Optional.ofNullable(events.get((int) id)); + } + + @Override + public List getEvents(long firstRecordId, int maxResults) throws IOException { + return getEvents(firstRecordId, maxResults, EventAuthorizer.GRANT_ALL, EventTransformer.EMPTY_TRANSFORMER); + } + + @Override + public List getEvents(long firstRecordId, int maxResults, EventAuthorizer authorizer, EventTransformer transformer) throws IOException { + final List events = new ArrayList<>(); + for (int i = 0; i < maxResults; i++) { + final Optional eventOption = getEvent(firstRecordId + i); + if (!eventOption.isPresent()) { + break; + } + + events.add(eventOption.get()); + } + + return events; + } + + @Override + public List getEvents(final List eventIds, final EventAuthorizer authorizer, final EventTransformer transformer) { + final List events = new ArrayList<>(); + for (final Long eventId : eventIds) { + final Optional eventOption; + try { + eventOption = getEvent(eventId); + } catch (final Exception e) { + logger.warn("Failed to retrieve event with ID " + eventId, e); + continue; + } + + if (!eventOption.isPresent()) { + continue; + } + + if (authorizer.isAuthorized(eventOption.get())) { + events.add(eventOption.get()); + } else { + final Optional transformedOption = transformer.transform(eventOption.get()); + if (transformedOption.isPresent()) { + events.add(transformedOption.get()); + } + } + } + + return events; + } + + @Override + public void reindexLatestEvents(EventIndex eventIndex) { + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestEventFileManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestEventFileManager.java new file mode 100644 index 0000000000..42b8be2f0b --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestEventFileManager.java @@ -0,0 +1,240 @@ +/* + * 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.store; + +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.Test; + +public class TestEventFileManager { + + @Test(timeout = 5000) + public void testTwoWriteLocks() throws InterruptedException { + final EventFileManager fileManager = new EventFileManager(); + final File f1 = new File("1.prov"); + final File gz = new File("1.prov.gz"); + + final AtomicBoolean obtained = new AtomicBoolean(false); + + final Thread t1 = new Thread(new Runnable() { + @Override + public void run() { + fileManager.obtainWriteLock(f1); + + synchronized (obtained) { + obtained.set(true); + obtained.notify(); + } + + try { + Thread.sleep(500L); + } catch (InterruptedException e) { + } + fileManager.releaseWriteLock(f1); + } + }); + + t1.start(); + + final Thread t2 = new Thread(new Runnable() { + @Override + public void run() { + synchronized (obtained) { + while (!obtained.get()) { + try { + obtained.wait(); + } catch (InterruptedException e) { + } + } + } + + fileManager.obtainWriteLock(gz); + fileManager.releaseWriteLock(gz); + } + }); + + final long start = System.nanoTime(); + t2.start(); + t2.join(); + final long nanos = System.nanoTime() - start; + assertTrue(nanos > TimeUnit.MILLISECONDS.toNanos(300L)); + } + + + @Test(timeout = 5000) + public void testTwoReadLocks() throws InterruptedException { + final EventFileManager fileManager = new EventFileManager(); + final File f1 = new File("1.prov"); + final File gz = new File("1.prov.gz"); + + final AtomicBoolean obtained = new AtomicBoolean(false); + + final Thread t1 = new Thread(new Runnable() { + @Override + public void run() { + fileManager.obtainReadLock(f1); + + synchronized (obtained) { + obtained.set(true); + obtained.notify(); + } + + try { + Thread.sleep(100000L); + } catch (InterruptedException e) { + } + fileManager.releaseReadLock(f1); + } + }); + + t1.start(); + + final Thread t2 = new Thread(new Runnable() { + @Override + public void run() { + synchronized (obtained) { + while (!obtained.get()) { + try { + obtained.wait(); + } catch (InterruptedException e) { + } + } + } + + fileManager.obtainReadLock(gz); + fileManager.releaseReadLock(gz); + } + }); + + final long start = System.nanoTime(); + t2.start(); + t2.join(); + final long nanos = System.nanoTime() - start; + assertTrue(nanos < TimeUnit.MILLISECONDS.toNanos(500L)); + } + + + @Test(timeout = 5000) + public void testWriteThenRead() throws InterruptedException { + final EventFileManager fileManager = new EventFileManager(); + final File f1 = new File("1.prov"); + final File gz = new File("1.prov.gz"); + + final AtomicBoolean obtained = new AtomicBoolean(false); + + final Thread t1 = new Thread(new Runnable() { + @Override + public void run() { + fileManager.obtainWriteLock(f1); + + synchronized (obtained) { + obtained.set(true); + obtained.notify(); + } + + try { + Thread.sleep(500L); + } catch (InterruptedException e) { + } + fileManager.releaseWriteLock(f1); + } + }); + + t1.start(); + + final Thread t2 = new Thread(new Runnable() { + @Override + public void run() { + synchronized (obtained) { + while (!obtained.get()) { + try { + obtained.wait(); + } catch (InterruptedException e) { + } + } + } + + fileManager.obtainReadLock(gz); + fileManager.releaseReadLock(gz); + } + }); + + final long start = System.nanoTime(); + t2.start(); + t2.join(); + final long nanos = System.nanoTime() - start; + assertTrue(nanos > TimeUnit.MILLISECONDS.toNanos(300L)); + } + + + @Test(timeout = 5000) + public void testReadThenWrite() throws InterruptedException { + final EventFileManager fileManager = new EventFileManager(); + final File f1 = new File("1.prov"); + final File gz = new File("1.prov.gz"); + + final AtomicBoolean obtained = new AtomicBoolean(false); + + final Thread t1 = new Thread(new Runnable() { + @Override + public void run() { + fileManager.obtainReadLock(f1); + + synchronized (obtained) { + obtained.set(true); + obtained.notify(); + } + + try { + Thread.sleep(500L); + } catch (InterruptedException e) { + } + fileManager.releaseReadLock(f1); + } + }); + + t1.start(); + + final Thread t2 = new Thread(new Runnable() { + @Override + public void run() { + synchronized (obtained) { + while (!obtained.get()) { + try { + obtained.wait(); + } catch (InterruptedException e) { + } + } + } + + fileManager.obtainWriteLock(gz); + fileManager.releaseWriteLock(gz); + } + }); + + final long start = System.nanoTime(); + t2.start(); + t2.join(); + final long nanos = System.nanoTime() - start; + assertTrue(nanos > TimeUnit.MILLISECONDS.toNanos(300L)); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestPartitionedWriteAheadEventStore.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestPartitionedWriteAheadEventStore.java new file mode 100644 index 0000000000..7c5e43b564 --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestPartitionedWriteAheadEventStore.java @@ -0,0 +1,468 @@ +/* + * 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.store; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.EventIdFirstSchemaRecordWriter; +import org.apache.nifi.provenance.IdentifierLookup; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.StandardProvenanceEventRecord; +import org.apache.nifi.provenance.authorization.EventAuthorizer; +import org.apache.nifi.provenance.authorization.EventTransformer; +import org.apache.nifi.provenance.serialization.RecordReaders; +import org.apache.nifi.provenance.serialization.RecordWriters; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.provenance.toc.StandardTocWriter; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.toc.TocWriter; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +public class TestPartitionedWriteAheadEventStore { + private static final RecordWriterFactory writerFactory = (file, idGen, compress, createToc) -> RecordWriters.newSchemaRecordWriter(file, idGen, compress, createToc); + private static final RecordReaderFactory readerFactory = (file, logs, maxChars) -> RecordReaders.newRecordReader(file, logs, maxChars); + + private final AtomicLong idGenerator = new AtomicLong(0L); + + @Rule + public TestName testName = new TestName(); + + @Before + public void resetIds() { + idGenerator.set(0L); + } + + + @Test + @Ignore + public void testPerformanceOfAccessingEvents() throws Exception { + final RecordWriterFactory recordWriterFactory = (file, idGenerator, compressed, createToc) -> { + final TocWriter tocWriter = createToc ? new StandardTocWriter(TocUtil.getTocFile(file), false, false) : null; + return new EventIdFirstSchemaRecordWriter(file, idGenerator, tocWriter, compressed, 1024 * 1024, IdentifierLookup.EMPTY); + }; + + final RecordReaderFactory recordReaderFactory = (file, logs, maxChars) -> RecordReaders.newRecordReader(file, logs, maxChars); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(createConfig(), + recordWriterFactory, recordReaderFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + assertEquals(-1, store.getMaxEventId()); + for (int i = 0; i < 100_000; i++) { + final ProvenanceEventRecord event1 = createEvent(); + store.addEvents(Collections.singleton(event1)); + } + + final List eventIdList = Arrays.asList(4L, 80L, 1024L, 40_000L, 80_000L, 99_000L); + + while (true) { + for (int i = 0; i < 100; i++) { + time(() -> store.getEvents(eventIdList, EventAuthorizer.GRANT_ALL, EventTransformer.EMPTY_TRANSFORMER), "Fetch Events"); + } + + Thread.sleep(1000L); + } + } + + private void time(final Callable task, final String taskDescription) throws Exception { + final long start = System.nanoTime(); + task.call(); + final long nanos = System.nanoTime() - start; + final long millis = TimeUnit.NANOSECONDS.toMillis(nanos); + System.out.println("Took " + millis + " ms to " + taskDescription); + } + + @Test + public void testSingleWriteThenRead() throws IOException { + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(createConfig(), writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + assertEquals(-1, store.getMaxEventId()); + final ProvenanceEventRecord event1 = createEvent(); + final StorageResult result = store.addEvents(Collections.singleton(event1)); + + final StorageSummary summary = result.getStorageLocations().values().iterator().next(); + final long eventId = summary.getEventId(); + final ProvenanceEventRecord eventWithId = addId(event1, eventId); + + assertEquals(0, store.getMaxEventId()); + + final ProvenanceEventRecord read = store.getEvent(eventId).get(); + assertEquals(eventWithId, read); + } + + @Test + public void testMultipleWritesThenReads() throws IOException { + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(createConfig(), writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + assertEquals(-1, store.getMaxEventId()); + + final int numEvents = 20; + final List events = new ArrayList<>(numEvents); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + assertEquals(i, store.getMaxEventId()); + + events.add(event); + } + + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord read = store.getEvent(i).get(); + assertEquals(events.get(i), read); + } + } + + + @Test() + public void testMultipleWritesThenGetAllInSingleRead() throws IOException { + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(createConfig(), writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + assertEquals(-1, store.getMaxEventId()); + + final int numEvents = 20; + final List events = new ArrayList<>(numEvents); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + assertEquals(i, store.getMaxEventId()); + + events.add(event); + } + + List eventsRead = store.getEvents(0L, numEvents, null, EventTransformer.EMPTY_TRANSFORMER); + assertNotNull(eventsRead); + + assertEquals(numEvents, eventsRead.size()); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord read = eventsRead.get(i); + assertEquals(events.get(i), read); + } + + eventsRead = store.getEvents(-1000, 1000, null, EventTransformer.EMPTY_TRANSFORMER); + assertNotNull(eventsRead); + assertTrue(eventsRead.isEmpty()); + + eventsRead = store.getEvents(10, 0, null, EventTransformer.EMPTY_TRANSFORMER); + assertNotNull(eventsRead); + assertTrue(eventsRead.isEmpty()); + + eventsRead = store.getEvents(10, 1, null, EventTransformer.EMPTY_TRANSFORMER); + assertNotNull(eventsRead); + assertFalse(eventsRead.isEmpty()); + assertEquals(1, eventsRead.size()); + assertEquals(events.get(10), eventsRead.get(0)); + + eventsRead = store.getEvents(20, 1000, null, EventTransformer.EMPTY_TRANSFORMER); + assertNotNull(eventsRead); + assertTrue(eventsRead.isEmpty()); + } + + @Test + public void testGetSize() throws IOException { + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(createConfig(), writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + long storeSize = 0L; + final int numEvents = 20; + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + final long newSize = store.getSize(); + assertTrue(newSize > storeSize); + storeSize = newSize; + } + } + + @Test + public void testMaxEventIdRestored() throws IOException { + final RepositoryConfiguration config = createConfig(); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + final int numEvents = 20; + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + } + + assertEquals(19, store.getMaxEventId()); + store.close(); + + final PartitionedWriteAheadEventStore recoveredStore = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + recoveredStore.initialize(); + assertEquals(19, recoveredStore.getMaxEventId()); + } + + @Test + public void testGetEvent() throws IOException { + final RepositoryConfiguration config = createConfig(); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + final int numEvents = 20; + final List events = new ArrayList<>(numEvents); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + events.add(event); + } + + // Ensure that each event is retrieved successfully. + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = store.getEvent(i).get(); + assertEquals(events.get(i), event); + } + + assertFalse(store.getEvent(-1L).isPresent()); + assertFalse(store.getEvent(20L).isPresent()); + } + + @Test + public void testGetEventsWithMinIdAndCount() throws IOException { + final RepositoryConfiguration config = createConfig(); + config.setMaxEventFileCount(100); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + final int numEvents = 50_000; + final List events = new ArrayList<>(numEvents); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + if (i < 1000) { + events.add(event); + } + } + + assertTrue(store.getEvents(-1000L, 1000).isEmpty()); + assertEquals(events, store.getEvents(0, events.size())); + assertEquals(events, store.getEvents(-30, events.size())); + assertEquals(events.subList(10, events.size()), store.getEvents(10L, events.size() - 10)); + assertTrue(store.getEvents(numEvents, 100).isEmpty()); + } + + @Test + public void testGetEventsWithMinIdAndCountWithAuthorizer() throws IOException { + final RepositoryConfiguration config = createConfig(); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + final int numEvents = 20; + final List events = new ArrayList<>(numEvents); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + events.add(event); + } + + final EventAuthorizer allowEventNumberedEventIds = new EventAuthorizer() { + @Override + public boolean isAuthorized(final ProvenanceEventRecord event) { + return event.getEventId() % 2 == 0L; + } + + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + if (!isAuthorized(event)) { + throw new AccessDeniedException(); + } + } + }; + + final List storedEvents = store.getEvents(0, 20, allowEventNumberedEventIds, EventTransformer.EMPTY_TRANSFORMER); + assertEquals(numEvents / 2, storedEvents.size()); + for (int i = 0; i < storedEvents.size(); i++) { + assertEquals(events.get(i * 2), storedEvents.get(i)); + } + } + + + @Test + public void testGetEventsWithStartOffsetAndCountWithNothingAuthorized() throws IOException { + final RepositoryConfiguration config = createConfig(); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + final int numEvents = 20; + final List events = new ArrayList<>(numEvents); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + events.add(event); + } + + final EventAuthorizer allowEventNumberedEventIds = EventAuthorizer.DENY_ALL; + final List storedEvents = store.getEvents(0, 20, allowEventNumberedEventIds, EventTransformer.EMPTY_TRANSFORMER); + assertTrue(storedEvents.isEmpty()); + } + + @Test + public void testGetSpecificEventIds() throws IOException { + final RepositoryConfiguration config = createConfig(); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + final int numEvents = 20; + final List events = new ArrayList<>(numEvents); + for (int i = 0; i < numEvents; i++) { + final ProvenanceEventRecord event = createEvent(); + store.addEvents(Collections.singleton(event)); + events.add(event); + } + + final EventAuthorizer allowEvenNumberedEventIds = new EventAuthorizer() { + @Override + public boolean isAuthorized(final ProvenanceEventRecord event) { + return event.getEventId() % 2 == 0L; + } + + @Override + public void authorize(ProvenanceEventRecord event) throws AccessDeniedException { + if (!isAuthorized(event)) { + throw new AccessDeniedException(); + } + } + }; + + final List evenEventIds = new ArrayList<>(); + final List oddEventIds = new ArrayList<>(); + final List allEventIds = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + final Long id = Long.valueOf(i); + allEventIds.add(id); + + if (i % 2 == 0) { + evenEventIds.add(id); + } else { + oddEventIds.add(id); + } + } + + final List storedEvents = store.getEvents(evenEventIds, allowEvenNumberedEventIds, EventTransformer.EMPTY_TRANSFORMER); + assertEquals(numEvents / 2, storedEvents.size()); + for (int i = 0; i < storedEvents.size(); i++) { + assertEquals(events.get(i * 2), storedEvents.get(i)); + } + + assertTrue(store.getEvents(oddEventIds, allowEvenNumberedEventIds, EventTransformer.EMPTY_TRANSFORMER).isEmpty()); + + final List allStoredEvents = store.getEvents(allEventIds, EventAuthorizer.GRANT_ALL, EventTransformer.EMPTY_TRANSFORMER); + assertEquals(events, allStoredEvents); + } + + + @Test + public void testWriteAfterRecoveringRepo() throws IOException { + final RepositoryConfiguration config = createConfig(); + final PartitionedWriteAheadEventStore store = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + store.initialize(); + + for (int i = 0; i < 4; i++) { + store.addEvents(Collections.singleton(createEvent())); + } + + store.close(); + + final PartitionedWriteAheadEventStore recoveredStore = new PartitionedWriteAheadEventStore(config, writerFactory, readerFactory, EventReporter.NO_OP, new EventFileManager()); + recoveredStore.initialize(); + + List recoveredEvents = recoveredStore.getEvents(0, 10); + assertEquals(4, recoveredEvents.size()); + + // ensure that we can still write to the store + for (int i = 0; i < 4; i++) { + recoveredStore.addEvents(Collections.singleton(createEvent())); + } + + recoveredEvents = recoveredStore.getEvents(0, 10); + assertEquals(8, recoveredEvents.size()); + + for (int i = 0; i < 8; i++) { + assertEquals(i, recoveredEvents.get(i).getEventId()); + } + } + + + private RepositoryConfiguration createConfig() { + return createConfig(2); + } + + private RepositoryConfiguration createConfig(final int numStorageDirs) { + final RepositoryConfiguration config = new RepositoryConfiguration(); + final String unitTestName = testName.getMethodName(); + final File storageDir = new File("target/storage/" + unitTestName + "/" + UUID.randomUUID().toString()); + + for (int i = 1; i <= numStorageDirs; i++) { + config.addStorageDirectory(String.valueOf(i), new File(storageDir, String.valueOf(i))); + } + + return config; + } + + private ProvenanceEventRecord addId(final ProvenanceEventRecord event, final long eventId) { + return new StandardProvenanceEventRecord.Builder() + .fromEvent(event) + .setEventId(eventId) + .build(); + } + + + private ProvenanceEventRecord createEvent() { + final String uuid = UUID.randomUUID().toString(); + final Map previousAttributes = new HashMap<>(); + previousAttributes.put("uuid", uuid); + final Map updatedAttributes = new HashMap<>(); + updatedAttributes.put("updated", "true"); + + return new StandardProvenanceEventRecord.Builder() + .setEventType(ProvenanceEventType.CONTENT_MODIFIED) + .setAttributes(previousAttributes, updatedAttributes) + .setComponentId("component-1") + .setComponentType("unit test") + .setEventTime(System.currentTimeMillis()) + .setFlowFileEntryDate(System.currentTimeMillis()) + .setFlowFileUUID(uuid) + .setLineageStartDate(System.currentTimeMillis()) + .setCurrentContentClaim("container", "section", "unit-test-id", 0L, 1024L) + .build(); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestWriteAheadStorePartition.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestWriteAheadStorePartition.java new file mode 100644 index 0000000000..3879411e1a --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/TestWriteAheadStorePartition.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.provenance.store; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.EventIdFirstSchemaRecordWriter; +import org.apache.nifi.provenance.IdentifierLookup; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.RepositoryConfiguration; +import org.apache.nifi.provenance.TestUtil; +import org.apache.nifi.provenance.index.EventIndex; +import org.apache.nifi.provenance.serialization.RecordReaders; +import org.apache.nifi.provenance.serialization.StorageSummary; +import org.apache.nifi.provenance.toc.StandardTocWriter; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.toc.TocWriter; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class TestWriteAheadStorePartition { + + @Test + @SuppressWarnings("unchecked") + public void testReindex() throws IOException { + final RepositoryConfiguration repoConfig = createConfig(1, "testReindex"); + repoConfig.setMaxEventFileCount(5); + + final String partitionName = repoConfig.getStorageDirectories().keySet().iterator().next(); + final File storageDirectory = repoConfig.getStorageDirectories().values().iterator().next(); + + final RecordWriterFactory recordWriterFactory = (file, idGenerator, compressed, createToc) -> { + final TocWriter tocWriter = createToc ? new StandardTocWriter(TocUtil.getTocFile(file), false, false) : null; + return new EventIdFirstSchemaRecordWriter(file, idGenerator, tocWriter, compressed, 32 * 1024, IdentifierLookup.EMPTY); + }; + + final RecordReaderFactory recordReaderFactory = (file, logs, maxChars) -> RecordReaders.newRecordReader(file, logs, maxChars); + + final WriteAheadStorePartition partition = new WriteAheadStorePartition(storageDirectory, partitionName, repoConfig, recordWriterFactory, + recordReaderFactory, new LinkedBlockingQueue<>(), new AtomicLong(0L), EventReporter.NO_OP); + + for (int i = 0; i < 100; i++) { + partition.addEvents(Collections.singleton(TestUtil.createEvent())); + } + + final Map reindexedEvents = new HashMap<>(); + final EventIndex eventIndex = Mockito.mock(EventIndex.class); + Mockito.doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final Map events = invocation.getArgumentAt(0, Map.class); + reindexedEvents.putAll(events); + return null; + } + }).when(eventIndex).reindexEvents(Mockito.anyMap()); + + Mockito.doReturn(18L).when(eventIndex).getMinimumEventIdToReindex("1"); + partition.reindexLatestEvents(eventIndex); + + final List eventIdsReindexed = reindexedEvents.values().stream() + .map(StorageSummary::getEventId) + .sorted() + .collect(Collectors.toList()); + + assertEquals(82, eventIdsReindexed.size()); + for (int i = 0; i < eventIdsReindexed.size(); i++) { + assertEquals(18 + i, eventIdsReindexed.get(i).intValue()); + } + } + + private RepositoryConfiguration createConfig(final int numStorageDirs, final String testName) { + final RepositoryConfiguration config = new RepositoryConfiguration(); + final File storageDir = new File("target/storage/" + testName + "/" + UUID.randomUUID().toString()); + + for (int i = 1; i <= numStorageDirs; i++) { + config.addStorageDirectory(String.valueOf(1), new File(storageDir, String.valueOf(i))); + } + + config.setJournalCount(4); + return config; + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/iterator/TestSelectiveRecordReaderEventIterator.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/iterator/TestSelectiveRecordReaderEventIterator.java new file mode 100644 index 0000000000..0089f6191d --- /dev/null +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/store/iterator/TestSelectiveRecordReaderEventIterator.java @@ -0,0 +1,146 @@ +/* + * 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.store.iterator; + +import static org.junit.Assert.assertEquals; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.provenance.EventIdFirstSchemaRecordWriter; +import org.apache.nifi.provenance.IdentifierLookup; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.TestUtil; +import org.apache.nifi.provenance.serialization.RecordReaders; +import org.apache.nifi.provenance.serialization.RecordWriter; +import org.apache.nifi.provenance.store.RecordReaderFactory; +import org.apache.nifi.provenance.toc.StandardTocWriter; +import org.apache.nifi.provenance.toc.TocUtil; +import org.apache.nifi.provenance.toc.TocWriter; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + + +public class TestSelectiveRecordReaderEventIterator { + + + private RecordWriter createWriter(final File file, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { + return new EventIdFirstSchemaRecordWriter(file, new AtomicLong(0L), tocWriter, compressed, uncompressedBlockSize, IdentifierLookup.EMPTY); + } + + @Test + public void testFilterUnneededFiles() { + final File file1 = new File("1.prov"); + final File file1000 = new File("1000.prov"); + final File file2000 = new File("2000.prov"); + final File file3000 = new File("3000.prov"); + + // Filter out the first file. + final List files = new ArrayList<>(); + files.add(file1); + files.add(file1000); + files.add(file2000); + files.add(file3000); + + List eventIds = new ArrayList<>(); + eventIds.add(1048L); + eventIds.add(2048L); + eventIds.add(3048L); + + List filteredFiles = SelectiveRecordReaderEventIterator.filterUnneededFiles(files, eventIds); + assertEquals(Arrays.asList(new File[] {file1000, file2000, file3000}), filteredFiles); + + // Filter out file at end + eventIds.clear(); + eventIds.add(1L); + eventIds.add(1048L); + + filteredFiles = SelectiveRecordReaderEventIterator.filterUnneededFiles(files, eventIds); + assertEquals(Arrays.asList(new File[] {file1, file1000}), filteredFiles); + } + + @Test + @Ignore("For local testing only. Runs indefinitely") + public void testPerformanceOfRandomAccessReads() throws Exception { + final File dir = new File("target/storage/" + UUID.randomUUID().toString()); + final File journalFile = new File(dir, "/4.prov.gz"); + final File tocFile = TocUtil.getTocFile(journalFile); + + final int blockSize = 1024 * 32; + try (final RecordWriter writer = createWriter(journalFile, new StandardTocWriter(tocFile, true, false), true, blockSize)) { + writer.writeHeader(0L); + + for (int i = 0; i < 100_000; i++) { + writer.writeRecord(TestUtil.createEvent()); + } + } + + final Long[] eventIds = new Long[] { + 4L, 80L, 1024L, 1025L, 1026L, 1027L, 1028L, 1029L, 1030L, 40_000L, 80_000L, 99_000L + }; + + final RecordReaderFactory readerFactory = (file, logs, maxChars) -> RecordReaders.newRecordReader(file, logs, maxChars); + + final List files = new ArrayList<>(); + files.add(new File(dir, "0.prov")); + files.add(new File(dir, "0.prov")); + files.add(new File(dir, "1.prov")); + files.add(new File(dir, "2.prov")); + files.add(new File(dir, "3.prov")); + files.add(journalFile); + files.add(new File(dir, "100000000.prov")); + + boolean loopForever = true; + while (loopForever) { + final long start = System.nanoTime(); + for (int i = 0; i < 1000; i++) { + final SelectiveRecordReaderEventIterator iterator = new SelectiveRecordReaderEventIterator( + Collections.singletonList(journalFile), readerFactory, Arrays.asList(eventIds), 32 * 1024); + + for (final long id : eventIds) { + time(() -> { + return iterator.nextEvent().orElse(null); + }, id); + } + } + + final long ms = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + System.out.println(ms + " ms total"); + } + } + + private void time(final Callable task, final long id) throws Exception { + final long start = System.nanoTime(); + final ProvenanceEventRecord event = task.call(); + Assert.assertNotNull(event); + Assert.assertEquals(id, event.getEventId()); + // System.out.println(event); + final long nanos = System.nanoTime() - start; + final long millis = TimeUnit.NANOSECONDS.toMillis(nanos); + // System.out.println("Took " + millis + " ms to " + taskDescription); + } +} diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java index e467676b4a..f08fed4559 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java @@ -124,7 +124,8 @@ public class VolatileProvenanceRepository implements ProvenanceRepository { } @Override - public void initialize(final EventReporter eventReporter, final Authorizer authorizer, final ProvenanceAuthorizableFactory resourceFactory) throws IOException { + public void initialize(final EventReporter eventReporter, final Authorizer authorizer, final ProvenanceAuthorizableFactory resourceFactory, + final IdentifierLookup idLookup) throws IOException { if (initialized.getAndSet(true)) { return; } @@ -542,7 +543,7 @@ public class VolatileProvenanceRepository implements ProvenanceRepository { if (event == null) { final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); - submission.getResult().update(Collections.emptyList()); + submission.getResult().update(Collections. emptyList(), 0L); return submission; } @@ -573,7 +574,7 @@ public class VolatileProvenanceRepository implements ProvenanceRepository { if (event == null) { final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); - submission.getResult().update(Collections.emptyList()); + submission.getResult().update(Collections. emptyList(), 0L); return submission; } @@ -681,7 +682,7 @@ public class VolatileProvenanceRepository implements ProvenanceRepository { @Override public void run() { final List records = ringBuffer.getSelectedElements(filter); - submission.getResult().update(records); + submission.getResult().update(records, records.size()); } }