NIFI-3356: Initial implementation of writeahead provenance repository

- The idea behind NIFI-3356 was to improve the efficiency and throughput of the Provenance Repository, as it is often the bottleneck. While testing the newly designed repository,
  a handful of other, fairly minor, changes were made to improve efficiency as well, as these came to light when testing the new repository:

- Use a BufferedOutputStream within StandardProcessSession (via a ClaimCache abstraction) in order to avoid continually writing to FileOutputStream when writing many small FlowFiles
- Updated threading model of MinimalLockingWriteAheadLog - now performs serialization outside of lock and writes to a 'synchronized' OutputStream
- Change minimum scheduling period for components from 30 microseconds to 1 nanosecond. ScheduledExecutor is very inconsistent with timing of task scheduling. With the bored.yield.duration
  now present, this value doesn't need to be set to 30 microseconds. This was originally done to avoid processors that had no work from dominating the CPU. However, now that we will yield
  when processors have no work, this results in slowing down processors that are able to perform work.
- Allow nifi.properties to specify multiple directories for FlowFile Repository
- If backpressure is engaged while running a batch of sessions, then stop batch processing earlier. This helps FlowFiles to move through the system much more smoothly instead of the
  herky-jerky queuing that we previously saw at very high rates of FlowFiles.
- Added NiFi PID to log message when starting nifi. This was simply an update to the log message that provides helpful information.

NIFI-3356: Fixed bug in ContentClaimWriteCache that resulted in data corruption and fixed bug in RepositoryConfiguration that threw exception if cache warm duration was set to empty string

NIFI-3356: Fixed NPE

NIFI-3356: Added debug-level performance monitoring

NIFI-3356: Updates to unit tests that failed after rebasing against master

NIFI-3356: Incorporated PR review feedback

NIFI-3356: Fixed bug where we would delete index directories that are still in use; also added additional debug logging and a simple util class that can be used to textualize provenance event files - useful in debugging

This closes #1493
This commit is contained in:
Mark Payne 2016-12-09 10:52:33 -05:00 committed by Oleg Zhurakousky
parent 8d467f3d1f
commit 96ed405d70
138 changed files with 12187 additions and 1052 deletions

View File

@ -79,6 +79,16 @@ public interface ProvenanceEventRecord {
*/ */
Map<String, String> getAttributes(); Map<String, String> 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 <code>null</code> 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 * @return all FlowFile attributes that existed on the FlowFile before this
* event occurred * event occurred

View File

@ -1067,11 +1067,14 @@ public class RunNiFi {
Process process = builder.start(); Process process = builder.start();
handleLogging(process); handleLogging(process);
Long pid = getPid(process, cmdLogger); 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; nifiPid = pid;
final Properties pidProperties = new Properties(); final Properties pidProperties = new Properties();
pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid)); pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid));
savePidProperties(pidProperties, cmdLogger); savePidProperties(pidProperties, cmdLogger);
cmdLogger.info("Launched Apache NiFi with Process ID " + pid);
} }
shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor); shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
@ -1129,11 +1132,14 @@ public class RunNiFi {
handleLogging(process); handleLogging(process);
pid = getPid(process, defaultLogger); 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; nifiPid = pid;
final Properties pidProperties = new Properties(); final Properties pidProperties = new Properties();
pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid)); pidProperties.setProperty(PID_KEY, String.valueOf(nifiPid));
savePidProperties(pidProperties, defaultLogger); savePidProperties(pidProperties, defaultLogger);
cmdLogger.info("Launched Apache NiFi with Process ID " + pid);
} }
shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor); shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);

View File

@ -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<ProvenanceEventRecord> records, long totalHitCount);
void setError(String error);
long getTotalHitCount();
boolean isFinished();
}

View File

