public Lineage computeLineage(final String flowFileUUID, final NiFiUser user) throws IOException { return computeLineage(Collections.singleton(flowFileUUID), user, LineageComputationType.FLOWFILE_LINEAGE, null); }
@Override public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException { return getEvents(firstRecordId, maxRecords, null); }
@Override public ComputeLineageSubmission submitLineageComputation(final long eventId, final NiFiUser user) { final ProvenanceEventRecord event = getEvent(eventId); if (event == null) { final String userId = user == null ? null : user.getIdentity(); final AsyncLineageSubmission result = new AsyncLineageSubmission(LineageComputationType.FLOWFILE_LINEAGE, eventId, Collections.emptySet(), 1, userId); result.getResult().setError("Could not find event with ID " + eventId); lineageSubmissionMap.put(result.getLineageIdentifier(), result); return result; } return submitLineageComputation(Collections.singleton(event.getFlowFileUuid()), user, LineageComputationType.FLOWFILE_LINEAGE, eventId); }
@Override public boolean select(final ProvenanceEventRecord value) { if (!isAuthorized(value, user)) { return false; } return value.getEventId() >= firstRecordId; } }, maxRecords);
@Override public AsyncLineageSubmission submitLineageComputation(final String flowFileUuid, final NiFiUser user) { return submitLineageComputation(Collections.singleton(flowFileUuid), user, LineageComputationType.FLOWFILE_LINEAGE, null); }
@Override public void registerEvents(final Iterable<ProvenanceEventRecord> events) { for (final ProvenanceEventRecord event : events) { registerEvent(event); } }
public QueryResult queryEvents(final Query query, final NiFiUser user) throws IOException { final QuerySubmission submission = submitQuery(query, user); final QueryResult result = submission.getResult(); while (!result.isFinished()) { try { Thread.sleep(100L); } catch (final InterruptedException ie) { } } if (result.getError() != null) { throw new IOException(result.getError()); } return result; }
@Override public QuerySubmission submitQuery(final Query query, final NiFiUser user) { 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"); } final String userId = user == null ? null : user.getIdentity(); if (query.getSearchTerms().isEmpty() && query.getStartDate() == null && query.getEndDate() == null) { final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1, userId); queryExecService.submit(new QueryRunnable(ringBuffer, createFilter(query, user), query.getMaxResults(), result)); querySubmissionMap.put(query.getIdentifier(), result); return result; } final AsyncQuerySubmission result = new AsyncQuerySubmission(query, 1, userId); querySubmissionMap.put(query.getIdentifier(), result); queryExecService.submit(new QueryRunnable(ringBuffer, createFilter(query, user), query.getMaxResults(), result)); return result; }
@Override public ComputeLineageSubmission submitExpandParents(final long eventId, final NiFiUser user) { final String userId = user == null ? null : user.getIdentity(); final ProvenanceEventRecord event = getEvent(eventId, user); if (event == null) { final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); submission.getResult().update(Collections.emptyList(), 0L); return submission; } switch (event.getEventType()) { case JOIN: case FORK: case REPLAY: case CLONE: return submitLineageComputation(event.getParentUuids(), user, LineageComputationType.EXPAND_PARENTS, eventId); default: { final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_PARENTS, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its parents cannot be expanded"); return submission; } } }
@Override public boolean select(final ProvenanceEventRecord event) { if (!isAuthorized(event, user)) { return false; } if (flowFileUuids.contains(event.getFlowFileUuid())) { return true; } for (final String parentId : event.getParentUuids()) { if (flowFileUuids.contains(parentId)) { return true; } } for (final String childId : event.getChildUuids()) { if (flowFileUuids.contains(childId)) { return true; } } return false; } };
private Lineage computeLineage(final Collection<String> flowFileUuids, final NiFiUser user, final LineageComputationType computationType, final Long eventId) throws IOException { final AsyncLineageSubmission submission = submitLineageComputation(flowFileUuids, user, computationType, eventId); final StandardLineageResult result = submission.getResult(); while (!result.isFinished()) { try { Thread.sleep(100L); } catch (final InterruptedException ie) { } } if (result.getError() != null) { throw new IOException(result.getError()); } return new FlowFileLineage(result.getNodes(), result.getEdges()); }
@Override public ComputeLineageSubmission submitExpandChildren(final long eventId, final NiFiUser user) { final String userId = user == null ? null : user.getIdentity(); final ProvenanceEventRecord event = getEvent(eventId, user); if (event == null) { final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); submission.getResult().update(Collections.emptyList(), 0L); return submission; } switch (event.getEventType()) { case JOIN: case FORK: case REPLAY: case CLONE: return submitLineageComputation(event.getChildUuids(), user, LineageComputationType.EXPAND_CHILDREN, eventId); default: { final AsyncLineageSubmission submission = new AsyncLineageSubmission(LineageComputationType.EXPAND_CHILDREN, eventId, Collections.emptyList(), 1, userId); lineageSubmissionMap.put(submission.getLineageIdentifier(), submission); submission.getResult().setError("Event ID " + eventId + " indicates an event of type " + event.getEventType() + " so its children cannot be expanded"); return submission; } } }
@Override public boolean select(final ProvenanceEventRecord event) { if (!isAuthorized(event, user)) { return false;