@ -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); public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES);
private static final Logger logger = LoggerFactory.getLogger(StandardLineageResult.class); private static final Logger logger = LoggerFactory.getLogger(StandardLineageResult.class);
@ -66,6 +66,7 @@ public class StandardLineageResult implements ComputeLineageResult {
private int numCompletedSteps = 0; private int numCompletedSteps = 0;
private volatile boolean canceled = false; private volatile boolean canceled = false;
private final Object completionMonitor = new Object();
public StandardLineageResult(final int numSteps, final Collection<String> flowFileUuids) { public StandardLineageResult(final int numSteps, final Collection<String> flowFileUuids) {
this.numSteps = numSteps; this.numSteps = numSteps;
@ -162,6 +163,7 @@ public class StandardLineageResult implements ComputeLineageResult {
} }
} }
@Override
public void setError(final String error) { public void setError(final String error) {
writeLock.lock(); writeLock.lock();
try { try {
@ -178,7 +180,10 @@ public class StandardLineageResult implements ComputeLineageResult {
} }
} }
public void update(final Collection<ProvenanceEventRecord> records) { @Override
public void update(final Collection<ProvenanceEventRecord> records, final long totalHitCount) {
boolean computationComplete = false;
writeLock.lock(); writeLock.lock();
try { try {
relevantRecords.addAll(records); relevantRecords.addAll(records);
@ -187,12 +192,22 @@ public class StandardLineageResult implements ComputeLineageResult {
updateExpiration(); updateExpiration();
if (numCompletedSteps >= numSteps && error == null) { if (numCompletedSteps >= numSteps && error == null) {
computationComplete = true;
computeLineage(); computeLineage();
computationNanos = System.nanoTime() - creationNanos; computationNanos = System.nanoTime() - creationNanos;
} }
} finally { } finally {
writeLock.unlock(); 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 * useful after all of the records have been successfully obtained
*/ */
private void computeLineage() { private void computeLineage() {
logger.debug("Computing lineage with the following events: {}", relevantRecords);
final long startNanos = System.nanoTime(); final long startNanos = System.nanoTime();
nodes.clear(); nodes.clear();
@ -324,4 +340,31 @@ public class StandardLineageResult implements ComputeLineageResult {
private void updateExpiration() { private void updateExpiration() {
expirationDate = new Date(System.currentTimeMillis() + TTL); 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();
}
}
} }

View File

@ -67,7 +67,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
private final Map<String, String> previousAttributes; private final Map<String, String> previousAttributes;
private final Map<String, String> updatedAttributes; private final Map<String, String> updatedAttributes;
private volatile long eventId; private volatile long eventId = -1L;
private StandardProvenanceEventRecord(final Builder builder) { private StandardProvenanceEventRecord(final Builder builder) {
this.eventTime = builder.eventTime; this.eventTime = builder.eventTime;
@ -369,14 +369,22 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
return false; return false;
} }
if (a == null && b != null) { if (a == null && b != null && !b.isEmpty()) {
return true; return true;
} }
if (a != null && b == null) { if (a == null && b.isEmpty()) {
return false;
}
if (a != null && !a.isEmpty() && b == null) {
return true; return true;
} }
if (a.isEmpty() && b == null) {
return false;
}
if (a.size() != b.size()) { if (a.size() != b.size()) {
return true; return true;
} }

View File

@ -22,7 +22,7 @@ import java.util.Comparator;
import java.util.Date; import java.util.Date;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.SortedSet;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock; 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.Query;
import org.apache.nifi.provenance.search.QueryResult; 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); public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES);
private final Query query; private final Query query;
@ -44,12 +47,12 @@ public class StandardQueryResult implements QueryResult {
private final Lock writeLock = rwLock.writeLock(); private final Lock writeLock = rwLock.writeLock();
// guarded by writeLock // guarded by writeLock
private final Set<ProvenanceEventRecord> matchingRecords = new TreeSet<>(new EventIdComparator()); private final SortedSet<ProvenanceEventRecord> matchingRecords = new TreeSet<>(new EventIdComparator());
private long totalHitCount;
private int numCompletedSteps = 0; private int numCompletedSteps = 0;
private Date expirationDate; private Date expirationDate;
private String error; private String error;
private long queryTime; private long queryTime;
private final Object completionMonitor = new Object();
private volatile boolean canceled = false; private volatile boolean canceled = false;
@ -65,22 +68,7 @@ public class StandardQueryResult implements QueryResult {
public List<ProvenanceEventRecord> getMatchingEvents() { public List<ProvenanceEventRecord> getMatchingEvents() {
readLock.lock(); readLock.lock();
try { try {
if (matchingRecords.size() <= query.getMaxResults()) {
return new ArrayList<>(matchingRecords); return new ArrayList<>(matchingRecords);
}
final List<ProvenanceEventRecord> copy = new ArrayList<>(query.getMaxResults());
int i = 0;
final Iterator<ProvenanceEventRecord> itr = matchingRecords.iterator();
while (itr.hasNext()) {
copy.add(itr.next());
if (++i >= query.getMaxResults()) {
break;
}
}
return copy;
} finally { } finally {
readLock.unlock(); readLock.unlock();
} }
@ -137,7 +125,7 @@ public class StandardQueryResult implements QueryResult {
public boolean isFinished() { public boolean isFinished() {
readLock.lock(); readLock.lock();
try { try {
return numCompletedSteps >= numSteps || canceled; return numCompletedSteps >= numSteps || canceled || matchingRecords.size() >= query.getMaxResults();
} finally { } finally {
readLock.unlock(); readLock.unlock();
} }
@ -147,6 +135,7 @@ public class StandardQueryResult implements QueryResult {
this.canceled = true; this.canceled = true;
} }
@Override
public void setError(final String error) { public void setError(final String error) {
writeLock.lock(); writeLock.lock();
try { try {
@ -163,22 +152,74 @@ public class StandardQueryResult implements QueryResult {
} }
} }
public void update(final Collection<ProvenanceEventRecord> matchingRecords, final long totalHits) { @Override
public void update(final Collection<ProvenanceEventRecord> newEvents, final long totalHits) {
boolean queryComplete = false;
writeLock.lock(); writeLock.lock();
try { try {
this.matchingRecords.addAll(matchingRecords); if (isFinished()) {
this.totalHitCount += totalHits; 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<ProvenanceEventRecord> itr = matchingRecords.iterator();
for (int i = 0; i < query.getMaxResults(); i++) {
itr.next();
}
while (itr.hasNext()) {
itr.next();
itr.remove();
}
}
numCompletedSteps++; numCompletedSteps++;
updateExpiration(); updateExpiration();
if (numCompletedSteps >= numSteps) { if (numCompletedSteps >= numSteps || this.matchingRecords.size() >= query.getMaxResults()) {
final long searchNanos = System.nanoTime() - creationNanos; final long searchNanos = System.nanoTime() - creationNanos;
queryTime = TimeUnit.MILLISECONDS.convert(searchNanos, TimeUnit.NANOSECONDS); 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 { } finally {
writeLock.unlock(); 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();
} }
/** /**

View File

@ -18,7 +18,6 @@ package org.apache.nifi.provenance.lineage;
import java.util.List; import java.util.List;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType; import org.apache.nifi.provenance.ProvenanceEventType;
@ -58,7 +57,7 @@ public class EventNode implements ProvenanceEventLineageNode {
@Override @Override
public String getFlowFileUuid() { public String getFlowFileUuid() {
return record.getAttributes().get(CoreAttributes.UUID.key()); return record.getFlowFileUuid();
} }
@Override @Override

View File

@ -29,6 +29,11 @@ public class FieldMapRecord implements Record {
this.values = convertFieldToName(values); this.values = convertFieldToName(values);
} }
public FieldMapRecord(final RecordSchema schema, final Map<String, Object> values) {
this.schema = schema;
this.values = new HashMap<>(values);
}
private static Map<String, Object> convertFieldToName(final Map<RecordField, Object> map) { private static Map<String, Object> convertFieldToName(final Map<RecordField, Object> map) {
final Map<String, Object> nameMap = new HashMap<>(map.size()); final Map<String, Object> nameMap = new HashMap<>(map.size());
for (final Map.Entry<RecordField, Object> entry : map.entrySet()) { for (final Map.Entry<RecordField, Object> entry : map.entrySet()) {

View File

@ -120,8 +120,8 @@ public class RecordSchema {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private static RecordField readField(final DataInputStream dis) throws IOException { private static RecordField readField(final DataInputStream dis) throws IOException {
final Map<String, Object> schemaFieldMap = new HashMap<>();
final int numElementsToRead = dis.readInt(); final int numElementsToRead = dis.readInt();
final Map<String, Object> schemaFieldMap = new HashMap<>(numElementsToRead);
for (int i = 0; i < numElementsToRead; i++) { for (int i = 0; i < numElementsToRead; i++) {
final String fieldName = dis.readUTF(); final String fieldName = dis.readUTF();
final String typeName = dis.readUTF(); final String typeName = dis.readUTF();

View File

@ -109,7 +109,15 @@ public class SchemaRecordReader {
} }
} }
try {
return readFieldValue(in, field.getFieldType(), field.getFieldName(), field.getSubFields()); 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);
}
} }

View File

@ -44,8 +44,12 @@ public class SchemaRecordWriter {
} }
private void writeRecordFields(final Record record, final OutputStream out) throws IOException { 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); 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); final Object value = record.getFieldValue(field);
try { try {

View File

@ -18,6 +18,9 @@ package org.wali;
import static java.util.Objects.requireNonNull; 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.DataInputStream;
import java.io.DataOutputStream; import java.io.DataOutputStream;
import java.io.EOFException; import java.io.EOFException;
@ -55,12 +58,9 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Pattern; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -670,11 +670,10 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
private final Path editDirectory; private final Path editDirectory;
private final int writeAheadLogVersion; private final int writeAheadLogVersion;
private final Lock lock = new ReentrantLock();
private DataOutputStream dataOut = null; private DataOutputStream dataOut = null;
private FileOutputStream fileOut = null; private FileOutputStream fileOut = null;
private boolean blackListed = false; private volatile boolean blackListed = false;
private boolean closed = false; private volatile boolean closed = false;
private DataInputStream recoveryIn; private DataInputStream recoveryIn;
private int recoveryVersion; private int recoveryVersion;
private String currentJournalFilename = ""; private String currentJournalFilename = "";
@ -707,26 +706,15 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
} }
public boolean tryClaim() { public boolean tryClaim() {
final boolean obtainedLock = lock.tryLock(); return !blackListed;
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;
} }
public void releaseClaim() { public void releaseClaim() {
lock.unlock();
} }
public void close() { public void close() {
this.closed = true;
// Note that here we are closing fileOut and NOT dataOut. // Note that here we are closing fileOut and NOT dataOut.
// This is very much intentional, not an oversight. This is done because of // This is very much intentional, not an oversight. This is done because of
// the way that the OutputStreams are structured. dataOut wraps a BufferedOutputStream, // the way that the OutputStreams are structured. dataOut wraps a BufferedOutputStream,
@ -761,18 +749,12 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
} }
} }
this.closed = true;
this.dataOut = null; this.dataOut = null;
this.fileOut = null; this.fileOut = null;
} }
public void blackList() { public void blackList() {
lock.lock();
try {
blackListed = true; blackListed = true;
} finally {
lock.unlock();
}
logger.debug("Blacklisted {}", this); logger.debug("Blacklisted {}", this);
} }
@ -783,8 +765,6 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
* @throws IOException if failure to rollover * @throws IOException if failure to rollover
*/ */
public OutputStream rollover() throws IOException { public OutputStream rollover() throws IOException {
lock.lock();
try {
// Note that here we are closing fileOut and NOT dataOut. See the note in the close() // Note that here we are closing fileOut and NOT dataOut. See the note in the close()
// method to understand the logic behind this. // method to understand the logic behind this.
final OutputStream oldOutputStream = fileOut; final OutputStream oldOutputStream = fileOut;
@ -829,9 +809,6 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
blackListed = false; blackListed = false;
return oldOutputStream; return oldOutputStream;
} finally {
lock.unlock();
}
} }
private long getJournalIndex(final File file) { private long getJournalIndex(final File file) {
@ -939,15 +916,11 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
return true; return true;
} }
public void update(final Collection<S> records, final long transactionId, final Map<Object, S> recordMap, final boolean forceSync) public void update(final Collection<S> records, final long transactionId, final Map<Object, S> recordMap, final boolean forceSync) throws IOException {
throws IOException { try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(256);
if (this.closed) { final DataOutputStream out = new DataOutputStream(baos)) {
throw new IllegalStateException("Partition is closed");
}
final DataOutputStream out = dataOut;
out.writeLong(transactionId); out.writeLong(transactionId);
final int numEditsToSerialize = records.size(); final int numEditsToSerialize = records.size();
int editsSerialized = 0; int editsSerialized = 0;
for (final S record : records) { for (final S record : records) {
@ -964,10 +937,20 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
out.flush(); out.flush();
if (this.closed) {
throw new IllegalStateException("Partition is closed");
}
baos.writeTo(dataOut);
dataOut.flush();
if (forceSync) { if (forceSync) {
synchronized (fileOut) {
fileOut.getFD().sync(); fileOut.getFD().sync();
} }
} }
}
}
private DataInputStream createDataInputStream(final Path path) throws IOException { private DataInputStream createDataInputStream(final Path path) throws IOException {
return new DataInputStream(new BufferedInputStream(Files.newInputStream(path))); return new DataInputStream(new BufferedInputStream(Files.newInputStream(path)));

View File

@ -53,6 +53,64 @@ public class TestMinimalLockingWriteAheadLog {
private static final Logger logger = LoggerFactory.getLogger(TestMinimalLockingWriteAheadLog.class); 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<DummyRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
final Collection<DummyRecord> 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 @Test
public void testRepoDoesntContinuallyGrowOnOutOfMemoryError() throws IOException, InterruptedException { public void testRepoDoesntContinuallyGrowOnOutOfMemoryError() throws IOException, InterruptedException {
final int numPartitions = 8; final int numPartitions = 8;
@ -557,21 +615,10 @@ public class TestMinimalLockingWriteAheadLog {
assertEquals(2, transactionIndicator); assertEquals(2, transactionIndicator);
} }
long transactionId = in.readLong(); // In previous implementations, we would still have a partial record written out.
assertEquals(2L, transactionId); // 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
long thirdSize = in.readLong(); // now be out of data
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!
final int nextByte = in.read(); final int nextByte = in.read();
assertEquals(-1, nextByte); assertEquals(-1, nextByte);
} }

View File

@ -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: + 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.content.repository.directory.content1=/repos/content1 +
nifi.provenance.repository.directory.content2=/repos/provenance2 + nifi.content.repository.directory.content2=/repos/content2 +
+ +
Providing three total locations, including _nifi.content.repository.directory.default_. 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 |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* |*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 === 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. |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 === Component Status Repository
The Component Status Repository contains the information for the Component Status History tool in the User Interface. These The Component Status Repository contains the information for the Component Status History tool in the User Interface. These

View File

@ -25,7 +25,7 @@ import org.apache.nifi.processor.exception.ProcessException;
public interface Triggerable { public interface Triggerable {
public static final long MINIMUM_SCHEDULING_NANOS = 30000L; public static final long MINIMUM_SCHEDULING_NANOS = 1L;
/** /**
* <p> * <p>

View File

@ -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<String> getComponentIdentifiers();
/**
* @return a list of component types that may generate Provenance Events
*/
List<String> getComponentTypes();
/**
*
* @return the identifiers of FlowFile Queues that are in the flow
*/
List<String> getQueueIdentifiers();
default Map<String, Integer> invertQueueIdentifiers() {
return invertList(getQueueIdentifiers());
}
default Map<String, Integer> invertComponentTypes() {
return invertList(getComponentTypes());
}
default Map<String, Integer> invertComponentIdentifiers() {
return invertList(getComponentIdentifiers());
}
default Map<String, Integer> invertList(final List<String> values) {
final Map<String, Integer> 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<String> getComponentIdentifiers() {
return Collections.emptyList();
}
@Override
public List<String> getComponentTypes() {
return Collections.emptyList();
}
@Override
public List<String> getQueueIdentifiers() {
return Collections.emptyList();
}
@Override
public Map<String, Integer> invertList(List<String> values) {
return Collections.emptyMap();
}
};
}

View File

@ -37,9 +37,10 @@ public interface ProvenanceRepository extends ProvenanceEventRepository {
* @param eventReporter to report to * @param eventReporter to report to
* @param authorizer the authorizer to use for authorizing individual events * @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 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 * @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; ProvenanceEventRecord getEvent(long id, NiFiUser user) throws IOException;

View File

@ -18,6 +18,7 @@ package org.apache.nifi.provenance.lineage;
import java.util.Date; import java.util.Date;
import java.util.List; 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 * @return Indicates whether or not the lineage has finished running
*/ */
boolean isFinished(); boolean isFinished();
boolean awaitCompletion(long time, TimeUnit unit) throws InterruptedException;
} }

View File

@ -93,4 +93,8 @@ public class Query {
public String toString() { public String toString() {
return "Query[ " + searchTerms + " ]"; return "Query[ " + searchTerms + " ]";
} }
public boolean isEmpty() {
return searchTerms.isEmpty() && maxFileSize == null && minFileSize == null && startDate == null && endDate == null;
}
} }

View File

@ -18,6 +18,7 @@ package org.apache.nifi.provenance.search;
import java.util.Date; import java.util.Date;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
@ -60,4 +61,6 @@ public interface QueryResult {
* @return Indicates whether or not the query has finished running * @return Indicates whether or not the query has finished running
*/ */
boolean isFinished(); boolean isFinished();
boolean awaitCompletion(long time, TimeUnit unit) throws InterruptedException;
} }

View File

@ -56,7 +56,7 @@ public class MockProvenanceRepository implements ProvenanceRepository {
} }
@Override @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 {
} }

View File

@ -63,7 +63,29 @@ public class MockFlowFile implements FlowFileRecord {
} }
public MockFlowFile(final long id, final FlowFile toCopy) { public MockFlowFile(final long id, final FlowFile toCopy) {
this(id); this.creationTime = System.nanoTime();
this.id = id;
entryDate = System.currentTimeMillis();
final Map<String, String> 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()); attributes.putAll(toCopy.getAttributes());
final byte[] dataToCopy = ((MockFlowFile) toCopy).data; final byte[] dataToCopy = ((MockFlowFile) toCopy).data;
this.data = new byte[dataToCopy.length]; this.data = new byte[dataToCopy.length];

View File

@ -111,7 +111,7 @@ public abstract class AbstractPort implements Port {
yieldPeriod = new AtomicReference<>("1 sec"); yieldPeriod = new AtomicReference<>("1 sec");
yieldExpiration = new AtomicLong(0L); yieldExpiration = new AtomicLong(0L);
schedulingPeriod = new AtomicReference<>("0 millis"); schedulingPeriod = new AtomicReference<>("0 millis");
schedulingNanos = new AtomicLong(30000); schedulingNanos = new AtomicLong(MINIMUM_SCHEDULING_NANOS);
scheduledState = new AtomicReference<>(ScheduledState.STOPPED); scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
} }

View File

@ -100,7 +100,7 @@ public class StandardFunnel implements Funnel {
yieldPeriod = new AtomicReference<>("250 millis"); yieldPeriod = new AtomicReference<>("250 millis");
yieldExpiration = new AtomicLong(0L); yieldExpiration = new AtomicLong(0L);
schedulingPeriod = new AtomicReference<>("0 millis"); schedulingPeriod = new AtomicReference<>("0 millis");
schedulingNanos = new AtomicLong(30000); schedulingNanos = new AtomicLong(MINIMUM_SCHEDULING_NANOS);
name = new AtomicReference<>("Funnel"); name = new AtomicReference<>("Funnel");
} }

View File

@ -152,6 +152,7 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.StandardProcessorInitializationContext; import org.apache.nifi.processor.StandardProcessorInitializationContext;
import org.apache.nifi.processor.StandardValidationContextFactory; import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.provenance.IdentifierLookup;
import org.apache.nifi.provenance.ProvenanceAuthorizableFactory; import org.apache.nifi.provenance.ProvenanceAuthorizableFactory;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType; 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.atomic.AtomicReference;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
import static java.util.Objects.requireNonNull; 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 // default repository implementations
public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository"; 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 { try {
this.provenanceRepository = createProvenanceRepository(nifiProperties); this.provenanceRepository = createProvenanceRepository(nifiProperties);
this.provenanceRepository.initialize(createEventReporter(bulletinRepository), authorizer, this); this.provenanceRepository.initialize(createEventReporter(bulletinRepository), authorizer, this, this);
} catch (final Exception e) { } catch (final Exception e) {
throw new RuntimeException("Unable to create Provenance Repository", e); throw new RuntimeException("Unable to create Provenance Repository", e);
} }
@ -3886,6 +3889,39 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
return replayEvent; return replayEvent;
} }
@Override
public List<String> getComponentIdentifiers() {
final List<String> 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<String> getComponentTypes() {
final Set<Class> procClasses = ExtensionManager.getExtensions(Processor.class);
final List<String> 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<String> getQueueIdentifiers() {
return getAllQueues().stream()
.map(q -> q.getIdentifier())
.collect(Collectors.toList());
}
public boolean isConnected() { public boolean isConnected() {
rwLock.readLock().lock(); rwLock.readLock().lock();
try { try {

View File

@ -911,6 +911,7 @@ public class FileSystemRepository implements ContentRepository {
} }
bytesWritten += len; bytesWritten += len;
scc.setLength(bytesWritten + initialLength); scc.setLength(bytesWritten + initialLength);
} }

View File

@ -16,6 +16,32 @@
*/ */
package org.apache.nifi.controller.repository; 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.commons.io.IOUtils;
import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection; 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.FlowFileQueue;
import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.repository.claim.ContentClaim; 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.claim.ResourceClaim;
import org.apache.nifi.controller.repository.io.DisableOnCloseInputStream; import org.apache.nifi.controller.repository.io.DisableOnCloseInputStream;
import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream; 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.Logger;
import org.slf4j.LoggerFactory; 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;
/** /**
* <p> * <p>
* Provides a ProcessSession that ensures all accesses, changes and transfers * Provides a ProcessSession that ensures all accesses, changes and transfers
@ -143,6 +144,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
private final Map<FlowFile, ProvenanceEventBuilder> forkEventBuilders = new HashMap<>(); private final Map<FlowFile, ProvenanceEventBuilder> forkEventBuilders = new HashMap<>();
private Checkpoint checkpoint = new Checkpoint(); private Checkpoint checkpoint = new Checkpoint();
private final ContentClaimWriteCache claimCache;
public StandardProcessSession(final ProcessContext context) { public StandardProcessSession(final ProcessContext context) {
this.context = context; this.context = context;
@ -180,7 +182,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
context.getProvenanceRepository(), this); context.getProvenanceRepository(), this);
this.sessionId = idGenerator.getAndIncrement(); this.sessionId = idGenerator.getAndIncrement();
this.connectableDescription = description; this.connectableDescription = description;
this.claimCache = new ContentClaimWriteCache(context.getContentRepository());
LOG.trace("Session {} created for {}", this, connectableDescription); LOG.trace("Session {} created for {}", this, connectableDescription);
processingStartTime = System.nanoTime(); processingStartTime = System.nanoTime();
} }
@ -312,6 +314,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final long commitStartNanos = System.nanoTime(); final long commitStartNanos = System.nanoTime();
resetReadClaim(); resetReadClaim();
try {
claimCache.flush();
} finally {
claimCache.reset();
}
final long updateProvenanceStart = System.nanoTime(); final long updateProvenanceStart = System.nanoTime();
updateProvenanceRepo(checkpoint); updateProvenanceRepo(checkpoint);
@ -375,7 +382,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
updateEventRepository(checkpoint); updateEventRepository(checkpoint);
final long updateEventRepositoryFinishNanos = System.nanoTime(); final long updateEventRepositoryFinishNanos = System.nanoTime();
final long updateEventRepositoryNanos = updateEventRepositoryFinishNanos - claimRemovalFinishNanos; final long updateEventRepositoryNanos = updateEventRepositoryFinishNanos - flowFileRepoUpdateFinishNanos;
// transfer the flowfiles to the connections' queues. // transfer the flowfiles to the connections' queues.
final Map<FlowFileQueue, Collection<FlowFileRecord>> recordMap = new HashMap<>(); final Map<FlowFileQueue, Collection<FlowFileRecord>> recordMap = new HashMap<>();
@ -454,7 +461,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
} catch (final Exception e1) { } catch (final Exception e1) {
e.addSuppressed(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<StandardRepositoryRecord> recordsToHandle = new HashSet<>(); final Set<StandardRepositoryRecord> recordsToHandle = new HashSet<>();
recordsToHandle.addAll(records.values()); recordsToHandle.addAll(records.values());
if (rollbackCheckpoint) { if (rollbackCheckpoint) {
@ -2033,6 +2051,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
} }
} }
claimCache.flush(claim);
final InputStream rawInStream = context.getContentRepository().read(claim); final InputStream rawInStream = context.getContentRepository().read(claim);
if (currentReadClaimStream != null) { if (currentReadClaimStream != null) {
@ -2047,6 +2066,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
// reuse the same InputStream for the next FlowFile // reuse the same InputStream for the next FlowFile
return new DisableOnCloseInputStream(currentReadClaimStream); return new DisableOnCloseInputStream(currentReadClaimStream);
} else { } else {
claimCache.flush(claim);
final InputStream rawInStream = context.getContentRepository().read(claim); final InputStream rawInStream = context.getContentRepository().read(claim);
try { try {
StreamUtils.skip(rawInStream, offset); StreamUtils.skip(rawInStream, offset);
@ -2077,6 +2097,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
try { try {
ensureNotAppending(record.getCurrentClaim()); ensureNotAppending(record.getCurrentClaim());
claimCache.flush(record.getCurrentClaim());
} catch (final IOException e) { } catch (final IOException e) {
throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e); throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e);
} }
@ -2241,6 +2262,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
try { try {
ensureNotAppending(record.getCurrentClaim()); ensureNotAppending(record.getCurrentClaim());
claimCache.flush(record.getCurrentClaim());
} catch (final IOException e) { } catch (final IOException e) {
throw new FlowFileAccessException("Unable to read from source " + source + " due to " + e.toString(), 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; long writtenToFlowFile = 0L;
ContentClaim newClaim = null; ContentClaim newClaim = null;
try { try {
newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); newClaim = claimCache.getContentClaim();
claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source); claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source);
ensureNotAppending(newClaim); ensureNotAppending(newClaim);
try (final OutputStream stream = context.getContentRepository().write(newClaim); try (final OutputStream stream = claimCache.write(newClaim);
final OutputStream disableOnClose = new DisableOnCloseOutputStream(stream); final OutputStream disableOnClose = new DisableOnCloseOutputStream(stream);
final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnClose)) { final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnClose)) {
try { try {
@ -2373,7 +2395,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() final FlowFileRecord newFile = new StandardFlowFileRecord.Builder()
.fromFlowFile(record.getCurrent()) .fromFlowFile(record.getCurrent())
.contentClaim(newClaim) .contentClaim(newClaim)
.contentClaimOffset(0) .contentClaimOffset(Math.max(0, newClaim.getLength() - writtenToFlowFile))
.size(writtenToFlowFile) .size(writtenToFlowFile)
.build(); .build();
@ -2396,6 +2418,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
ContentClaim newClaim = null; ContentClaim newClaim = null;
try { try {
if (outStream == null) { if (outStream == null) {
claimCache.flush(oldClaim);
try (final InputStream oldClaimIn = context.getContentRepository().read(oldClaim)) { try (final InputStream oldClaimIn = context.getContentRepository().read(oldClaim)) {
newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant());
claimLog.debug("Creating ContentClaim {} for 'append' for {}", newClaim, source); claimLog.debug("Creating ContentClaim {} for 'append' for {}", newClaim, source);
@ -2568,16 +2592,20 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
long writtenToFlowFile = 0L; long writtenToFlowFile = 0L;
ContentClaim newClaim = null; ContentClaim newClaim = null;
try { try {
newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); newClaim = claimCache.getContentClaim();
claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source); claimLog.debug("Creating ContentClaim {} for 'write' for {}", newClaim, source);
ensureNotAppending(newClaim); ensureNotAppending(newClaim);
if (currClaim != null) {
claimCache.flush(currClaim.getResourceClaim());
}
try (final InputStream is = getInputStream(source, currClaim, record.getCurrentClaimOffset(), true); try (final InputStream is = getInputStream(source, currClaim, record.getCurrentClaimOffset(), true);
final InputStream limitedIn = new LimitedInputStream(is, source.getSize()); final InputStream limitedIn = new LimitedInputStream(is, source.getSize());
final InputStream disableOnCloseIn = new DisableOnCloseInputStream(limitedIn); final InputStream disableOnCloseIn = new DisableOnCloseInputStream(limitedIn);
final ByteCountingInputStream countingIn = new ByteCountingInputStream(disableOnCloseIn, bytesRead); 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 OutputStream disableOnCloseOut = new DisableOnCloseOutputStream(os);
final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnCloseOut)) { final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnCloseOut)) {
@ -2626,7 +2654,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() final FlowFileRecord newFile = new StandardFlowFileRecord.Builder()
.fromFlowFile(record.getCurrent()) .fromFlowFile(record.getCurrent())
.contentClaim(newClaim) .contentClaim(newClaim)
.contentClaimOffset(0L) .contentClaimOffset(Math.max(0L, newClaim.getLength() - writtenToFlowFile))
.size(writtenToFlowFile) .size(writtenToFlowFile)
.build(); .build();
@ -2668,8 +2696,11 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
removeTemporaryClaim(record); removeTemporaryClaim(record);
final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()) final FlowFileRecord newFile = new StandardFlowFileRecord.Builder()
.contentClaim(newClaim).contentClaimOffset(claimOffset).size(newSize) .fromFlowFile(record.getCurrent())
.contentClaim(newClaim)
.contentClaimOffset(claimOffset)
.size(newSize)
.addAttribute(CoreAttributes.FILENAME.key(), source.toFile().getName()) .addAttribute(CoreAttributes.FILENAME.key(), source.toFile().getName())
.build(); .build();
record.setWorking(newFile, CoreAttributes.FILENAME.key(), source.toFile().getName()); record.setWorking(newFile, CoreAttributes.FILENAME.key(), source.toFile().getName());
@ -2708,7 +2739,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
} }
removeTemporaryClaim(record); 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); record.setWorking(newFile);
return newFile; return newFile;
} }
@ -2720,6 +2756,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
try { try {
ensureNotAppending(record.getCurrentClaim()); ensureNotAppending(record.getCurrentClaim());
claimCache.flush(record.getCurrentClaim());
final long copyCount = context.getContentRepository().exportTo(record.getCurrentClaim(), destination, append, record.getCurrentClaimOffset(), source.getSize()); final long copyCount = context.getContentRepository().exportTo(record.getCurrentClaim(), destination, append, record.getCurrentClaimOffset(), source.getSize());
bytesRead += copyCount; bytesRead += copyCount;
bytesWritten += copyCount; bytesWritten += copyCount;
@ -2741,6 +2778,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
try { try {
ensureNotAppending(record.getCurrentClaim()); ensureNotAppending(record.getCurrentClaim());
claimCache.flush(record.getCurrentClaim());
} catch (final IOException e) { } catch (final IOException e) {
throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e); throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e);
} }

View File

@ -19,6 +19,7 @@ package org.apache.nifi.controller.repository;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -27,6 +28,8 @@ import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
@ -74,6 +77,7 @@ import org.wali.WriteAheadRepository;
* </p> * </p>
*/ */
public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncListener { 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 AtomicLong flowFileSequenceGenerator = new AtomicLong(0L);
private final boolean alwaysSync; private final boolean alwaysSync;
@ -82,7 +86,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
private volatile ScheduledFuture<?> checkpointFuture; private volatile ScheduledFuture<?> checkpointFuture;
private final long checkpointDelayMillis; private final long checkpointDelayMillis;
private final Path flowFileRepositoryPath; private final SortedSet<Path> flowFileRepositoryPaths = new TreeSet<>();
private final int numPartitions; private final int numPartitions;
private final ScheduledExecutorService checkpointExecutor; private final ScheduledExecutorService checkpointExecutor;
@ -120,7 +124,6 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
public WriteAheadFlowFileRepository() { public WriteAheadFlowFileRepository() {
alwaysSync = false; alwaysSync = false;
checkpointDelayMillis = 0l; checkpointDelayMillis = 0l;
flowFileRepositoryPath = null;
numPartitions = 0; numPartitions = 0;
checkpointExecutor = null; checkpointExecutor = null;
} }
@ -129,7 +132,13 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
alwaysSync = Boolean.parseBoolean(nifiProperties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_ALWAYS_SYNC, "false")); alwaysSync = Boolean.parseBoolean(nifiProperties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_ALWAYS_SYNC, "false"));
// determine the database file path and ensure it exists // 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(); numPartitions = nifiProperties.getFlowFileRepositoryPartitions();
checkpointDelayMillis = FormatUtils.getTimeDuration(nifiProperties.getFlowFileRepositoryCheckpointInterval(), TimeUnit.MILLISECONDS); 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 { public void initialize(final ResourceClaimManager claimManager) throws IOException {
this.claimManager = claimManager; 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: 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 // 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 // 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 // delete backup. On restore, if no files exist in partition's directory, would have to check backup directory
serdeFactory = new RepositoryRecordSerdeFactory(claimManager); 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 @Override
@ -167,12 +179,22 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
@Override @Override
public long getStorageCapacity() throws IOException { 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 @Override
public long getUsableStorageSpace() throws IOException { 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 @Override

View File

@ -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<ResourceClaim, OutputStream> streamMap = new HashMap<>();
private final Queue<ContentClaim> 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;
}
}

View File

@ -64,6 +64,39 @@ public class ContentClaimFieldMap implements Record {
return schema; 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 @Override
public String toString() { public String toString() {
return "ContentClaimFieldMap[" + contentClaim + "]"; return "ContentClaimFieldMap[" + contentClaim + "]";

View File

@ -17,6 +17,7 @@
package org.apache.nifi.controller.repository.schema; 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.NamedValue;
import org.apache.nifi.repository.schema.Record; import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema; import org.apache.nifi.repository.schema.RecordSchema;
@ -39,7 +40,10 @@ public class RepositoryRecordUpdate implements Record {
@Override @Override
public Object getFieldValue(final String fieldName) { public Object getFieldValue(final String fieldName) {
if (RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V2.equals(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 UpdateType updateType = UpdateType.valueOf(actionType);
final String actionName; final String actionName;

View File

@ -83,7 +83,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
for (int i = 0; i < maxThreadCount; i++) { for (int i = 0; i < maxThreadCount; i++) {
final Runnable eventDrivenTask = new EventDrivenTask(workerQueue); 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; final int tasksToAdd = maxThreadCount - oldMax;
for (int i = 0; i < tasksToAdd; i++) { for (int i = 0; i < tasksToAdd; i++) {
final Runnable eventDrivenTask = new EventDrivenTask(workerQueue); final Runnable eventDrivenTask = new EventDrivenTask(workerQueue);
flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 30000, TimeUnit.NANOSECONDS); flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 1L, TimeUnit.NANOSECONDS);
} }
} }
} }

View File

@ -87,6 +87,13 @@ public enum ProcessorStatusDescriptor {
Formatter.DURATION, Formatter.DURATION,
s -> TimeUnit.MILLISECONDS.convert(s.getProcessingNanos(), TimeUnit.NANOSECONDS))), s -> TimeUnit.MILLISECONDS.convert(s.getProcessingNanos(), TimeUnit.NANOSECONDS))),
TASK_NANOS(new StandardMetricDescriptor<ProcessorStatus>(
"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<ProcessorStatus>( FLOWFILES_REMOVED(new StandardMetricDescriptor<ProcessorStatus>(
"flowFilesRemoved", "flowFilesRemoved",
"FlowFiles Removed (5 mins)", "FlowFiles Removed (5 mins)",
@ -122,35 +129,50 @@ public enum ProcessorStatusDescriptor {
} }
)), )),
AVERAGE_TASK_MILLIS(new StandardMetricDescriptor<ProcessorStatus>( AVERAGE_TASK_NANOS(new StandardMetricDescriptor<ProcessorStatus>(
"averageTaskMillis", "averageTaskNanos",
"Average Task Duration", "Average Task Duration (nanoseconds)",
"The average duration it took this Processor to complete a task, as averaged over the past 5 minutes", "The average number of nanoseconds it took this Processor to complete a task, over the past 5 minutes",
Formatter.DURATION, Formatter.COUNT,
s -> s.getInvocations() == 0 ? 0 : TimeUnit.MILLISECONDS.convert(s.getProcessingNanos(), TimeUnit.NANOSECONDS) / s.getInvocations(), s -> s.getInvocations() == 0 ? 0 : s.getProcessingNanos() / s.getInvocations(),
new ValueReducer<StatusSnapshot, Long>() { new ValueReducer<StatusSnapshot, Long>() {
@Override @Override
public Long reduce(final List<StatusSnapshot> values) { public Long reduce(final List<StatusSnapshot> values) {
long procMillis = 0L; long procNanos = 0L;
int invocations = 0; int invocations = 0;
for (final StatusSnapshot snapshot : values) { for (final StatusSnapshot snapshot : values) {
procMillis += snapshot.getStatusMetrics().get(TASK_MILLIS.getDescriptor()).longValue(); final Long taskNanos = snapshot.getStatusMetrics().get(TASK_NANOS.getDescriptor());
invocations += snapshot.getStatusMetrics().get(TASK_COUNT.getDescriptor()).intValue(); if (taskNanos != null) {
procNanos += taskNanos.longValue();
}
final Long taskInvocations = snapshot.getStatusMetrics().get(TASK_COUNT.getDescriptor());
if (taskInvocations != null) {
invocations += taskInvocations.intValue();
}
} }
if (invocations == 0) { if (invocations == 0) {
return 0L; return 0L;
} }
return procMillis / invocations; return procNanos / invocations;
} }
})); }));
private MetricDescriptor<ProcessorStatus> descriptor;
private final MetricDescriptor<ProcessorStatus> descriptor;
private final boolean visible;
private ProcessorStatusDescriptor(final MetricDescriptor<ProcessorStatus> descriptor) { private ProcessorStatusDescriptor(final MetricDescriptor<ProcessorStatus> descriptor) {
this(descriptor, true);
}
private ProcessorStatusDescriptor(final MetricDescriptor<ProcessorStatus> descriptor, final boolean visible) {
this.descriptor = descriptor; this.descriptor = descriptor;
this.visible = visible;
} }
public String getField() { public String getField() {
@ -160,4 +182,8 @@ public enum ProcessorStatusDescriptor {
public MetricDescriptor<ProcessorStatus> getDescriptor() { public MetricDescriptor<ProcessorStatus> getDescriptor() {
return descriptor; return descriptor;
} }
public boolean isVisible() {
return visible;
}
} }

View File

@ -93,8 +93,10 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit
snapshot.setTimestamp(capture.getCaptureDate()); snapshot.setTimestamp(capture.getCaptureDate());
for (final ProcessorStatusDescriptor descriptor : ProcessorStatusDescriptor.values()) { for (final ProcessorStatusDescriptor descriptor : ProcessorStatusDescriptor.values()) {
if (descriptor.isVisible()) {
snapshot.addStatusMetric(descriptor.getDescriptor(), descriptor.getDescriptor().getValueFunction().getValue(status)); snapshot.addStatusMetric(descriptor.getDescriptor(), descriptor.getDescriptor().getValueFunction().getValue(status));
} }
}
history.addStatusSnapshot(snapshot); history.addStatusSnapshot(snapshot);
return true; return true;

View File

@ -82,6 +82,13 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
return procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || !Connectables.hasNonLoopConnection(procNode) || Connectables.flowFilesQueued(procNode); 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 @Override
public Boolean call() { public Boolean call() {
// make sure processor is not yielded // make sure processor is not yielded
@ -127,6 +134,7 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
scheduleState.incrementActiveThreadCount(); scheduleState.incrementActiveThreadCount();
final long startNanos = System.nanoTime(); final long startNanos = System.nanoTime();
final long finishIfBackpressureEngaged = startNanos + (batchNanos / 25L);
final long finishNanos = startNanos + batchNanos; final long finishNanos = startNanos + batchNanos;
int invocationCount = 0; int invocationCount = 0;
try { try {
@ -140,10 +148,16 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
return false; return false;
} }
if (System.nanoTime() > finishNanos) { final long nanoTime = System.nanoTime();
if (nanoTime > finishNanos) {
return false; return false;
} }
if (nanoTime > finishIfBackpressureEngaged && isBackPressureEngaged()) {
return false;
}
if (!isWorkToDo(procNode)) { if (!isWorkToDo(procNode)) {
break; break;
} }

View File

@ -34,10 +34,15 @@ import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.StandardCopyOption; import java.nio.file.StandardCopyOption;
import java.nio.file.StandardOpenOption; import java.nio.file.StandardOpenOption;
import java.text.NumberFormat;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.nifi.controller.repository.claim.ContentClaim; 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.Logger;
import ch.qos.logback.classic.spi.ILoggingEvent; import ch.qos.logback.classic.spi.ILoggingEvent;
import ch.qos.logback.core.read.ListAppender; import ch.qos.logback.core.read.ListAppender;
import java.util.HashMap;
import java.util.Map;
public class TestFileSystemRepository { public class TestFileSystemRepository {
@ -88,6 +91,38 @@ public class TestFileSystemRepository {
repository.shutdown(); 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 @Test
public void testMinimalArchiveCleanupIntervalHonoredAndLogged() throws Exception { public void testMinimalArchiveCleanupIntervalHonoredAndLogged() throws Exception {
// We are going to construct our own repository using different properties, so // We are going to construct our own repository using different properties, so

View File

@ -86,6 +86,7 @@ import org.apache.nifi.util.NiFiProperties;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock; import org.mockito.invocation.InvocationOnMock;
@ -987,6 +988,7 @@ public class TestStandardProcessSession {
} }
@Test @Test
@Ignore
public void testManyFilesOpened() throws IOException { public void testManyFilesOpened() throws IOException {
StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000]; StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000];
@ -1672,9 +1674,9 @@ public class TestStandardProcessSession {
@Override @Override
public int incrementClaimaintCount(ContentClaim claim) { public int incrementClaimaintCount(ContentClaim claim) {
final AtomicInteger count = claimantCounts.get(claim); AtomicInteger count = claimantCounts.get(claim);
if (count == null) { if (count == null) {
throw new IllegalArgumentException("Unknown Claim: " + claim); count = new AtomicInteger(0);
} }
return count.incrementAndGet(); return count.incrementAndGet();
} }
@ -1835,6 +1837,11 @@ public class TestStandardProcessSession {
fos.write(b); fos.write(b);
((StandardContentClaim) claim).setLength(claim.getLength() + b.length); ((StandardContentClaim) claim).setLength(claim.getLength() + b.length);
} }
@Override
public void close() throws IOException {
super.close();
}
}; };
} }

View File

@ -18,6 +18,7 @@ package org.apache.nifi.controller.repository;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any; import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
@ -33,12 +34,17 @@ import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.StandardFlowFileQueue; 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.FlowFileQueue;
import org.apache.nifi.controller.queue.ListFlowFileStatus;
import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.repository.claim.ContentClaim; import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim; 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.repository.claim.StandardResourceClaimManager;
import org.apache.nifi.controller.swap.StandardSwapContents; import org.apache.nifi.controller.swap.StandardSwapContents;
import org.apache.nifi.controller.swap.StandardSwapSummary; 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.NiFiProperties;
import org.apache.nifi.util.file.FileUtils; import org.apache.nifi.util.file.FileUtils;
import org.junit.After; import org.junit.After;
import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock; import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
import org.wali.MinimalLockingWriteAheadLog;
import org.wali.WriteAheadRepository;
public class TestWriteAheadFlowFileRepository { 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<FlowFilePrioritizer> getPriorities() {
return null;
}
@Override
public SwapSummary recoverSwappedFlowFiles() {
return null;
}
@Override
public void purgeSwapFiles() {
}
@Override
public void setPriorities(List<FlowFilePrioritizer> 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<FlowFileRecord> flowFiles) {
}
@Override
public boolean isFull() {
return false;
}
@Override
public void put(FlowFileRecord file) {
}
@Override
public void putAll(Collection<FlowFileRecord> files) {
}
@Override
public FlowFileRecord poll(Set<FlowFileRecord> expiredRecords) {
return null;
}
@Override
public List<FlowFileRecord> poll(int maxResults, Set<FlowFileRecord> expiredRecords) {
return null;
}
@Override
public long drainQueue(Queue<FlowFileRecord> sourceQueue, List<FlowFileRecord> destination, int maxResults, Set<FlowFileRecord> expiredRecords) {
return 0;
}
@Override
public List<FlowFileRecord> poll(FlowFileFilter filter, Set<FlowFileRecord> 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<RepositoryRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serdeFactory, null);
final Collection<RepositoryRecord> 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<RepositoryRecord> 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<String, String> 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 @Test
public void testResourceClaimsIncremented() throws IOException { public void testResourceClaimsIncremented() throws IOException {
final ResourceClaimManager claimManager = new StandardResourceClaimManager(); final ResourceClaimManager claimManager = new StandardResourceClaimManager();

View File

@ -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);
}
}

View File

@ -98,7 +98,7 @@
<nifi.provenance.repository.rollover.time>30 secs</nifi.provenance.repository.rollover.time> <nifi.provenance.repository.rollover.time>30 secs</nifi.provenance.repository.rollover.time>
<nifi.provenance.repository.rollover.size>100 MB</nifi.provenance.repository.rollover.size> <nifi.provenance.repository.rollover.size>100 MB</nifi.provenance.repository.rollover.size>
<nifi.provenance.repository.query.threads>2</nifi.provenance.repository.query.threads> <nifi.provenance.repository.query.threads>2</nifi.provenance.repository.query.threads>
<nifi.provenance.repository.index.threads>1</nifi.provenance.repository.index.threads> <nifi.provenance.repository.index.threads>2</nifi.provenance.repository.index.threads>
<nifi.provenance.repository.compress.on.rollover>true</nifi.provenance.repository.compress.on.rollover> <nifi.provenance.repository.compress.on.rollover>true</nifi.provenance.repository.compress.on.rollover>
<nifi.provenance.repository.indexed.fields>EventType, FlowFileUUID, Filename, ProcessorID, Relationship</nifi.provenance.repository.indexed.fields> <nifi.provenance.repository.indexed.fields>EventType, FlowFileUUID, Filename, ProcessorID, Relationship</nifi.provenance.repository.indexed.fields>
<nifi.provenance.repository.indexed.attributes /> <nifi.provenance.repository.indexed.attributes />
@ -106,6 +106,8 @@
<nifi.provenance.repository.always.sync>false</nifi.provenance.repository.always.sync> <nifi.provenance.repository.always.sync>false</nifi.provenance.repository.always.sync>
<nifi.provenance.repository.journal.count>16</nifi.provenance.repository.journal.count> <nifi.provenance.repository.journal.count>16</nifi.provenance.repository.journal.count>
<nifi.provenance.repository.max.attribute.length>65536</nifi.provenance.repository.max.attribute.length> <nifi.provenance.repository.max.attribute.length>65536</nifi.provenance.repository.max.attribute.length>
<nifi.provenance.repository.concurrent.merge.threads>2</nifi.provenance.repository.concurrent.merge.threads>
<nifi.provenance.repository.warm.cache.frequency>1 hour</nifi.provenance.repository.warm.cache.frequency>
<!-- volatile provenance repository properties --> <!-- volatile provenance repository properties -->
<nifi.provenance.repository.buffer.size>100000</nifi.provenance.repository.buffer.size> <nifi.provenance.repository.buffer.size>100000</nifi.provenance.repository.buffer.size>

View File

@ -32,18 +32,26 @@ public abstract class AbstractRecordWriter implements RecordWriter {
private static final Logger logger = LoggerFactory.getLogger(AbstractRecordWriter.class); private static final Logger logger = LoggerFactory.getLogger(AbstractRecordWriter.class);
private final File file; private final File file;
private final String storageLocation;
private final TocWriter tocWriter; private final TocWriter tocWriter;
private final Lock lock = new ReentrantLock(); private final Lock lock = new ReentrantLock();
private volatile boolean dirty = false; private volatile boolean dirty = false;
private volatile boolean closed = false; private volatile boolean closed = false;
private int recordsWritten = 0;
public AbstractRecordWriter(final File file, final TocWriter writer) throws IOException { public AbstractRecordWriter(final File file, final TocWriter writer) throws IOException {
logger.trace("Creating Record Writer for {}", file); logger.trace("Creating Record Writer for {}", file);
this.file = 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; this.tocWriter = writer;
} }
@ -51,7 +59,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
public synchronized void close() throws IOException { public synchronized void close() throws IOException {
closed = true; closed = true;
logger.trace("Closing Record Writer for {}", file == null ? null : file.getName()); logger.trace("Closing Record Writer for {}", getStorageLocation());
lock(); lock();
try { try {
@ -94,9 +102,8 @@ public abstract class AbstractRecordWriter implements RecordWriter {
} }
} }
@Override protected String getStorageLocation() {
public int getRecordsWritten() { return storageLocation;
return recordsWritten;
} }
@Override @Override
@ -133,6 +140,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
this.dirty = true; this.dirty = true;
} }
@Override
public boolean isDirty() { public boolean isDirty() {
return dirty; return dirty;
} }
@ -142,7 +150,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
} }
@Override @Override
public void sync() throws IOException { public synchronized void sync() throws IOException {
try { try {
if (tocWriter != null) { if (tocWriter != null) {
tocWriter.sync(); tocWriter.sync();

View File

@ -18,34 +18,37 @@
package org.apache.nifi.provenance; package org.apache.nifi.provenance;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; 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.EventRecord;
import org.apache.nifi.provenance.schema.EventRecordFields;
import org.apache.nifi.provenance.schema.ProvenanceEventSchema; import org.apache.nifi.provenance.schema.ProvenanceEventSchema;
import org.apache.nifi.provenance.serialization.CompressableRecordWriter; import org.apache.nifi.provenance.serialization.CompressableRecordWriter;
import org.apache.nifi.provenance.toc.TocWriter; import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.repository.schema.Record; import org.apache.nifi.repository.schema.Record;
import org.apache.nifi.repository.schema.RecordSchema; import org.apache.nifi.repository.schema.RecordSchema;
import org.apache.nifi.repository.schema.SchemaRecordWriter; import org.apache.nifi.repository.schema.SchemaRecordWriter;
import org.apache.nifi.stream.io.DataOutputStream;
public class ByteArraySchemaRecordWriter extends CompressableRecordWriter { public class ByteArraySchemaRecordWriter extends CompressableRecordWriter {
private static final RecordSchema eventSchema = ProvenanceEventSchema.PROVENANCE_EVENT_SCHEMA_V1; 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 int SERIALIZATION_VERSION = 1;
public static final String SERIALIZATION_NAME = "ByteArraySchemaRecordWriter"; public static final String SERIALIZATION_NAME = "ByteArraySchemaRecordWriter";
private final SchemaRecordWriter recordWriter = new SchemaRecordWriter(); private final SchemaRecordWriter recordWriter = new SchemaRecordWriter();
public ByteArraySchemaRecordWriter(final File file, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { public ByteArraySchemaRecordWriter(final File file, final AtomicLong idGenerator, final TocWriter tocWriter, final boolean compressed,
super(file, tocWriter, compressed, uncompressedBlockSize); 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 { public ByteArraySchemaRecordWriter(final OutputStream out, final String storageLocation, final AtomicLong idGenerator, final TocWriter tocWriter, final boolean compressed,
super(out, tocWriter, compressed, uncompressedBlockSize); final int uncompressedBlockSize) throws IOException {
super(out, storageLocation, idGenerator, tocWriter, compressed, uncompressedBlockSize);
} }
@Override @Override

View File

@ -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<String> componentIds;
private List<String> componentTypes;
private List<String> queueIds;
private List<String> 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<String>) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.COMPONENT_IDS);
componentTypes = (List<String>) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.COMPONENT_TYPES);
queueIds = (List<String>) headerRecord.getFieldValue(EventIdFirstHeaderSchema.FieldNames.QUEUE_IDS);
eventTypes = (List<String>) 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<StandardProvenanceEventRecord> 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();
}
}

View File

@ -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<String, Integer> componentIdMap;
private final Map<String, Integer> componentTypeMap;
private final Map<String, Integer> queueIdMap;
private static final Map<String, Integer> eventTypeMap;
private static final List<String> eventTypeNames;
private static final TimedBuffer<TimestampedLong> serializeTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
private static final TimedBuffer<TimestampedLong> lockTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
private static final TimedBuffer<TimestampedLong> writeTimes = new TimedBuffer<>(TimeUnit.SECONDS, 60, new LongEntityAccess());
private static final TimedBuffer<TimestampedLong> 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<String, Object> 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;
}
}

View File

@ -63,7 +63,7 @@ public class IndexConfiguration {
private Map<File, List<File>> recoverIndexDirectories() { private Map<File, List<File>> recoverIndexDirectories() {
final Map<File, List<File>> indexDirectoryMap = new HashMap<>(); final Map<File, List<File>> indexDirectoryMap = new HashMap<>();
for (final File storageDirectory : repoConfig.getStorageDirectories()) { for (final File storageDirectory : repoConfig.getStorageDirectories().values()) {
final List<File> indexDirectories = new ArrayList<>(); final List<File> indexDirectories = new ArrayList<>();
final File[] matching = storageDirectory.listFiles(new FileFilter() { final File[] matching = storageDirectory.listFiles(new FileFilter() {
@Override @Override
@ -85,6 +85,10 @@ public class IndexConfiguration {
} }
private Long getFirstEntryTime(final File provenanceLogFile) { private Long getFirstEntryTime(final File provenanceLogFile) {
if (provenanceLogFile == null) {
return null;
}
try (final RecordReader reader = RecordReaders.newRecordReader(provenanceLogFile, null, Integer.MAX_VALUE)) { try (final RecordReader reader = RecordReaders.newRecordReader(provenanceLogFile, null, Integer.MAX_VALUE)) {
final StandardProvenanceEventRecord firstRecord = reader.nextRecord(); final StandardProvenanceEventRecord firstRecord = reader.nextRecord();
if (firstRecord == null) { if (firstRecord == null) {
@ -121,10 +125,14 @@ public class IndexConfiguration {
} }
} }
public File getWritableIndexDirectory(final File provenanceLogFile, final long newIndexTimestamp) { 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(); lock.lock();
try { try {
final File storageDirectory = provenanceLogFile.getParentFile();
List<File> indexDirectories = this.indexDirectoryMap.get(storageDirectory); List<File> indexDirectories = this.indexDirectoryMap.get(storageDirectory);
if (indexDirectories == null) { if (indexDirectories == null) {
final File newDir = addNewIndex(storageDirectory, provenanceLogFile, newIndexTimestamp); final File newDir = addNewIndex(storageDirectory, provenanceLogFile, newIndexTimestamp);

View File

@ -16,64 +16,6 @@
*/ */
package org.apache.nifi.provenance; 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.EOFException;
import java.io.File; import java.io.File;
import java.io.FileFilter; import java.io.FileFilter;
@ -82,7 +24,6 @@ import java.io.FilenameFilter;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -119,6 +60,69 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Pattern; import java.util.regex.Pattern;
import java.util.stream.Collectors; 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 class PersistentProvenanceRepository implements ProvenanceRepository {
public static final String EVENT_CATEGORY = "Provenance Repository"; public static final String EVENT_CATEGORY = "Provenance Repository";
@ -209,7 +213,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
} }
public PersistentProvenanceRepository(final NiFiProperties nifiProperties) throws IOException { 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 { public PersistentProvenanceRepository(final RepositoryConfiguration configuration, final int rolloverCheckMillis) throws IOException {
@ -220,7 +224,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
this.configuration = configuration; this.configuration = configuration;
this.maxAttributeChars = configuration.getMaxAttributeChars(); this.maxAttributeChars = configuration.getMaxAttributeChars();
for (final File file : configuration.getStorageDirectories()) { for (final File file : configuration.getStorageDirectories().values()) {
final Path storageDirectory = file.toPath(); final Path storageDirectory = file.toPath();
final Path journalDirectory = storageDirectory.resolve("journals"); final Path journalDirectory = storageDirectory.resolve("journals");
@ -234,7 +238,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
this.maxPartitionMillis = configuration.getMaxEventFileLife(TimeUnit.MILLISECONDS); this.maxPartitionMillis = configuration.getMaxEventFileLife(TimeUnit.MILLISECONDS);
this.maxPartitionBytes = configuration.getMaxEventFileCapacity(); this.maxPartitionBytes = configuration.getMaxEventFileCapacity();
this.indexConfig = new IndexConfiguration(configuration); this.indexConfig = new IndexConfiguration(configuration);
this.indexManager = new SimpleIndexManager(); this.indexManager = new SimpleIndexManager(configuration);
this.alwaysSync = configuration.isAlwaysSync(); this.alwaysSync = configuration.isAlwaysSync();
this.rolloverCheckMillis = rolloverCheckMillis; this.rolloverCheckMillis = rolloverCheckMillis;
@ -254,7 +258,8 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
} }
@Override @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(); writeLock.lock();
try { try {
if (initialized.getAndSet(true)) { if (initialized.getAndSet(true)) {
@ -325,82 +330,10 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
} }
} }
private static RepositoryConfiguration createRepositoryConfiguration(final NiFiProperties nifiProperties) throws IOException {
final Map<String, Path> 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<SearchableField> searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true);
final List<SearchableField> searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false);
// We always want to index the Event Time.
if (!searchableFields.contains(SearchableFields.EventTime)) {
searchableFields.add(SearchableFields.EventTime);
}
final 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 in order to override for unit tests
protected RecordWriter[] createWriters(final RepositoryConfiguration config, final long initialRecordId) throws IOException { protected RecordWriter[] createWriters(final RepositoryConfiguration config, final long initialRecordId) throws IOException {
final List<File> storageDirectories = config.getStorageDirectories(); final List<File> storageDirectories = new ArrayList<>(config.getStorageDirectories().values());
final RecordWriter[] writers = new RecordWriter[config.getJournalCount()]; final RecordWriter[] writers = new RecordWriter[config.getJournalCount()];
for (int i = 0; i < config.getJournalCount(); i++) { 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 journalDirectory = new File(storageDirectory, "journals");
final File journalFile = new File(journalDirectory, String.valueOf(initialRecordId) + ".journal." + i); 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); writers[i].writeHeader(initialRecordId);
} }
@ -460,7 +393,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
return Result.Approved.equals(result.getResult()); 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) { if (authorizer == null) {
return; return;
} }
@ -474,11 +407,11 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
eventAuthorizable.authorize(authorizer, RequestAction.READ, user, event.getAttributes()); eventAuthorizable.authorize(authorizer, RequestAction.READ, user, event.getAttributes());
} }
private List<ProvenanceEventRecord> filterUnauthorizedEvents(final List<ProvenanceEventRecord> events, final NiFiUser user) { public List<ProvenanceEventRecord> filterUnauthorizedEvents(final List<ProvenanceEventRecord> events, final NiFiUser user) {
return events.stream().filter(event -> isAuthorized(event, user)).collect(Collectors.<ProvenanceEventRecord>toList()); return events.stream().filter(event -> isAuthorized(event, user)).collect(Collectors.<ProvenanceEventRecord>toList());
} }
private Set<ProvenanceEventRecord> replaceUnauthorizedWithPlaceholders(final Set<ProvenanceEventRecord> events, final NiFiUser user) { public Set<ProvenanceEventRecord> replaceUnauthorizedWithPlaceholders(final Set<ProvenanceEventRecord> events, final NiFiUser user) {
return events.stream().map(event -> isAuthorized(event, user) ? event : new PlaceholderProvenanceEvent(event)).collect(Collectors.toSet()); 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; long minIndexedId = Long.MAX_VALUE;
final List<File> filesToRecover = new ArrayList<>(); final List<File> filesToRecover = new ArrayList<>();
for (final File file : configuration.getStorageDirectories()) { for (final File file : configuration.getStorageDirectories().values()) {
final File[] matchingFiles = file.listFiles(new FileFilter() { final File[] matchingFiles = file.listFiles(new FileFilter() {
@Override @Override
public boolean accept(final File pathname) { public boolean accept(final File pathname) {
@ -780,10 +713,10 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
try { try {
long recordsWritten = 0L; long recordsWritten = 0L;
for (final ProvenanceEventRecord nextRecord : records) { for (final ProvenanceEventRecord nextRecord : records) {
final long eventId = idGenerator.getAndIncrement(); final StorageSummary persistedEvent = writer.writeRecord(nextRecord);
bytesWritten += writer.writeRecord(nextRecord, eventId); bytesWritten += persistedEvent.getSerializedLength();
recordsWritten++; recordsWritten++;
logger.trace("Wrote record with ID {} to {}", eventId, writer); logger.trace("Wrote record with ID {} to {}", persistedEvent.getEventId(), writer);
} }
writer.flush(); writer.flush();
@ -1175,7 +1108,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
*/ */
private List<File> getAllIndexDirectories() { private List<File> getAllIndexDirectories() {
final List<File> allIndexDirs = new ArrayList<>(); final List<File> allIndexDirs = new ArrayList<>();
for (final File storageDir : configuration.getStorageDirectories()) { for (final File storageDir : configuration.getStorageDirectories().values()) {
final File[] indexDirs = storageDir.listFiles(new FilenameFilter() { final File[] indexDirs = storageDir.listFiles(new FilenameFilter() {
@Override @Override
public boolean accept(final File dir, final String name) { public boolean accept(final File dir, final String name) {
@ -1237,7 +1170,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
protected int getJournalCount() { protected int getJournalCount() {
// determine how many 'journals' we have in the journals directories // determine how many 'journals' we have in the journals directories
int journalFileCount = 0; 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 journalsDir = new File(storageDir, "journals");
final File[] journalFiles = journalsDir.listFiles(); final File[] journalFiles = journalsDir.listFiles();
if (journalFiles != null) { if (journalFiles != null) {
@ -1313,7 +1246,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
// Choose a storage directory to store the merged file in. // Choose a storage directory to store the merged file in.
final long storageDirIdx = storageDirectoryIndex.getAndIncrement(); final long storageDirIdx = storageDirectoryIndex.getAndIncrement();
final List<File> storageDirs = configuration.getStorageDirectories(); final List<File> storageDirs = new ArrayList<>(configuration.getStorageDirectories().values());
final File storageDir = storageDirs.get((int) (storageDirIdx % storageDirs.size())); final File storageDir = storageDirs.get((int) (storageDirIdx % storageDirs.size()));
Future<?> future = null; Future<?> future = null;
@ -1479,8 +1412,8 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
final Map<String, List<File>> journalMap = new HashMap<>(); final Map<String, List<File>> journalMap = new HashMap<>();
// Map journals' basenames to the files with that basename. // Map journals' basenames to the files with that basename.
final List<File> storageDirs = configuration.getStorageDirectories(); final List<File> storageDirs = new ArrayList<>(configuration.getStorageDirectories().values());
for (final File storageDir : configuration.getStorageDirectories()) { for (final File storageDir : storageDirs) {
final File journalDir = new File(storageDir, "journals"); final File journalDir = new File(storageDir, "journals");
if (!journalDir.exists()) { if (!journalDir.exists()) {
continue; 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 // 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. // 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); writer.writeHeader(minEventId);
final IndexingAction indexingAction = createIndexingAction(); final IndexingAction indexingAction = createIndexingAction();
@ -1741,7 +1674,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
final AtomicBoolean finishedAdding = new AtomicBoolean(false); final AtomicBoolean finishedAdding = new AtomicBoolean(false);
final List<Future<?>> futures = new ArrayList<>(); final List<Future<?>> futures = new ArrayList<>();
final IndexWriter indexWriter = getIndexManager().borrowIndexWriter(indexingDirectory); final EventIndexWriter indexWriter = getIndexManager().borrowIndexWriter(indexingDirectory);
try { try {
final ExecutorService exec = Executors.newFixedThreadPool(configuration.getIndexThreadPoolSize(), new ThreadFactory() { final ExecutorService exec = Executors.newFixedThreadPool(configuration.getIndexThreadPoolSize(), new ThreadFactory() {
@Override @Override
@ -1772,7 +1705,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
continue; continue;
} }
indexingAction.index(tuple.getKey(), indexWriter, tuple.getValue()); indexingAction.index(tuple.getKey(), indexWriter.getIndexWriter(), tuple.getValue());
} catch (final Throwable t) { } catch (final Throwable t) {
logger.error("Failed to index Provenance Event for " + writerFile + " to " + indexingDirectory, t); logger.error("Failed to index Provenance Event for " + writerFile + " to " + indexingDirectory, t);
if (indexingFailureCount.incrementAndGet() >= MAX_INDEXING_FAILURE_COUNT) { if (indexingFailureCount.incrementAndGet() >= MAX_INDEXING_FAILURE_COUNT) {
@ -1795,7 +1728,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
final StandardProvenanceEventRecord record = entry.getKey(); final StandardProvenanceEventRecord record = entry.getKey();
final RecordReader reader = entry.getValue(); final RecordReader reader = entry.getValue();
writer.writeRecord(record, record.getEventId()); writer.writeRecord(record);
final int blockIndex = writer.getTocWriter().getCurrentBlockIndex(); final int blockIndex = writer.getTocWriter().getCurrentBlockIndex();
boolean accepted = false; boolean accepted = false;
@ -1879,7 +1812,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
} }
} }
} finally { } finally {
getIndexManager().returnIndexWriter(indexingDirectory, indexWriter); getIndexManager().returnIndexWriter(indexWriter);
} }
indexConfig.setMaxIdIndexed(maxId); indexConfig.setMaxIdIndexed(maxId);
@ -1945,7 +1878,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
* events indexed, etc. * events indexed, etc.
*/ */
protected IndexingAction createIndexingAction() { protected IndexingAction createIndexingAction() {
return new IndexingAction(this); return new IndexingAction(configuration.getSearchableFields(), configuration.getSearchableAttributes());
} }
private StandardProvenanceEventRecord truncateAttributes(final StandardProvenanceEventRecord original) { private StandardProvenanceEventRecord truncateAttributes(final StandardProvenanceEventRecord original) {
@ -2322,7 +2255,7 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
if (event == null) { if (event == null) {
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1, userId); final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1, userId);
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList()); submission.getResult().update(Collections.emptyList(), 0L);
return submission; return submission;
} }
@ -2359,9 +2292,9 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
try { try {
final ProvenanceEventRecord event = getEvent(eventId); final ProvenanceEventRecord event = getEvent(eventId);
if (event == null) { if (event == null) {
final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.<String>emptyList(), 1, userId); final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.emptyList(), 1, userId);
lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission);
submission.getResult().update(Collections.<ProvenanceEventRecord>emptyList()); submission.getResult().update(Collections.emptyList(), 0L);
return submission; return submission;
} }
@ -2642,11 +2575,21 @@ public class PersistentProvenanceRepository implements ProvenanceRepository {
} }
try { try {
final Set<ProvenanceEventRecord> matchingRecords = LineageQuery.computeLineageForFlowFiles(PersistentProvenanceRepository.this, final DocumentToEventConverter converter = new DocumentToEventConverter() {
getIndexManager(), indexDir, null, flowFileUuids, maxAttributeChars); @Override
public Set<ProvenanceEventRecord> 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<ProvenanceEventRecord> matchingRecords = LineageQuery.computeLineageForFlowFiles(getIndexManager(), indexDir, null, flowFileUuids, converter);
final StandardLineageResult result = submission.getResult(); 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", 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()); 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 { private class RemoveExpiredQueryResults implements Runnable {
@Override @Override
public void run() { public void run() {
try { 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;
}
}
} }

View File

@ -17,20 +17,35 @@
package org.apache.nifi.provenance; package org.apache.nifi.provenance;
import java.io.File; import java.io.File;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.provenance.search.SearchableField; 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 { public class RepositoryConfiguration {
private static final Logger logger = LoggerFactory.getLogger(RepositoryConfiguration.class);
private final List<File> 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<String, File> storageDirectories = new LinkedHashMap<>();
private long recordLifeMillis = TimeUnit.MILLISECONDS.convert(24, TimeUnit.HOURS); private long recordLifeMillis = TimeUnit.MILLISECONDS.convert(24, TimeUnit.HOURS);
private long storageCapacity = 1024L * 1024L * 1024L; // 1 GB private long storageCapacity = 1024L * 1024L * 1024L; // 1 GB
private long eventFileMillis = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES); private long eventFileMillis = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES);
private long eventFileBytes = 1024L * 1024L * 5L; // 5 MB private long eventFileBytes = 1024L * 1024L * 5L; // 5 MB
private int maxFileEvents = Integer.MAX_VALUE;
private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB private long desiredIndexBytes = 1024L * 1024L * 500L; // 500 MB
private int journalCount = 16; private int journalCount = 16;
private int compressionBlockBytes = 1024 * 1024; private int compressionBlockBytes = 1024 * 1024;
@ -43,6 +58,8 @@ public class RepositoryConfiguration {
private int queryThreadPoolSize = 2; private int queryThreadPoolSize = 2;
private int indexThreadPoolSize = 1; private int indexThreadPoolSize = 1;
private boolean allowRollover = true; private boolean allowRollover = true;
private int concurrentMergeThreads = 4;
private Integer warmCacheFrequencyMinutes = null;
public void setAllowRollover(final boolean allow) { public void setAllowRollover(final boolean allow) {
this.allowRollover = allow; this.allowRollover = allow;
@ -52,7 +69,6 @@ public class RepositoryConfiguration {
return allowRollover; return allowRollover;
} }
public int getCompressionBlockBytes() { public int getCompressionBlockBytes() {
return compressionBlockBytes; return compressionBlockBytes;
} }
@ -66,8 +82,8 @@ public class RepositoryConfiguration {
* *
* @return the directories where provenance files will be stored * @return the directories where provenance files will be stored
*/ */
public List<File> getStorageDirectories() { public Map<String, File> getStorageDirectories() {
return Collections.unmodifiableList(storageDirectories); return Collections.unmodifiableMap(storageDirectories);
} }
/** /**
@ -75,8 +91,12 @@ public class RepositoryConfiguration {
* *
* @param storageDirectory the directory to store provenance files * @param storageDirectory the directory to store provenance files
*/ */
public void addStorageDirectory(final File storageDirectory) { public void addStorageDirectory(final String partitionName, final File storageDirectory) {
this.storageDirectories.add(storageDirectory); this.storageDirectories.put(partitionName, storageDirectory);
}
public void addStorageDirectories(final Map<String, File> storageDirectories) {
this.storageDirectories.putAll(storageDirectories);
} }
/** /**
@ -147,6 +167,20 @@ public class RepositoryConfiguration {
this.eventFileBytes = maxEventFileBytes; 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 * @return the fields that should be indexed
*/ */
@ -218,6 +252,14 @@ public class RepositoryConfiguration {
this.indexThreadPoolSize = indexThreadPoolSize; this.indexThreadPoolSize = indexThreadPoolSize;
} }
public void setConcurrentMergeThreads(final int mergeThreads) {
this.concurrentMergeThreads = mergeThreads;
}
public int getConcurrentMergeThreads() {
return concurrentMergeThreads;
}
/** /**
* <p> * <p>
* Specifies the desired size of each Provenance Event index shard, in * Specifies the desired size of each Provenance Event index shard, in
@ -310,4 +352,90 @@ public class RepositoryConfiguration {
this.maxAttributeChars = maxAttributeChars; this.maxAttributeChars = maxAttributeChars;
} }
public void setWarmCacheFrequencyMinutes(Integer frequencyMinutes) {
this.warmCacheFrequencyMinutes = frequencyMinutes;
}
public Optional<Integer> getWarmCacheFrequencyMinutes() {
return Optional.ofNullable(warmCacheFrequencyMinutes);
}
public static RepositoryConfiguration create(final NiFiProperties nifiProperties) {
final Map<String, Path> 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<SearchableField> searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true);
final List<SearchableField> searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false);
// We always want to index the Event Time.
if (!searchableFields.contains(SearchableFields.EventTime)) {
searchableFields.add(SearchableFields.EventTime);
}
final RepositoryConfiguration config = new RepositoryConfiguration();
for (final Map.Entry<String, Path> 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;
}
} }

View File

@ -32,6 +32,9 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
public class StandardRecordReader extends CompressableRecordReader { 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 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}"); 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 @Override
public StandardProvenanceEventRecord nextRecord(final DataInputStream dis, final int serializationVersion) throws IOException { 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 " 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 // Schema changed drastically in version 6 so we created a new method to handle old records

View File

@ -16,17 +16,18 @@
*/ */
package org.apache.nifi.provenance; package org.apache.nifi.provenance;
import java.io.DataOutputStream;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.io.UTFDataFormatException; import java.io.UTFDataFormatException;
import java.util.Collection; import java.util.Collection;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.provenance.serialization.CompressableRecordWriter; import org.apache.nifi.provenance.serialization.CompressableRecordWriter;
import org.apache.nifi.provenance.serialization.RecordWriter; import org.apache.nifi.provenance.serialization.RecordWriter;
import org.apache.nifi.provenance.toc.TocWriter; import org.apache.nifi.provenance.toc.TocWriter;
import org.apache.nifi.stream.io.DataOutputStream;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -45,15 +46,16 @@ public class StandardRecordWriter extends CompressableRecordWriter implements Re
private final File file; private final File file;
public StandardRecordWriter(final File file, final TocWriter writer, final boolean compressed, final int uncompressedBlockSize) throws IOException { public StandardRecordWriter(final File file, final AtomicLong idGenerator, final TocWriter writer, final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(file, writer, compressed, uncompressedBlockSize); super(file, idGenerator, writer, compressed, uncompressedBlockSize);
logger.trace("Creating Record Writer for {}", file.getName()); logger.trace("Creating Record Writer for {}", file.getName());
this.file = file; this.file = file;
} }
public StandardRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { public StandardRecordWriter(final OutputStream out, final String storageLocation, final AtomicLong idGenerator, final TocWriter tocWriter,
super(out, tocWriter, compressed, uncompressedBlockSize); final boolean compressed, final int uncompressedBlockSize) throws IOException {
super(out, storageLocation, idGenerator, tocWriter, compressed, uncompressedBlockSize);
this.file = null; this.file = null;
} }

View File

@ -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;
/**
* <p>
* 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.
* </p>
*
* <p>
* 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.
* </p>
*
* <p>
* 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.
* </p>
*
* <p>
* 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.
* </p>
*/
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<ProvenanceEventRecord> 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<ProvenanceEventRecord, StorageSummary> locationMap = storageResult.getStorageLocations();
if (!locationMap.isEmpty()) {
eventIndex.addEvents(locationMap);
}
}
@Override
public List<ProvenanceEventRecord> 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<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords, final NiFiUser user) throws IOException {
final List<ProvenanceEventRecord> 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<SearchableField> getSearchableFields() {
return Collections.unmodifiableList(config.getSearchableFields());
}
@Override
public List<SearchableField> getSearchableAttributes() {
return Collections.unmodifiableList(config.getSearchableAttributes());
}
}

View File

@ -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<ProvenanceEventRecord> filterUnauthorizedEvents(List<ProvenanceEventRecord> 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<ProvenanceEventRecord> replaceUnauthorizedWithPlaceholders(Set<ProvenanceEventRecord> 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<ProvenanceEventRecord> filterUnauthorizedEvents(List<ProvenanceEventRecord> events) {
return events;
}
@Override
public Set<ProvenanceEventRecord> replaceUnauthorizedWithPlaceholders(Set<ProvenanceEventRecord> 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<ProvenanceEventRecord> filterUnauthorizedEvents(List<ProvenanceEventRecord> events) {
return Collections.emptyList();
}
@Override
public Set<ProvenanceEventRecord> replaceUnauthorizedWithPlaceholders(Set<ProvenanceEventRecord> events) {
return events.stream()
.map(event -> new PlaceholderProvenanceEvent(event))
.collect(Collectors.toSet());
}
};
}

View File

@ -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<ProvenanceEventRecord> 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));
}

View File

@ -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());
}
}

View File

@ -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<ProvenanceEventRecord, StorageSummary> 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<ProvenanceEventRecord, StorageSummary> 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 <code>null</code> 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 <code>null</code> 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;
}

View File

@ -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();
}

View File

@ -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<Document> documents, int commitThreshold) throws IOException;
File getDirectory();
long commit() throws IOException;
int getEventsIndexedSinceCommit();
long getEventsIndexed();
IndexWriter getIndexWriter();
}

View File

@ -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);
}
}

View File

@ -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<List<Long>> evaluate(Query query);
}

View File

@ -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;
}

View File

@ -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<SearchableField> searchableEventFields;
private final Set<SearchableField> searchableAttributeFields;
public ConvertEventToLuceneDocument(final List<SearchableField> searchableEventFields, final List<SearchableField> 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);
}
}
}

View File

@ -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<StoredDocument> 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<StoredDocument> 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<StoredDocument> 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<StoredDocument> 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<String, List<StoredDocument>> docsByPartition = toIndex.stream()
.collect(Collectors.groupingBy(doc -> doc.getStorageSummary().getPartitionName().get()));
for (final Map.Entry<String, List<StoredDocument>> entry : docsByPartition.entrySet()) {
final String partitionName = entry.getKey();
final List<StoredDocument> 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<IndexableDocument> toIndex, final CommitPreference commitPreference) throws IOException {
if (toIndex.isEmpty()) {
return;
}
final Map<File, List<IndexableDocument>> docsByIndexDir = toIndex.stream().collect(Collectors.groupingBy(doc -> doc.getIndexDirectory()));
for (final Map.Entry<File, List<IndexableDocument>> entry : docsByIndexDir.entrySet()) {
final File indexDirectory = entry.getKey();
final List<IndexableDocument> 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<Long> query = NumericRangeQuery.newLongRange(
SearchableFields.Identifier.getSearchableFieldName(), minId, maxId, true, true);
indexWriter.getIndexWriter().deleteDocuments(query);
final List<Document> 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<StoredDocument> 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<Document> 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<File> 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);
}
}

View File

@ -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<Long, List<IndexLocation>> indexLocationByTimestamp = new TreeMap<>();
private final Map<String, IndexLocation> activeIndices = new HashMap<>();
public IndexDirectoryManager(final RepositoryConfiguration repoConfig) {
this.repoConfig = repoConfig;
}
public synchronized void initialize() {
final Map<File, Tuple<Long, IndexLocation>> latestIndexByStorageDir = new HashMap<>();
for (final Map.Entry<String, File> 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<IndexLocation> dirsForTimestamp = indexLocationByTimestamp.computeIfAbsent(startTime, t -> new ArrayList<>());
final IndexLocation indexLoc = new IndexLocation(indexDir, startTime, partitionName, repoConfig.getDesiredIndexSize());
dirsForTimestamp.add(indexLoc);
final Tuple<Long, IndexLocation> 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<Long, IndexLocation> tuple : latestIndexByStorageDir.values()) {
final IndexLocation indexLoc = tuple.getValue();
activeIndices.put(indexLoc.getPartitionName(), indexLoc);
}
}
public synchronized void deleteDirectory(final File directory) {
final Iterator<Map.Entry<Long, List<IndexLocation>>> itr = indexLocationByTimestamp.entrySet().iterator();
while (itr.hasNext()) {
final Map.Entry<Long, List<IndexLocation>> entry = itr.next();
final List<IndexLocation> 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<File> getDirectoriesBefore(final long timestamp) {
final List<File> 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<String, List<IndexLocation>> 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<IndexLocation> 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<IndexLocation> flattenDirectoriesByTimestamp() {
final List<IndexLocation> startTimeWithFile = new ArrayList<>();
for (final Map.Entry<Long, List<IndexLocation>> entry : indexLocationByTimestamp.entrySet()) {
for (final IndexLocation indexLoc : entry.getValue()) {
startTimeWithFile.add(indexLoc);
}
}
return startTimeWithFile;
}
public synchronized List<File> getDirectories(final Long startTime, final Long endTime) {
final List<File> 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<String, List<IndexLocation>> startTimeWithFileByStorageDirectory = flattenDirectoriesByTimestamp().stream()
.collect(Collectors.groupingBy(indexLoc -> indexLoc.getPartitionName()));
for (final List<IndexLocation> locationList : startTimeWithFileByStorageDirectory.values()) {
selected.addAll(getDirectories(startTime, endTime, locationList));
}
return selected;
}
public synchronized List<File> 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<String, List<IndexLocation>> startTimeWithFileByStorageDirectory = flattenDirectoriesByTimestamp().stream()
.collect(Collectors.groupingBy(indexLoc -> indexLoc.getPartitionName()));
final List<IndexLocation> indexLocations = startTimeWithFileByStorageDirectory.get(partitionName);
if (indexLocations == null) {
return Collections.emptyList();
}
return getDirectories(startTime, endTime, indexLocations);
}
protected static List<File> getDirectories(final Long startTime, final Long endTime, final List<IndexLocation> locations) {
final List<File> 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 <code>true</code> if the index directory has reached its max threshold and should no
* longer be written to, <code>false</code> 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<String, IndexLocation> 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<File> 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;
}
}

View File

@ -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 + "]";
}
}

View File

@ -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;
}
}

View File

@ -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<String, RingBuffer<Long>> latestRecords = new ConcurrentHashMap<>();
@Override
public void update(final ProvenanceEventRecord event, final StorageSummary storageSummary) {
final String componentId = event.getComponentId();
final RingBuffer<Long> ringBuffer = latestRecords.computeIfAbsent(componentId, id -> new RingBuffer<>(1000));
ringBuffer.add(storageSummary.getEventId());
}
@Override
public Optional<List<Long>> 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<SearchTerm> 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<Long> ringBuffer = latestRecords.get(term.getValue());
if (ringBuffer == null || ringBuffer.getSize() < query.getMaxResults()) {
return Optional.empty();
}
List<Long> eventIds = ringBuffer.asList();
if (eventIds.size() > query.getMaxResults()) {
eventIds = eventIds.subList(0, query.getMaxResults());
}
return Optional.of(eventIds);
}
}

View File

@ -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<Long> latestRecords = new RingBuffer<>(1000);
@Override
public void update(final ProvenanceEventRecord event, final StorageSummary storageSummary) {
latestRecords.add(storageSummary.getEventId());
}
@Override
public Optional<List<Long>> evaluate(final Query query) {
if (latestRecords.getSize() < query.getMaxResults()) {
return Optional.empty();
}
if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) {
final List<Long> 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();
}
}
}

View File

@ -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);
}
}
}

View File

@ -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<String, AsyncQuerySubmission> querySubmissionMap = new ConcurrentHashMap<>();
private final ConcurrentMap<String, AsyncLineageSubmission> lineageSubmissionMap = new ConcurrentHashMap<>();
private final BlockingQueue<StoredDocument> documentQueue = new LinkedBlockingQueue<>(1000);
private final List<EventIndexTask> 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<TimestampedLong> queuePauseNanos = new TimedBuffer<>(TimeUnit.SECONDS, 300, new LongEntityAccess());
private final TimedBuffer<TimestampedLong> eventsIndexed = new TimedBuffer<>(TimeUnit.SECONDS, 300, new LongEntityAccess());
private final AtomicLong eventCount = new AtomicLong(0L);
private final EventReporter eventReporter;
private final List<CachedQuery> 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<Integer> 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<File> 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<ProvenanceEventRecord, StorageSummary> 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<IndexableDocument> indexableDocs = new ArrayList<>(events.size());
for (final Map.Entry<ProvenanceEventRecord, StorageSummary> 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<File> 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<File> 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<ProvenanceEventRecord, StorageSummary> events) {
eventsIndexed.add(new TimestampedLong((long) events.size()));
for (final Map.Entry<ProvenanceEventRecord, StorageSummary> entry : events.entrySet()) {
addEvent(entry.getKey(), entry.getValue());
}
}
@Override
public ComputeLineageSubmission submitLineageComputation(final long eventId, final NiFiUser user, final EventAuthorizer eventAuthorizer) {
final Optional<ProvenanceEventRecord> 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.<String> 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.<String> 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<String> flowFileUuids, final NiFiUser user, final EventAuthorizer eventAuthorizer,
final LineageComputationType computationType, final Long eventId, final long startTimestamp, final long endTimestamp) {
final List<File> 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<File> 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<String> 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<List<Long>> eventIdListOption = cachedQuery.evaluate(query);
if (eventIdListOption.isPresent()) {
final AsyncQuerySubmission submission = new AsyncQuerySubmission(query, 1, userId);
querySubmissionMap.put(query.getIdentifier(), submission);
final List<Long> eventIds = eventIdListOption.get();
queryExecutor.submit(() -> {
List<ProvenanceEventRecord> 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<File> 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<ProvenanceEventRecord> 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.<String> 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.<String> 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<ProvenanceEventRecord> 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.<String> 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.<String> 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<ProvenanceEventRecord> 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<File> 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<Map.Entry<String, AsyncQuerySubmission>> queryIterator = querySubmissionMap.entrySet().iterator();
while (queryIterator.hasNext()) {
final Map.Entry<String, AsyncQuerySubmission> entry = queryIterator.next();
final StandardQueryResult result = entry.getValue().getResult();
if (entry.getValue().isCanceled() || result.isFinished() && result.getExpiration().before(now)) {
queryIterator.remove();
}
}
final Iterator<Map.Entry<String, AsyncLineageSubmission>> lineageIterator = lineageSubmissionMap.entrySet().iterator();
while (lineageIterator.hasNext()) {
final Map.Entry<String, AsyncLineageSubmission> 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);
}
}
}
}

View File

@ -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<String> 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<List<ProvenanceEventRecord>, 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<List<ProvenanceEventRecord>, 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.<ProvenanceEventRecord> emptyList(), 0);
}
final long start = System.nanoTime();
final List<Long> 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<ProvenanceEventRecord> 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);
}
}

View File

@ -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;
}
}

View File

@ -36,6 +36,8 @@ import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -47,7 +49,8 @@ public class CachingIndexManager implements Closeable, IndexManager {
private final Map<File, List<ActiveIndexSearcher>> activeSearchers = new HashMap<>(); private final Map<File, List<ActiveIndexSearcher>> activeSearchers = new HashMap<>();
public void removeIndex(final File indexDirectory) { @Override
public boolean removeIndex(final File indexDirectory) {
final File absoluteFile = indexDirectory.getAbsoluteFile(); final File absoluteFile = indexDirectory.getAbsoluteFile();
logger.info("Removing index {}", indexDirectory); logger.info("Removing index {}", indexDirectory);
@ -62,6 +65,8 @@ public class CachingIndexManager implements Closeable, IndexManager {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.warn("", ioe); logger.warn("", ioe);
} }
return false;
} }
} }
@ -76,24 +81,29 @@ public class CachingIndexManager implements Closeable, IndexManager {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.warn("", ioe); logger.warn("", ioe);
} }
return false;
} }
} }
} }
} finally { } finally {
lock.unlock(); lock.unlock();
} }
return true;
} }
public IndexWriter borrowIndexWriter(final File indexingDirectory) throws IOException { @Override
final File absoluteFile = indexingDirectory.getAbsoluteFile(); public EventIndexWriter borrowIndexWriter(final File indexDirectory) throws IOException {
logger.trace("Borrowing index writer for {}", indexingDirectory); final File absoluteFile = indexDirectory.getAbsoluteFile();
logger.trace("Borrowing index writer for {}", indexDirectory);
lock.lock(); lock.lock();
try { try {
IndexWriterCount writerCount = writerCounts.remove(absoluteFile); IndexWriterCount writerCount = writerCounts.remove(absoluteFile);
if (writerCount == null) { if (writerCount == null) {
final List<Closeable> closeables = new ArrayList<>(); final List<Closeable> closeables = new ArrayList<>();
final Directory directory = FSDirectory.open(indexingDirectory); final Directory directory = FSDirectory.open(indexDirectory);
closeables.add(directory); closeables.add(directory);
try { try {
@ -104,8 +114,9 @@ public class CachingIndexManager implements Closeable, IndexManager {
config.setWriteLockTimeout(300000L); config.setWriteLockTimeout(300000L);
final IndexWriter indexWriter = new IndexWriter(directory, config); final IndexWriter indexWriter = new IndexWriter(directory, config);
writerCount = new IndexWriterCount(indexWriter, analyzer, directory, 1); final EventIndexWriter eventIndexWriter = new LuceneEventIndexWriter(indexWriter, indexDirectory);
logger.debug("Providing new index writer for {}", indexingDirectory); writerCount = new IndexWriterCount(eventIndexWriter, analyzer, directory, 1);
logger.debug("Providing new index writer for {}", indexDirectory);
} catch (final IOException ioe) { } catch (final IOException ioe) {
for (final Closeable closeable : closeables) { for (final Closeable closeable : closeables) {
try { try {
@ -124,12 +135,12 @@ public class CachingIndexManager implements Closeable, IndexManager {
final List<ActiveIndexSearcher> searchers = activeSearchers.get(absoluteFile); final List<ActiveIndexSearcher> searchers = activeSearchers.get(absoluteFile);
if (searchers != null) { if (searchers != null) {
for (final ActiveIndexSearcher activeSearcher : searchers) { 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(); activeSearcher.poison();
} }
} }
} else { } 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(), writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(),
writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1)); writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1));
} }
@ -140,26 +151,48 @@ public class CachingIndexManager implements Closeable, IndexManager {
} }
} }
public void returnIndexWriter(final File indexingDirectory, final IndexWriter writer) {
final File absoluteFile = indexingDirectory.getAbsoluteFile(); @Override
logger.trace("Returning Index Writer for {} to IndexManager", indexingDirectory); 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(); lock.lock();
try { try {
final IndexWriterCount count = writerCounts.remove(absoluteFile); final IndexWriterCount count = writerCounts.get(absoluteFile);
try { try {
if (count == null) { if (count == null) {
logger.warn("Index Writer {} was returned to IndexManager for {}, but this writer is not known. " 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(); writer.close();
} else if (count.getCount() <= 1) { } else if (count.getCount() <= 1) {
// we are finished with this writer. // we are finished with this writer.
logger.debug("Decrementing count for Index Writer for {} to {}; Closing writer", indexingDirectory, count.getCount() - 1); 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(); count.close();
} finally {
writerCounts.remove(absoluteFile);
}
}
}
} else { } else {
// decrement the count. // 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)); writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), count.getDirectory(), count.getCount() - 1));
} }
} catch (final IOException ioe) { } catch (final IOException 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(); final File absoluteFile = indexDir.getAbsoluteFile();
logger.trace("Borrowing index searcher for {}", indexDir); logger.trace("Borrowing index searcher for {}", indexDir);
@ -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 // 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. // 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) {
// if refCount == 0, then the reader has been closed, so we cannot use the searcher // 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 {}; " logger.debug("Reference count for cached Index Searcher for {} is currently {}; "
@ -223,9 +257,10 @@ public class CachingIndexManager implements Closeable, IndexManager {
try { try {
final DirectoryReader directoryReader = DirectoryReader.open(directory); final DirectoryReader directoryReader = DirectoryReader.open(directory);
final IndexSearcher searcher = new IndexSearcher(directoryReader); 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. // 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); currentlyCached.add(cached);
return cached.getSearcher(); return cached.getSearcher();
@ -252,13 +287,14 @@ public class CachingIndexManager implements Closeable, IndexManager {
// create a new Index Searcher from the writer so that we don't have an issue with trying // 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 // 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. // Lucene, this indicates that an IndexWriter already has the directory open.
final IndexWriter writer = writerCount.getWriter(); final EventIndexWriter writer = writerCount.getWriter();
final DirectoryReader directoryReader = DirectoryReader.open(writer, false); final DirectoryReader directoryReader = DirectoryReader.open(writer.getIndexWriter(), false);
final IndexSearcher searcher = new IndexSearcher(directoryReader); 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 // 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. // 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); currentlyCached.add(activeSearcher);
return activeSearcher.getSearcher(); 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(); final File absoluteFile = indexDirectory.getAbsoluteFile();
logger.trace("Returning index searcher for {} to IndexManager", indexDirectory); logger.trace("Returning index searcher for {} to IndexManager", indexDirectory);
@ -438,7 +476,7 @@ public class CachingIndexManager implements Closeable, IndexManager {
private static class ActiveIndexSearcher { private static class ActiveIndexSearcher {
private final IndexSearcher searcher; private final EventIndexSearcher searcher;
private final DirectoryReader directoryReader; private final DirectoryReader directoryReader;
private final File indexDirectory; private final File indexDirectory;
private final Directory directory; private final Directory directory;
@ -446,7 +484,7 @@ public class CachingIndexManager implements Closeable, IndexManager {
private final AtomicInteger referenceCount = new AtomicInteger(1); private final AtomicInteger referenceCount = new AtomicInteger(1);
private volatile boolean poisoned = false; private volatile boolean poisoned = false;
public ActiveIndexSearcher(final IndexSearcher searcher, final File indexDirectory, final DirectoryReader directoryReader, public ActiveIndexSearcher(final EventIndexSearcher searcher, final File indexDirectory, final DirectoryReader directoryReader,
final Directory directory, final boolean cache) { final Directory directory, final boolean cache) {
this.searcher = searcher; this.searcher = searcher;
this.directoryReader = directoryReader; this.directoryReader = directoryReader;
@ -459,7 +497,7 @@ public class CachingIndexManager implements Closeable, IndexManager {
return cache; return cache;
} }
public IndexSearcher getSearcher() { public EventIndexSearcher getSearcher() {
return searcher; return searcher;
} }
@ -499,12 +537,12 @@ public class CachingIndexManager implements Closeable, IndexManager {
private static class IndexWriterCount implements Closeable { private static class IndexWriterCount implements Closeable {
private final IndexWriter writer; private final EventIndexWriter writer;
private final Analyzer analyzer; private final Analyzer analyzer;
private final Directory directory; private final Directory directory;
private final int count; 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.writer = writer;
this.analyzer = analyzer; this.analyzer = analyzer;
this.directory = directory; this.directory = directory;
@ -519,7 +557,7 @@ public class CachingIndexManager implements Closeable, IndexManager {
return directory; return directory;
} }
public IndexWriter getWriter() { public EventIndexWriter getWriter() {
return writer; return writer;
} }

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.Term;
import org.apache.nifi.provenance.IndexConfiguration; import org.apache.nifi.provenance.IndexConfiguration;
import org.apache.nifi.provenance.PersistentProvenanceRepository; import org.apache.nifi.provenance.PersistentProvenanceRepository;
import org.apache.nifi.provenance.expiration.ExpirationAction; 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.RecordReader;
import org.apache.nifi.provenance.serialization.RecordReaders; import org.apache.nifi.provenance.serialization.RecordReaders;
import org.slf4j.Logger; 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(), ".")); final Term term = new Term(FieldNames.STORAGE_FILENAME, LuceneUtil.substringBefore(expiredFile.getName(), "."));
boolean deleteDir = false; boolean deleteDir = false;
final IndexWriter writer = indexManager.borrowIndexWriter(indexingDirectory); final EventIndexWriter writer = indexManager.borrowIndexWriter(indexingDirectory);
try { try {
writer.deleteDocuments(term); final IndexWriter indexWriter = writer.getIndexWriter();
writer.commit(); indexWriter.deleteDocuments(term);
final int docsLeft = writer.numDocs(); indexWriter.commit();
final int docsLeft = indexWriter.numDocs();
deleteDir = docsLeft <= 0; deleteDir = docsLeft <= 0;
logger.debug("After expiring {}, there are {} docs left for index {}", expiredFile, docsLeft, indexingDirectory); logger.debug("After expiring {}, there are {} docs left for index {}", expiredFile, docsLeft, indexingDirectory);
} finally { } finally {
indexManager.returnIndexWriter(indexingDirectory, writer); indexManager.returnIndexWriter(writer);
} }
// we've confirmed that all documents have been removed. Delete the index directory. // we've confirmed that all documents have been removed. Delete the index directory.

View File

@ -30,25 +30,25 @@ import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; 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.document.Document;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
class DocsReader { public class DocsReader {
private final Logger logger = LoggerFactory.getLogger(DocsReader.class); private final Logger logger = LoggerFactory.getLogger(DocsReader.class);
public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final AuthorizationCheck authCheck, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles, public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final EventAuthorizer authorizer, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles,
final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException { final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
if (retrievalCount.get() >= maxResults) { if (retrievalCount.get() >= maxResults) {
return Collections.emptySet(); return Collections.emptySet();
@ -67,7 +67,7 @@ class DocsReader {
final long readDocuments = System.nanoTime() - start; final long readDocuments = System.nanoTime() - start;
logger.debug("Reading {} Lucene Documents took {} millis", docs.size(), TimeUnit.NANOSECONDS.toMillis(readDocuments)); 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; return record;
} }
public Set<ProvenanceEventRecord> read(final List<Document> docs, final AuthorizationCheck authCheck, final Collection<Path> allProvenanceLogFiles, public Set<ProvenanceEventRecord> read(final List<Document> docs, final EventAuthorizer authorizer, final Collection<Path> allProvenanceLogFiles,
final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException { final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
if (retrievalCount.get() >= maxResults) { if (retrievalCount.get() >= maxResults) {
@ -114,38 +114,33 @@ class DocsReader {
} }
final long start = System.nanoTime(); final long start = System.nanoTime();
final Set<ProvenanceEventRecord> matchingRecords = new LinkedHashSet<>();
Set<ProvenanceEventRecord> matchingRecords = new LinkedHashSet<>(); final Map<String, List<Document>> byStorageNameDocGroups = LuceneUtil.groupDocsByStorageFileName(docs);
Map<String, List<Document>> byStorageNameDocGroups = LuceneUtil.groupDocsByStorageFileName(docs);
int eventsReadThisFile = 0; int eventsReadThisFile = 0;
int logFileCount = 0; int logFileCount = 0;
for (String storageFileName : byStorageNameDocGroups.keySet()) { for (String storageFileName : byStorageNameDocGroups.keySet()) {
File provenanceEventFile = LuceneUtil.getProvenanceLogFile(storageFileName, allProvenanceLogFiles); final File provenanceEventFile = LuceneUtil.getProvenanceLogFile(storageFileName, allProvenanceLogFiles);
if (provenanceEventFile != null) { if (provenanceEventFile == null) {
try (RecordReader reader = RecordReaders.newRecordReader(provenanceEventFile, allProvenanceLogFiles, logger.warn("Could not find Provenance Log File with "
maxAttributeChars)) { + "basename {} in the Provenance Repository; assuming "
+ "file has expired and continuing without it", storageFileName);
continue;
}
Iterator<Document> docIter = byStorageNameDocGroups.get(storageFileName).iterator(); try (final RecordReader reader = RecordReaders.newRecordReader(provenanceEventFile, allProvenanceLogFiles, maxAttributeChars)) {
final Iterator<Document> docIter = byStorageNameDocGroups.get(storageFileName).iterator();
while (docIter.hasNext() && retrievalCount.getAndIncrement() < maxResults) { while (docIter.hasNext() && retrievalCount.getAndIncrement() < maxResults) {
ProvenanceEventRecord event = this.getRecord(docIter.next(), reader); final ProvenanceEventRecord event = getRecord(docIter.next(), reader);
if (event != null && authCheck.isAuthorized(event)) { if (event != null && authorizer.isAuthorized(event)) {
matchingRecords.add(event); matchingRecords.add(event);
eventsReadThisFile++; eventsReadThisFile++;
} }
} }
} catch (final Exception e) {
} catch (Exception e) { logger.warn("Failed to read Provenance Events. The event file '"
logger.warn("Failed while trying to read Provenance Events. The event file '" + provenanceEventFile.getAbsolutePath() + "' may be missing or corrupt.", e);
+ provenanceEventFile.getAbsolutePath() +
"' may be missing or corrupted.", e);
}
} else {
logger.warn("Could not find Provenance Log File with "
+ "basename {} in the Provenance Repository; assuming "
+ "file has expired and continuing without it", storageFileName);
} }
} }

View File

@ -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<ProvenanceEventRecord> convert(TopDocs topDocs, IndexReader indexReader) throws IOException;
}

View File

@ -21,17 +21,19 @@ import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import org.apache.lucene.index.IndexWriter; import org.apache.nifi.provenance.index.EventIndexSearcher;
import org.apache.lucene.search.IndexSearcher; import org.apache.nifi.provenance.index.EventIndexWriter;
public interface IndexManager extends Closeable { 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);
} }

View File

@ -21,18 +21,20 @@ import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Date; import java.util.Date;
import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.nifi.authorization.AccessDeniedException;
import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.provenance.PersistentProvenanceRepository; import org.apache.nifi.provenance.PersistentProvenanceRepository;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.StandardQueryResult; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -86,14 +88,14 @@ public class IndexSearch {
final Query luceneQuery = LuceneUtil.convertQuery(provenanceQuery); final Query luceneQuery = LuceneUtil.convertQuery(provenanceQuery);
final long start = System.nanoTime(); final long start = System.nanoTime();
IndexSearcher searcher = null; EventIndexSearcher searcher = null;
try { try {
searcher = indexManager.borrowIndexSearcher(indexDirectory); searcher = indexManager.borrowIndexSearcher(indexDirectory);
final long searchStartNanos = System.nanoTime(); final long searchStartNanos = System.nanoTime();
final long openSearcherNanos = searchStartNanos - start; final long openSearcherNanos = searchStartNanos - start;
logger.debug("Searching {} for {}", this, provenanceQuery); 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 finishSearch = System.nanoTime();
final long searchNanos = finishSearch - searchStartNanos; final long searchNanos = finishSearch - searchStartNanos;
@ -107,9 +109,29 @@ public class IndexSearch {
final DocsReader docsReader = new DocsReader(); 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<ProvenanceEventRecord> filterUnauthorizedEvents(List<ProvenanceEventRecord> events) {
return repository.filterUnauthorizedEvents(events, user);
}
@Override
public Set<ProvenanceEventRecord> replaceUnauthorizedWithPlaceholders(Set<ProvenanceEventRecord> events) {
return repository.replaceUnauthorizedWithPlaceholders(events, user);
}
};
matchingRecords = docsReader.read(topDocs, authorizer, searcher.getIndexSearcher().getIndexReader(), repository.getAllLogFiles(), retrievedCount,
provenanceQuery.getMaxResults(), maxAttributeChars); provenanceQuery.getMaxResults(), maxAttributeChars);
final long readRecordsNanos = System.nanoTime() - finishSearch; final long readRecordsNanos = System.nanoTime() - finishSearch;
@ -133,7 +155,7 @@ public class IndexSearch {
return sqr; return sqr;
} finally { } finally {
if ( searcher != null ) { if ( searcher != null ) {
indexManager.returnIndexSearcher(indexDirectory, searcher); indexManager.returnIndexSearcher(searcher);
} }
} }
} }

View File

@ -19,6 +19,7 @@ package org.apache.nifi.provenance.lucene;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.List;
import java.util.Set; import java.util.Set;
import org.apache.lucene.document.Document; 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.document.StringField;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.provenance.PersistentProvenanceRepository;
import org.apache.nifi.provenance.ProvenanceEventType; import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.SearchableFields; import org.apache.nifi.provenance.SearchableFields;
import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.provenance.search.SearchableField; import org.apache.nifi.provenance.search.SearchableField;
public class IndexingAction { public class IndexingAction {
private final Set<SearchableField> nonAttributeSearchableFields; private final Set<SearchableField> searchableEventFields;
private final Set<SearchableField> attributeSearchableFields; private final Set<SearchableField> searchableAttributeFields;
public IndexingAction(final PersistentProvenanceRepository repo) { public IndexingAction(final List<SearchableField> searchableEventFields, final List<SearchableField> searchableAttributes) {
attributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(repo.getConfiguration().getSearchableAttributes())); this.searchableEventFields = Collections.unmodifiableSet(new HashSet<>(searchableEventFields));
nonAttributeSearchableFields = Collections.unmodifiableSet(new HashSet<>(repo.getConfiguration().getSearchableFields())); this.searchableAttributeFields = Collections.unmodifiableSet(new HashSet<>(searchableAttributes));
} }
private void addField(final Document doc, final SearchableField field, final String value, final Store store) { 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; return;
} }
@ -67,7 +67,7 @@ public class IndexingAction {
addField(doc, SearchableFields.SourceQueueIdentifier, record.getSourceQueueIdentifier(), Store.NO); addField(doc, SearchableFields.SourceQueueIdentifier, record.getSourceQueueIdentifier(), Store.NO);
addField(doc, SearchableFields.TransitURI, record.getTransitUri(), 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); addField(doc, searchableField, LuceneUtil.truncateIndexField(record.getAttribute(searchableField.getSearchableFieldName())), Store.NO);
} }

View File

@ -25,18 +25,15 @@ import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.nifi.provenance.PersistentProvenanceRepository;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.SearchableFields; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -46,8 +43,8 @@ public class LineageQuery {
public static final int MAX_LINEAGE_UUIDS = 100; public static final int MAX_LINEAGE_UUIDS = 100;
private static final Logger logger = LoggerFactory.getLogger(LineageQuery.class); private static final Logger logger = LoggerFactory.getLogger(LineageQuery.class);
public static Set<ProvenanceEventRecord> computeLineageForFlowFiles(final PersistentProvenanceRepository repo, final IndexManager indexManager, final File indexDirectory, public static Set<ProvenanceEventRecord> computeLineageForFlowFiles(final IndexManager indexManager, final File indexDirectory,
final String lineageIdentifier, final Collection<String> flowFileUuids, final int maxAttributeChars) throws IOException { final String lineageIdentifier, final Collection<String> flowFileUuids, final DocumentToEventConverter docsToEventConverter) throws IOException {
if (requireNonNull(flowFileUuids).size() > MAX_LINEAGE_UUIDS) { 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())); 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"); throw new IllegalArgumentException("Must specify either Lineage Identifier or FlowFile UUIDs to compute lineage");
} }
final IndexSearcher searcher; final EventIndexSearcher searcher;
try { try {
searcher = indexManager.borrowIndexSearcher(indexDirectory); searcher = indexManager.borrowIndexSearcher(indexDirectory);
try { try {
@ -75,16 +72,10 @@ public class LineageQuery {
final long searchStart = System.nanoTime(); final long searchStart = System.nanoTime();
logger.debug("Searching {} for {}", indexDirectory, flowFileIdQuery); 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(); final long searchEnd = System.nanoTime();
// Always authorized. We do this because we need to pull back the event, regardless of whether or not final Set<ProvenanceEventRecord> recs = docsToEventConverter.convert(uuidQueryTopDocs, searcher.getIndexSearcher().getIndexReader());
// 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<ProvenanceEventRecord> recs = docsReader.read(uuidQueryTopDocs, authCheck, searcher.getIndexReader(), repo.getAllLogFiles(),
new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars);
final long readDocsEnd = System.nanoTime(); final long readDocsEnd = System.nanoTime();
logger.debug("Finished Lineage Query against {}; Lucene search took {} millis, reading records took {} millis", logger.debug("Finished Lineage Query against {}; Lucene search took {} millis, reading records took {} millis",
@ -92,7 +83,7 @@ public class LineageQuery {
return recs; return recs;
} finally { } finally {
indexManager.returnIndexSearcher(indexDirectory, searcher); indexManager.returnIndexSearcher(searcher);
} }
} catch (final FileNotFoundException fnfe) { } catch (final FileNotFoundException fnfe) {
// nothing has been indexed yet, or the data has already aged off // nothing has been indexed yet, or the data has already aged off

View File

@ -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);
}
}
}

View File

@ -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> 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<Document> 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;
}
}
}

View File

@ -24,90 +24,164 @@ import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory; 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
public class SimpleIndexManager implements IndexManager { public class SimpleIndexManager implements IndexManager {
private static final Logger logger = LoggerFactory.getLogger(SimpleIndexManager.class); private static final Logger logger = LoggerFactory.getLogger(SimpleIndexManager.class);
private final ConcurrentMap<Object, List<Closeable>> closeables = new ConcurrentHashMap<>(); private final Map<File, IndexWriterCount> writerCounts = new HashMap<>(); // guarded by synchronizing on map itself
private final Map<File, IndexWriterCount> writerCounts = new HashMap<>(); private final ExecutorService searchExecutor;
private final RepositoryConfiguration repoConfig;
private final ExecutorService searchExecutor = Executors.newCachedThreadPool();
public SimpleIndexManager(final RepositoryConfiguration repoConfig) {
this.repoConfig = repoConfig;
this.searchExecutor = Executors.newFixedThreadPool(repoConfig.getQueryThreadPoolSize(), new NamedThreadFactory("Search Lucene Index"));
}
@Override @Override
public void close() throws IOException { public void close() throws IOException {
logger.debug("Shutting down SimpleIndexManager search executor"); logger.debug("Shutting down SimpleIndexManager search executor");
this.searchExecutor.shutdown();
searchExecutor.shutdown();
try { try {
if (!this.searchExecutor.awaitTermination(5, TimeUnit.SECONDS)) { if (!searchExecutor.awaitTermination(5, TimeUnit.SECONDS)) {
this.searchExecutor.shutdownNow(); searchExecutor.shutdownNow();
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
this.searchExecutor.shutdownNow(); searchExecutor.shutdownNow();
} }
} }
@Override @Override
public IndexSearcher borrowIndexSearcher(final File indexDir) throws IOException { public EventIndexSearcher borrowIndexSearcher(final File indexDir) throws IOException {
logger.debug("Creating index searcher for {}", indexDir); 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); final Directory directory = FSDirectory.open(indexDir);
final DirectoryReader directoryReader = DirectoryReader.open(directory); 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 IndexSearcher searcher = new IndexSearcher(directoryReader, this.searchExecutor);
final List<Closeable> closeableList = new ArrayList<>(2); logger.trace("Created index searcher {} for {}", searcher, indexDir);
closeableList.add(directoryReader); return new LuceneEventIndexSearcher(searcher, indexDir, null, directoryReader);
closeableList.add(directory);
closeables.put(searcher, closeableList);
logger.debug("Created index searcher {} for {}", searcher, indexDir);
return searcher;
} }
@Override @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); logger.debug("Closing index searcher {} for {}", searcher, indexDirectory);
closeQuietly(searcher);
final List<Closeable> closeableList = closeables.get(searcher);
if (closeableList != null) {
for (final Closeable closeable : closeableList) {
closeQuietly(closeable);
}
}
logger.debug("Closed index searcher {}", searcher); logger.debug("Closed index searcher {}", searcher);
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()));
}
}
if (closeWriter) {
try {
close(count);
} catch (final Exception e) {
logger.warn("Failed to close Index Writer {} due to {}", count.getWriter(), e.toString(), e);
}
}
} }
@Override @Override
public void removeIndex(final File indexDirectory) { public boolean removeIndex(final File indexDirectory) {
} final File absoluteFile = indexDirectory.getAbsoluteFile();
logger.debug("Attempting to remove index {} from SimpleIndexManager", absoluteFile);
IndexWriterCount writerCount;
@Override synchronized (writerCounts) {
public synchronized IndexWriter borrowIndexWriter(final File indexingDirectory) throws IOException { writerCount = writerCounts.remove(absoluteFile);
final File absoluteFile = indexingDirectory.getAbsoluteFile();
logger.trace("Borrowing index writer for {}", indexingDirectory);
IndexWriterCount writerCount = writerCounts.remove(absoluteFile);
if (writerCount == null) { 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
}
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<Closeable> closeables = new ArrayList<>(); final List<Closeable> closeables = new ArrayList<>();
final Directory directory = FSDirectory.open(indexingDirectory); final Directory directory = FSDirectory.open(indexDirectory);
closeables.add(directory); closeables.add(directory);
try { try {
@ -115,11 +189,18 @@ public class SimpleIndexManager implements IndexManager {
closeables.add(analyzer); closeables.add(analyzer);
final IndexWriterConfig config = new IndexWriterConfig(LuceneUtil.LUCENE_VERSION, analyzer); final IndexWriterConfig config = new IndexWriterConfig(LuceneUtil.LUCENE_VERSION, analyzer);
config.setWriteLockTimeout(300000L);
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 IndexWriter indexWriter = new IndexWriter(directory, config);
writerCount = new IndexWriterCount(indexWriter, analyzer, directory, 1); final EventIndexWriter eventIndexWriter = new LuceneEventIndexWriter(indexWriter, indexDirectory);
logger.debug("Providing new index writer for {}", indexingDirectory);
final IndexWriterCount writerCount = new IndexWriterCount(eventIndexWriter, analyzer, directory, 1, false);
logger.debug("Providing new index writer for {}", indexDirectory);
return writerCount;
} catch (final IOException ioe) { } catch (final IOException ioe) {
for (final Closeable closeable : closeables) { for (final Closeable closeable : closeables) {
try { try {
@ -131,48 +212,118 @@ public class SimpleIndexManager implements IndexManager {
throw ioe; 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); writerCounts.put(absoluteFile, writerCount);
} else { } else {
logger.debug("Providing existing index writer for {} and incrementing count to {}", indexingDirectory, writerCount.getCount() + 1); logger.trace("Providing existing index writer for {} and incrementing count to {}", indexDirectory, writerCount.getCount() + 1);
writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(), writerCounts.put(absoluteFile, new IndexWriterCount(writerCount.getWriter(),
writerCount.getAnalyzer(), writerCount.getDirectory(), writerCount.getCount() + 1)); 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(); return writerCount.getWriter();
} }
@Override
public void returnIndexWriter(final EventIndexWriter writer) {
returnIndexWriter(writer, true, true);
}
@Override @Override
public synchronized void returnIndexWriter(final File indexingDirectory, final IndexWriter writer) { public void returnIndexWriter(final EventIndexWriter writer, final boolean commit, final boolean isCloseable) {
final File absoluteFile = indexingDirectory.getAbsoluteFile(); final File indexDirectory = writer.getDirectory();
logger.trace("Returning Index Writer for {} to IndexManager", indexingDirectory); final File absoluteFile = indexDirectory.getAbsoluteFile();
logger.trace("Returning Index Writer for {} to IndexManager", indexDirectory);
final IndexWriterCount count = writerCounts.remove(absoluteFile);
boolean unused = false;
IndexWriterCount count = null;
boolean close = isCloseable;
try { try {
synchronized (writerCounts) {
count = writerCounts.get(absoluteFile);
if (count != null && count.isCloseableWhenUnused()) {
close = true;
}
if (count == null) { if (count == null) {
logger.warn("Index Writer {} was returned to IndexManager for {}, but this writer is not known. " 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(); writer.close();
} else if (count.getCount() <= 1) { } else if (count.getCount() <= 1) {
// we are finished with this writer. // we are finished with this writer.
logger.debug("Decrementing count for Index Writer for {} to {}; Closing writer", indexingDirectory, count.getCount() - 1); unused = true;
try { if (close) {
writer.commit(); logger.debug("Decrementing count for Index Writer for {} to {}; closing writer", indexDirectory, count.getCount() - 1);
} finally { writerCounts.remove(absoluteFile);
count.close(); } 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 { } else {
// decrement the count. // decrement the count.
logger.debug("Decrementing count for Index Writer for {} to {}", indexingDirectory, count.getCount() - 1); if (close) {
writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(), count.getDirectory(), count.getCount() - 1)); 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);
} }
} catch (final IOException ioe) {
logger.warn("Failed to close Index Writer {} due to {}", writer, ioe); writerCounts.put(absoluteFile, new IndexWriterCount(count.getWriter(), count.getAnalyzer(),
if (logger.isDebugEnabled()) { count.getDirectory(), count.getCount() - 1, close));
logger.warn("", ioe);
} }
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);
}
}
// 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 { protected static class IndexWriterCount implements Closeable {
private final IndexWriter writer; private final EventIndexWriter writer;
private final Analyzer analyzer; private final Analyzer analyzer;
private final Directory directory; private final Directory directory;
private final int count; 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.writer = writer;
this.analyzer = analyzer; this.analyzer = analyzer;
this.directory = directory; this.directory = directory;
this.count = count; this.count = count;
this.closeableWhenUnused = closeableWhenUnused;
}
public boolean isCloseableWhenUnused() {
return closeableWhenUnused;
} }
public Analyzer getAnalyzer() { public Analyzer getAnalyzer() {
@ -212,7 +369,7 @@ public class SimpleIndexManager implements IndexManager {
return directory; return directory;
} }
public IndexWriter getWriter() { public EventIndexWriter getWriter() {
return writer; return writer;
} }
@ -224,5 +381,10 @@ public class SimpleIndexManager implements IndexManager {
public void close() throws IOException { public void close() throws IOException {
closeQuietly(writer, analyzer, directory); closeQuietly(writer, analyzer, directory);
} }
@Override
public String toString() {
return "IndexWriterCount[count=" + count + ", writer=" + writer + ", closeableWhenUnused=" + closeableWhenUnused + "]";
}
} }
} }

View File

@ -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";
}

View File

@ -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<RecordField> 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);
}
}

View File

@ -69,47 +69,47 @@ public class EventRecord implements Record {
@Override @Override
public Object getFieldValue(final String fieldName) { public Object getFieldValue(final String fieldName) {
switch (fieldName) { switch (fieldName) {
case EventRecordFields.Names.EVENT_IDENTIFIER: case EventFieldNames.EVENT_IDENTIFIER:
return eventId; return eventId;
case EventRecordFields.Names.ALTERNATE_IDENTIFIER: case EventFieldNames.ALTERNATE_IDENTIFIER:
return event.getAlternateIdentifierUri(); return event.getAlternateIdentifierUri();
case EventRecordFields.Names.CHILD_UUIDS: case EventFieldNames.CHILD_UUIDS:
return event.getChildUuids(); return event.getChildUuids();
case EventRecordFields.Names.COMPONENT_ID: case EventFieldNames.COMPONENT_ID:
return event.getComponentId(); return event.getComponentId();
case EventRecordFields.Names.COMPONENT_TYPE: case EventFieldNames.COMPONENT_TYPE:
return event.getComponentType(); return event.getComponentType();
case EventRecordFields.Names.CONTENT_CLAIM: case EventFieldNames.CONTENT_CLAIM:
return contentClaimRecord; return contentClaimRecord;
case EventRecordFields.Names.EVENT_DETAILS: case EventFieldNames.EVENT_DETAILS:
return event.getDetails(); return event.getDetails();
case EventRecordFields.Names.EVENT_DURATION: case EventFieldNames.EVENT_DURATION:
return event.getEventDuration(); return event.getEventDuration();
case EventRecordFields.Names.EVENT_TIME: case EventFieldNames.EVENT_TIME:
return event.getEventTime(); return event.getEventTime();
case EventRecordFields.Names.EVENT_TYPE: case EventFieldNames.EVENT_TYPE:
return event.getEventType().name(); return event.getEventType().name();
case EventRecordFields.Names.FLOWFILE_ENTRY_DATE: case EventFieldNames.FLOWFILE_ENTRY_DATE:
return event.getFlowFileEntryDate(); return event.getFlowFileEntryDate();
case EventRecordFields.Names.FLOWFILE_UUID: case EventFieldNames.FLOWFILE_UUID:
return event.getFlowFileUuid(); return event.getFlowFileUuid();
case EventRecordFields.Names.LINEAGE_START_DATE: case EventFieldNames.LINEAGE_START_DATE:
return event.getLineageStartDate(); return event.getLineageStartDate();
case EventRecordFields.Names.PARENT_UUIDS: case EventFieldNames.PARENT_UUIDS:
return event.getParentUuids(); return event.getParentUuids();
case EventRecordFields.Names.PREVIOUS_ATTRIBUTES: case EventFieldNames.PREVIOUS_ATTRIBUTES:
return event.getPreviousAttributes(); return event.getPreviousAttributes();
case EventRecordFields.Names.PREVIOUS_CONTENT_CLAIM: case EventFieldNames.PREVIOUS_CONTENT_CLAIM:
return previousClaimRecord; return previousClaimRecord;
case EventRecordFields.Names.RELATIONSHIP: case EventFieldNames.RELATIONSHIP:
return event.getRelationship(); return event.getRelationship();
case EventRecordFields.Names.SOURCE_QUEUE_IDENTIFIER: case EventFieldNames.SOURCE_QUEUE_IDENTIFIER:
return event.getSourceQueueIdentifier(); return event.getSourceQueueIdentifier();
case EventRecordFields.Names.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER: case EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER:
return event.getSourceSystemFlowFileIdentifier(); return event.getSourceSystemFlowFileIdentifier();
case EventRecordFields.Names.TRANSIT_URI: case EventFieldNames.TRANSIT_URI:
return event.getTransitUri(); return event.getTransitUri();
case EventRecordFields.Names.UPDATED_ATTRIBUTES: case EventFieldNames.UPDATED_ATTRIBUTES:
return event.getUpdatedAttributes(); return event.getUpdatedAttributes();
} }
@ -119,48 +119,52 @@ public class EventRecord implements Record {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public static StandardProvenanceEventRecord getEvent(final Record record, final String storageFilename, final long storageByteOffset, final int maxAttributeLength) { public static StandardProvenanceEventRecord getEvent(final Record record, final String storageFilename, final long storageByteOffset, final int maxAttributeLength) {
final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder(); final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder();
builder.setAlternateIdentifierUri((String) record.getFieldValue(EventRecordFields.Names.ALTERNATE_IDENTIFIER)); builder.setAlternateIdentifierUri((String) record.getFieldValue(EventFieldNames.ALTERNATE_IDENTIFIER));
builder.setChildUuids((List<String>) record.getFieldValue(EventRecordFields.Names.CHILD_UUIDS)); builder.setChildUuids((List<String>) record.getFieldValue(EventFieldNames.CHILD_UUIDS));
builder.setComponentId((String) record.getFieldValue(EventRecordFields.Names.COMPONENT_ID)); builder.setComponentId((String) record.getFieldValue(EventFieldNames.COMPONENT_ID));
builder.setComponentType((String) record.getFieldValue(EventRecordFields.Names.COMPONENT_TYPE)); builder.setComponentType((String) record.getFieldValue(EventFieldNames.COMPONENT_TYPE));
builder.setDetails((String) record.getFieldValue(EventRecordFields.Names.EVENT_DETAILS)); builder.setDetails((String) record.getFieldValue(EventFieldNames.EVENT_DETAILS));
builder.setEventDuration((Long) record.getFieldValue(EventRecordFields.Names.EVENT_DURATION)); builder.setEventDuration((Long) record.getFieldValue(EventFieldNames.EVENT_DURATION));
builder.setEventTime((Long) record.getFieldValue(EventRecordFields.Names.EVENT_TIME)); builder.setEventTime((Long) record.getFieldValue(EventFieldNames.EVENT_TIME));
builder.setEventType(ProvenanceEventType.valueOf((String) record.getFieldValue(EventRecordFields.Names.EVENT_TYPE))); builder.setEventType(ProvenanceEventType.valueOf((String) record.getFieldValue(EventFieldNames.EVENT_TYPE)));
builder.setFlowFileEntryDate((Long) record.getFieldValue(EventRecordFields.Names.FLOWFILE_ENTRY_DATE)); builder.setFlowFileEntryDate((Long) record.getFieldValue(EventFieldNames.FLOWFILE_ENTRY_DATE));
builder.setFlowFileUUID((String) record.getFieldValue(EventRecordFields.Names.FLOWFILE_UUID)); builder.setFlowFileUUID((String) record.getFieldValue(EventFieldNames.FLOWFILE_UUID));
builder.setLineageStartDate((Long) record.getFieldValue(EventRecordFields.Names.LINEAGE_START_DATE)); builder.setLineageStartDate((Long) record.getFieldValue(EventFieldNames.LINEAGE_START_DATE));
builder.setParentUuids((List<String>) record.getFieldValue(EventRecordFields.Names.PARENT_UUIDS)); builder.setParentUuids((List<String>) record.getFieldValue(EventFieldNames.PARENT_UUIDS));
builder.setPreviousAttributes(truncateAttributes((Map<String, String>) record.getFieldValue(EventRecordFields.Names.PREVIOUS_ATTRIBUTES), maxAttributeLength)); builder.setPreviousAttributes(truncateAttributes((Map<String, String>) record.getFieldValue(EventFieldNames.PREVIOUS_ATTRIBUTES), maxAttributeLength));
builder.setEventId((Long) record.getFieldValue(EventRecordFields.Names.EVENT_IDENTIFIER)); builder.setRelationship((String) record.getFieldValue(EventFieldNames.RELATIONSHIP));
builder.setRelationship((String) record.getFieldValue(EventRecordFields.Names.RELATIONSHIP)); builder.setSourceQueueIdentifier((String) record.getFieldValue(EventFieldNames.SOURCE_QUEUE_IDENTIFIER));
builder.setSourceQueueIdentifier((String) record.getFieldValue(EventRecordFields.Names.SOURCE_QUEUE_IDENTIFIER)); builder.setSourceSystemFlowFileIdentifier((String) record.getFieldValue(EventFieldNames.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER));
builder.setSourceSystemFlowFileIdentifier((String) record.getFieldValue(EventRecordFields.Names.SOURCE_SYSTEM_FLOWFILE_IDENTIFIER)); builder.setTransitUri((String) record.getFieldValue(EventFieldNames.TRANSIT_URI));
builder.setTransitUri((String) record.getFieldValue(EventRecordFields.Names.TRANSIT_URI)); builder.setUpdatedAttributes(truncateAttributes((Map<String, String>) record.getFieldValue(EventFieldNames.UPDATED_ATTRIBUTES), maxAttributeLength));
builder.setUpdatedAttributes(truncateAttributes((Map<String, String>) record.getFieldValue(EventRecordFields.Names.UPDATED_ATTRIBUTES), maxAttributeLength));
final Long eventId = (Long) record.getFieldValue(EventFieldNames.EVENT_IDENTIFIER);
if (eventId != null) {
builder.setEventId(eventId);
}
builder.setStorageLocation(storageFilename, storageByteOffset); 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) { if (currentClaimRecord == null) {
builder.setCurrentContentClaim(null, null, null, null, 0L); builder.setCurrentContentClaim(null, null, null, null, 0L);
} else { } else {
builder.setCurrentContentClaim( builder.setCurrentContentClaim(
(String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_CONTAINER), (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_CONTAINER),
(String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SECTION), (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SECTION),
(String) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_IDENTIFIER), (String) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_IDENTIFIER),
(Long) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_OFFSET), (Long) currentClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_OFFSET),
(Long) currentClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SIZE)); (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) { if (previousClaimRecord != null) {
builder.setPreviousContentClaim( builder.setPreviousContentClaim(
(String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_CONTAINER), (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_CONTAINER),
(String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SECTION), (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SECTION),
(String) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_IDENTIFIER), (String) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_IDENTIFIER),
(Long) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_OFFSET), (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_OFFSET),
(Long) previousClaimRecord.getFieldValue(EventRecordFields.Names.CONTENT_CLAIM_SIZE)); (Long) previousClaimRecord.getFieldValue(EventFieldNames.CONTENT_CLAIM_SIZE));
} }
return builder.build(); return builder.build();

View File

@ -29,82 +29,48 @@ import org.apache.nifi.repository.schema.SimpleRecordField;
public class EventRecordFields { 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. // General Event fields.
public static final RecordField RECORD_IDENTIFIER = new SimpleRecordField(Names.EVENT_IDENTIFIER, FieldType.LONG, EXACTLY_ONE); public static final RecordField RECORD_IDENTIFIER = new SimpleRecordField(EventFieldNames.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_TYPE = new SimpleRecordField(EventFieldNames.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 EVENT_TIME = new SimpleRecordField(EventFieldNames.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 FLOWFILE_ENTRY_DATE = new SimpleRecordField(EventFieldNames.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 EVENT_DURATION = new SimpleRecordField(EventFieldNames.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 LINEAGE_START_DATE = new SimpleRecordField(EventFieldNames.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_ID = new SimpleRecordField(EventFieldNames.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 COMPONENT_TYPE = new SimpleRecordField(EventFieldNames.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 FLOWFILE_UUID = new SimpleRecordField(EventFieldNames.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 EVENT_DETAILS = new SimpleRecordField(EventFieldNames.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 SOURCE_QUEUE_IDENTIFIER = new SimpleRecordField(EventFieldNames.SOURCE_QUEUE_IDENTIFIER, FieldType.STRING, ZERO_OR_ONE);
// Attributes // Attributes
public static final RecordField ATTRIBUTE_NAME = new SimpleRecordField(Names.ATTRIBUTE_NAME, FieldType.LONG_STRING, EXACTLY_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(Names.ATTRIBUTE_VALUE, 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(Names.ATTRIBUTE_VALUE, FieldType.LONG_STRING, ZERO_OR_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 PREVIOUS_ATTRIBUTES = new MapRecordField(EventFieldNames.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 UPDATED_ATTRIBUTES = new MapRecordField(EventFieldNames.UPDATED_ATTRIBUTES, ATTRIBUTE_NAME, ATTRIBUTE_VALUE_OPTIONAL, EXACTLY_ONE);
// Content Claims // 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_CONTAINER = new SimpleRecordField(EventFieldNames.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_SECTION = new SimpleRecordField(EventFieldNames.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_IDENTIFIER = new SimpleRecordField(EventFieldNames.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_OFFSET = new SimpleRecordField(EventFieldNames.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 CONTENT_CLAIM_SIZE = new SimpleRecordField(EventFieldNames.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 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); 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); CONTENT_CLAIM_CONTAINER, CONTENT_CLAIM_SECTION, CONTENT_CLAIM_IDENTIFIER, CONTENT_CLAIM_OFFSET, CONTENT_CLAIM_SIZE);
// EventType-Specific fields // EventType-Specific fields
// for FORK, JOIN, CLONE, REPLAY // 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 PARENT_UUIDS = new SimpleRecordField(EventFieldNames.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 CHILD_UUIDS = new SimpleRecordField(EventFieldNames.CHILD_UUIDS, FieldType.STRING, ZERO_OR_MORE);
// for SEND/RECEIVE/FETCH // for SEND/RECEIVE/FETCH
public static final RecordField TRANSIT_URI = new SimpleRecordField(Names.TRANSIT_URI, 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(Names.SOURCE_QUEUE_IDENTIFIER, 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 // for ADD_INFO
public static final RecordField ALTERNATE_IDENTIFIER = new SimpleRecordField(Names.ALTERNATE_IDENTIFIER, 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(Names.RELATIONSHIP, FieldType.STRING, ZERO_OR_ONE); public static final RecordField RELATIONSHIP = new SimpleRecordField(EventFieldNames.RELATIONSHIP, FieldType.STRING, ZERO_OR_ONE);
} }

View File

@ -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<String, Integer> componentIdMap;
private final Map<String, Integer> componentTypeMap;
private final Map<String, Integer> queueIdMap;
private final Map<String, Integer> 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<String, Integer> componentIdMap,
final Map<String, Integer> componentTypeMap, final Map<String, Integer> queueIdMap, final Map<String, Integer> 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<RecordField, Object> 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<RecordField, Object> lookupValues = Collections.singletonMap(LookupTableEventRecordFields.NO_VALUE, EventFieldNames.NO_VALUE);
final List<RecordField> noValueFields = Collections.singletonList(contentClaimSchema.getField(EventFieldNames.NO_VALUE));
return new FieldMapRecord(lookupValues, new RecordSchema(noValueFields));
}
final Map<RecordField, Object> 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<RecordField> 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<String> 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<String, Integer> lookup) {
if (literalValue == null) {
final Map<RecordField, Object> 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<RecordField, Object> 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<RecordField, Object> 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<Record> recordSupplier) {
if (newValue == null || EventFieldNames.NO_VALUE.equals(newValue.getSchema().getFields().get(0).getFieldName())) {
final Map<RecordField, Object> 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<RecordField, Object> 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<String> componentIds, final List<String> componentTypes,
final List<String> queueIds, final List<String> eventTypes) {
final Map<String, String> previousAttributes = truncateAttributes((Map<String, String>) record.getFieldValue(EventFieldNames.PREVIOUS_ATTRIBUTES), maxAttributeLength);
final Map<String, String> updatedAttributes = truncateAttributes((Map<String, String>) record.getFieldValue(EventFieldNames.UPDATED_ATTRIBUTES), maxAttributeLength);
final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder();
builder.setAlternateIdentifierUri((String) record.getFieldValue(EventFieldNames.ALTERNATE_IDENTIFIER));
builder.setChildUuids((List<String>) record.getFieldValue(EventFieldNames.CHILD_UUIDS));
builder.setDetails((String) record.getFieldValue(EventFieldNames.EVENT_DETAILS));
builder.setParentUuids((List<String>) 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<String, String> truncateAttributes(final Map<String, String> 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<String, String> truncated = new HashMap<>();
for (final Map.Entry<String, String> 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;
}
}

View File

@ -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);
}

View File

@ -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<RecordField> 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;
}
}

View File

@ -46,11 +46,15 @@ import org.apache.nifi.repository.schema.RecordField;
import org.apache.nifi.repository.schema.RecordSchema; import org.apache.nifi.repository.schema.RecordSchema;
public class ProvenanceEventSchema { 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<RecordField> fields = new ArrayList<>(); final List<RecordField> fields = new ArrayList<>();
if (includeEventId) {
fields.add(RECORD_IDENTIFIER); fields.add(RECORD_IDENTIFIER);
}
fields.add(EVENT_TYPE); fields.add(EVENT_TYPE);
fields.add(EVENT_TIME); fields.add(EVENT_TIME);
fields.add(FLOWFILE_ENTRY_DATE); fields.add(FLOWFILE_ENTRY_DATE);

View File

@ -17,18 +17,18 @@
package org.apache.nifi.provenance.serialization; package org.apache.nifi.provenance.serialization;
import java.io.BufferedInputStream;
import java.io.DataInputStream; import java.io.DataInputStream;
import java.io.EOFException; import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.Optional;
import java.util.zip.GZIPInputStream; import java.util.zip.GZIPInputStream;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.apache.nifi.provenance.StandardProvenanceEventRecord;
import org.apache.nifi.provenance.StandardRecordReader;
import org.apache.nifi.provenance.toc.TocReader; 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.ByteCountingInputStream;
import org.apache.nifi.stream.io.LimitingInputStream; import org.apache.nifi.stream.io.LimitingInputStream;
import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.stream.io.StreamUtils;
@ -36,7 +36,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
public abstract class CompressableRecordReader implements RecordReader { 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 ByteCountingInputStream rawInputStream;
private final String filename; private final String filename;
@ -48,6 +48,7 @@ public abstract class CompressableRecordReader implements RecordReader {
private DataInputStream dis; private DataInputStream dis;
private ByteCountingInputStream byteCountingIn; private ByteCountingInputStream byteCountingIn;
private StandardProvenanceEventRecord pushbackEvent = null;
public CompressableRecordReader(final InputStream in, final String filename, final int maxAttributeChars) throws IOException { public CompressableRecordReader(final InputStream in, final String filename, final int maxAttributeChars) throws IOException {
this(in, filename, null, maxAttributeChars); this(in, filename, null, maxAttributeChars);
@ -120,6 +121,8 @@ public abstract class CompressableRecordReader implements RecordReader {
try { try {
StreamUtils.skip(rawInputStream, bytesToSkip); StreamUtils.skip(rawInputStream, bytesToSkip);
logger.debug("Skipped stream from offset {} to {} ({} bytes skipped)", curOffset, offset, 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) { } catch (final IOException e) {
throw new IOException("Failed to skip to offset " + offset + " for block " + blockIndex + " of Provenance Log " + filename, e); throw new IOException("Failed to skip to offset " + offset + " for block " + blockIndex + " of Provenance Log " + filename, e);
} }
@ -177,7 +180,9 @@ public abstract class CompressableRecordReader implements RecordReader {
return byteCountingIn.getBytesConsumed(); return byteCountingIn.getBytesConsumed();
} }
private boolean isData() throws IOException { @Override
public boolean isData() {
try {
byteCountingIn.mark(1); byteCountingIn.mark(1);
int nextByte = byteCountingIn.read(); int nextByte = byteCountingIn.read();
byteCountingIn.reset(); byteCountingIn.reset();
@ -195,6 +200,9 @@ public abstract class CompressableRecordReader implements RecordReader {
} }
return nextByte >= 0; return nextByte >= 0;
} catch (final IOException ioe) {
return false;
}
} }
@Override @Override
@ -268,6 +276,12 @@ public abstract class CompressableRecordReader implements RecordReader {
@Override @Override
public StandardProvenanceEventRecord nextRecord() throws IOException { public StandardProvenanceEventRecord nextRecord() throws IOException {
if (pushbackEvent != null) {
final StandardProvenanceEventRecord toReturn = pushbackEvent;
pushbackEvent = null;
return toReturn;
}
if (isData()) { if (isData()) {
return nextRecord(dis, serializationVersion); return nextRecord(dis, serializationVersion);
} else { } else {
@ -275,6 +289,65 @@ public abstract class CompressableRecordReader implements RecordReader {
} }
} }
protected Optional<Integer> 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<ProvenanceEventRecord> 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<Integer> 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<StandardProvenanceEventRecord> 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<StandardProvenanceEventRecord> 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 abstract StandardProvenanceEventRecord nextRecord(DataInputStream in, int serializationVersion) throws IOException;
protected void readHeader(DataInputStream in, int serializationVersion) throws IOException { protected void readHeader(DataInputStream in, int serializationVersion) throws IOException {

View File

@ -17,17 +17,18 @@
package org.apache.nifi.provenance.serialization; package org.apache.nifi.provenance.serialization;
import java.io.BufferedOutputStream;
import java.io.DataOutputStream;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.provenance.AbstractRecordWriter; import org.apache.nifi.provenance.AbstractRecordWriter;
import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.toc.TocWriter; 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.ByteCountingOutputStream;
import org.apache.nifi.stream.io.DataOutputStream;
import org.apache.nifi.stream.io.GZIPOutputStream; import org.apache.nifi.stream.io.GZIPOutputStream;
import org.apache.nifi.stream.io.NonCloseableOutputStream; import org.apache.nifi.stream.io.NonCloseableOutputStream;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -40,14 +41,16 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter {
private final ByteCountingOutputStream rawOutStream; private final ByteCountingOutputStream rawOutStream;
private final boolean compressed; private final boolean compressed;
private final int uncompressedBlockSize; private final int uncompressedBlockSize;
private final AtomicLong idGenerator;
private DataOutputStream out; private DataOutputStream out;
private ByteCountingOutputStream byteCountingOut; private ByteCountingOutputStream byteCountingOut;
private long lastBlockOffset = 0L; private long blockStartOffset = 0L;
private int recordCount = 0; 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); super(file, writer);
logger.trace("Creating Record Writer for {}", file.getName()); logger.trace("Creating Record Writer for {}", file.getName());
@ -55,18 +58,25 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter {
this.fos = new FileOutputStream(file); this.fos = new FileOutputStream(file);
rawOutStream = new ByteCountingOutputStream(fos); rawOutStream = new ByteCountingOutputStream(fos);
this.uncompressedBlockSize = uncompressedBlockSize; this.uncompressedBlockSize = uncompressedBlockSize;
this.idGenerator = idGenerator;
} }
public CompressableRecordWriter(final OutputStream out, final TocWriter tocWriter, final boolean compressed, final int uncompressedBlockSize) throws IOException { public CompressableRecordWriter(final OutputStream out, final String storageLocation, final AtomicLong idGenerator, final TocWriter tocWriter, final boolean compressed,
super(null, tocWriter); final int uncompressedBlockSize) throws IOException {
super(storageLocation, tocWriter);
this.fos = null; this.fos = null;
this.compressed = compressed; this.compressed = compressed;
this.uncompressedBlockSize = uncompressedBlockSize; this.uncompressedBlockSize = uncompressedBlockSize;
this.rawOutStream = new ByteCountingOutputStream(out); this.rawOutStream = new ByteCountingOutputStream(out);
this.idGenerator = idGenerator;
} }
protected AtomicLong getIdGenerator() {
return idGenerator;
}
@Override @Override
public synchronized void writeHeader(final long firstEventId) throws IOException { public synchronized void writeHeader(final long firstEventId) throws IOException {
if (isDirty()) { if (isDirty()) {
@ -74,13 +84,13 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter {
} }
try { try {
lastBlockOffset = rawOutStream.getBytesWritten(); blockStartOffset = rawOutStream.getBytesWritten();
resetWriteStream(firstEventId); resetWriteStream(firstEventId);
out.writeUTF(getSerializationName()); out.writeUTF(getSerializationName());
out.writeInt(getSerializationVersion()); out.writeInt(getSerializationVersion());
writeHeader(firstEventId, out); writeHeader(firstEventId, out);
out.flush(); out.flush();
lastBlockOffset = rawOutStream.getBytesWritten(); blockStartOffset = getBytesWritten();
} catch (final IOException ioe) { } catch (final IOException ioe) {
markDirty(); markDirty();
throw ioe; 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 * @param eventId the first id that will be written to the new block
* @throws IOException if unable to flush/close the current streams properly * @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 { try {
if (out != null) { if (out != null) {
out.flush(); out.flush();
@ -114,13 +124,13 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter {
out.close(); out.close();
} }
if (tocWriter != null) { if (tocWriter != null && eventId != null) {
tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId); tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId);
} }
writableStream = new BufferedOutputStream(new GZIPOutputStream(new NonCloseableOutputStream(rawOutStream), 1), 65536); writableStream = new BufferedOutputStream(new GZIPOutputStream(new NonCloseableOutputStream(rawOutStream), 1), 65536);
} else { } else {
if (tocWriter != null) { if (tocWriter != null && eventId != null) {
tocWriter.addBlockOffset(rawOutStream.getBytesWritten(), eventId); 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 @Override
public long writeRecord(final ProvenanceEventRecord record, final long recordIdentifier) throws IOException { public synchronized StorageSummary writeRecord(final ProvenanceEventRecord record) throws IOException {
if (isDirty()) { if (isDirty()) {
throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository"); throw new IOException("Cannot update Provenance Repository because this Record Writer has already failed to write to the Repository");
} }
try { try {
final long recordIdentifier = record.getEventId() == -1L ? idGenerator.getAndIncrement() : record.getEventId();
final long startBytes = byteCountingOut.getBytesWritten(); final long startBytes = byteCountingOut.getBytesWritten();
// add a new block to the TOC if needed. ensureStreamState(recordIdentifier, startBytes);
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);
}
}
writeRecord(record, recordIdentifier, out); writeRecord(record, recordIdentifier, out);
recordCount++; 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) { } catch (final IOException ioe) {
markDirty(); markDirty();
throw ioe; throw ioe;
@ -170,7 +181,12 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter {
} }
@Override @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(); out.flush();
} }
@ -180,22 +196,26 @@ public abstract class CompressableRecordWriter extends AbstractRecordWriter {
} }
@Override @Override
protected OutputStream getBufferedOutputStream() { protected synchronized DataOutputStream getBufferedOutputStream() {
return out; return out;
} }
@Override @Override
protected OutputStream getUnderlyingOutputStream() { protected synchronized OutputStream getUnderlyingOutputStream() {
return fos; return fos;
} }
@Override @Override
protected void syncUnderlyingOutputStream() throws IOException { protected synchronized void syncUnderlyingOutputStream() throws IOException {
if (fos != null) { if (fos != null) {
fos.getFD().sync(); 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 writeRecord(final ProvenanceEventRecord event, final long eventId, final DataOutputStream out) throws IOException;
protected abstract void writeHeader(final long firstEventId, final DataOutputStream out) throws IOException; protected abstract void writeHeader(final long firstEventId, final DataOutputStream out) throws IOException;

View File

@ -18,7 +18,9 @@
package org.apache.nifi.provenance.serialization; package org.apache.nifi.provenance.serialization;
import java.io.IOException; 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.StandardProvenanceEventRecord;
import org.apache.nifi.provenance.toc.TocReader; import org.apache.nifi.provenance.toc.TocReader;
@ -69,4 +71,14 @@ public class EmptyRecordReader implements RecordReader {
public long getMaxEventId() throws IOException { public long getMaxEventId() throws IOException {
return 0; return 0;
} }
@Override
public Optional<ProvenanceEventRecord> skipToEvent(long eventId) throws IOException {
return Optional.empty();
}
@Override
public boolean isData() {
return false;
}
} }

View File

@ -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;
/**
* <p>
* 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}.
* </p>
*/
public class EventFileCompressor implements Runnable {
private static final Logger logger = LoggerFactory.getLogger(EventFileCompressor.class);
private final BlockingQueue<File> filesToCompress;
private final EventFileManager eventFileManager;
private volatile boolean shutdown = false;
public EventFileCompressor(final BlockingQueue<File> 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);
}
}

View File

@ -18,7 +18,9 @@ package org.apache.nifi.provenance.serialization;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; 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.StandardProvenanceEventRecord;
import org.apache.nifi.provenance.toc.TocReader; import org.apache.nifi.provenance.toc.TocReader;
@ -50,13 +52,25 @@ public interface RecordReader extends Closeable {
/** /**
* Skips to the specified compression block * 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 * @throws IOException if the underlying stream throws IOException, or if the reader has already
* read passed the specified compression block index * read passed the specified compression block index
* @throws IllegalStateException if the RecordReader does not have a TableOfContents associated with it * @throws IllegalStateException if the RecordReader does not have a TableOfContents associated with it
*/ */
void skipToBlock(int blockIndex) throws IOException; 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<ProvenanceEventRecord> skipToEvent(long eventId) throws IOException;
/** /**
* Returns the block index that the Reader is currently reading from. * 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()} * 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 * @throws IOException if unable to get id of the last event
*/ */
long getMaxEventId() throws IOException; long getMaxEventId() throws IOException;
/**
* Returns <code>true</code> if there is more data for hte Record Reader to read, <code>false</code> otherwise.
*
* @return <code>true</code> if there is more data for hte Record Reader to read, <code>false</code> otherwise.
*/
boolean isData();
} }

View File

@ -30,8 +30,9 @@ import java.util.zip.GZIPInputStream;
import org.apache.nifi.provenance.ByteArraySchemaRecordReader; import org.apache.nifi.provenance.ByteArraySchemaRecordReader;
import org.apache.nifi.provenance.ByteArraySchemaRecordWriter; 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.StandardRecordReader;
import org.apache.nifi.provenance.StandardRecordWriter;
import org.apache.nifi.provenance.lucene.LuceneUtil; import org.apache.nifi.provenance.lucene.LuceneUtil;
import org.apache.nifi.provenance.toc.StandardTocReader; import org.apache.nifi.provenance.toc.StandardTocReader;
import org.apache.nifi.provenance.toc.TocReader; import org.apache.nifi.provenance.toc.TocReader;
@ -78,10 +79,10 @@ public class RecordReaders {
String filename = file.getName(); String filename = file.getName();
openStream: while ( fis == null ) { openStream: while ( fis == null ) {
final File dir = file.getParentFile(); 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 // 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.indexed.gz" // 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 // 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. // compressing by the time that we are querying the data.
for ( final String extension : new String[] {".prov.gz", ".prov"} ) { for ( final String extension : new String[] {".prov.gz", ".prov"} ) {
@ -123,7 +124,7 @@ public class RecordReaders {
} }
switch (serializationName) { switch (serializationName) {
case StandardRecordWriter.SERIALIZATION_NAME: { case StandardRecordReader.SERIALIZATION_NAME: {
if (tocFile.exists()) { if (tocFile.exists()) {
final TocReader tocReader = new StandardTocReader(tocFile); final TocReader tocReader = new StandardTocReader(tocFile);
return new StandardRecordReader(bufferedInStream, filename, tocReader, maxAttributeChars); return new StandardRecordReader(bufferedInStream, filename, tocReader, maxAttributeChars);
@ -139,6 +140,14 @@ public class RecordReaders {
return new ByteArraySchemaRecordReader(bufferedInStream, filename, maxAttributeChars); 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: { default: {
throw new IOException("Unable to read data from file " + file + " because the file was written using an unknown Serializer: " + serializationName); throw new IOException("Unable to read data from file " + file + " because the file was written using an unknown Serializer: " + serializationName);
} }

View File

@ -37,11 +37,10 @@ public interface RecordWriter extends Closeable {
* Writes the given record out to the underlying stream * Writes the given record out to the underlying stream
* *
* @param record the record to write * @param record the record to write
* @param recordIdentifier the new identifier of the record
* @return the number of bytes written for the given records * @return the number of bytes written for the given records
* @throws IOException if unable to write the record to the stream * @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 * 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(); int getRecordsWritten();
/**
* @return the number of bytes written to this writer
*/
long getBytesWritten();
/** /**
* @return the file that this RecordWriter is writing to * @return the file that this RecordWriter is writing to
*/ */
@ -88,6 +92,11 @@ public interface RecordWriter extends Closeable {
*/ */
void markDirty(); void markDirty();
/**
* @return <code>true</code> if {@link #markDirty()} has been called, <code>false</code> otherwise
*/
boolean isDirty();
/** /**
* Syncs the content written to this writer to disk. * Syncs the content written to this writer to disk.
* *

View File

@ -18,6 +18,7 @@ package org.apache.nifi.provenance.serialization;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.provenance.ByteArraySchemaRecordWriter; import org.apache.nifi.provenance.ByteArraySchemaRecordWriter;
import org.apache.nifi.provenance.toc.StandardTocWriter; import org.apache.nifi.provenance.toc.StandardTocWriter;
@ -27,13 +28,14 @@ import org.apache.nifi.provenance.toc.TocWriter;
public class RecordWriters { public class RecordWriters {
private static final int DEFAULT_COMPRESSION_BLOCK_SIZE = 1024 * 1024; // 1 MB 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 { public static RecordWriter newSchemaRecordWriter(final File file, final AtomicLong idGenerator, final boolean compressed, final boolean createToc) throws IOException {
return newSchemaRecordWriter(file, compressed, createToc, DEFAULT_COMPRESSION_BLOCK_SIZE); 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; 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);
} }
} }

View File

@ -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<String> 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() + "]";
}
}

View File

@ -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<String, Tuple<ReadWriteLock, Integer>> lockMap = new ConcurrentHashMap<>();
private String getMapKey(final File file) {
return LuceneUtil.substringBefore(file.getName(), ".prov");
}
private ReadWriteLock updateCount(final File file, final Function<Integer, Integer> update) {
final String key = getMapKey(file);
boolean updated = false;
Tuple<ReadWriteLock, Integer> updatedTuple = null;
while (!updated) {
final Tuple<ReadWriteLock, Integer> 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<ReadWriteLock, Integer> 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<ReadWriteLock, Integer> updatedTuple = new Tuple<>(tuple.getKey(), tuple.getValue() - 1);
updated = lockMap.replace(key, tuple, updatedTuple);
if (updated) {
tuple.getKey().writeLock().unlock();
}
}
}
}
}

View File

@ -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;
/**
* <p>
* An Event Store is responsible for storing Provenance Events and retrieving them at a later time.
* </p>
*/
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<ProvenanceEventRecord> 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<ProvenanceEventRecord> 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<ProvenanceEventRecord> 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<ProvenanceEventRecord> 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<ProvenanceEventRecord> getEvents(List<Long> 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);
}

Some files were not shown because too many files have changed in this diff Show More