final StandardRepositoryRecord repoRecord = records.get(eventFlowFile.getId()); if (repoRecord.getCurrent() != null && repoRecord.getCurrentClaim() != null) { final ContentClaim currentClaim = repoRecord.getCurrentClaim(); final long currentOffset = repoRecord.getCurrentClaimOffset(); final long size = eventFlowFile.getSize(); if (repoRecord.getOriginal() != null && repoRecord.getOriginalClaim() != null) { final ContentClaim originalClaim = repoRecord.getOriginalClaim(); final long originalOffset = repoRecord.getOriginal().getContentClaimOffset(); final long originalSize = repoRecord.getOriginal().getSize(); final FlowFileQueue originalQueue = repoRecord.getOriginalQueue(); if (originalQueue != null) { recordBuilder.setSourceQueueIdentifier(originalQueue.getIdentifier()); recordBuilder.setAttributes(repoRecord.getOriginalAttributes(), repoRecord.getUpdatedAttributes()); recordBuilder.setEventDuration(TimeUnit.NANOSECONDS.toMillis(commitNanos - repoRecord.getStartNanos()));
private RepositoryRecord createRepositoryRecord(final FlowFileRecord flowFile) { final StandardRepositoryRecord record = new StandardRepositoryRecord(flowFileQueue, flowFile); record.markForDelete(); return record; }
private void requireFlowFileQueue(final StandardRepositoryRecord repoRecord, final String queueId) { if (queueId == null || queueId.trim().isEmpty()) { logger.warn("{} does not have a Queue associated with it; this record will be discarded", repoRecord.getCurrent()); repoRecord.markForAbort(); } else if (repoRecord.getOriginalQueue() == null) { logger.warn("{} maps to unknown Queue {}; this record will be discarded", repoRecord.getCurrent(), queueId); repoRecord.markForAbort(); } }
@Override public String toString() { return "StandardRepositoryRecord[UpdateType=" + getType() + ",Record=" + getCurrent() + "]"; }
final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord); record.markForDelete(); return record; record = new StandardRepositoryRecord(queue, flowFile); if (swapLocation != null) { record.setSwapLocation(swapLocation); record.markForAbort(); } else if (queue == null) { logger.warn("{} maps to unknown FlowFile Queue {}; this record will be discarded", flowFile, connectionId); record.markForAbort();
if (record.isMarkedForDelete()) { continue; final Relationship relationship = record.getTransferRelationship(); if (relationship == null) { rollback(); throw new FlowFileHandlingException(record.getCurrent() + " transfer relationship not specified"); record.setDestination(record.getOriginalQueue()); } else if (destinations.isEmpty()) { record.markForDelete(); dropEvent = provenanceReporter.generateDropEvent(record.getCurrent(), "Auto-Terminated by " + relationship.getName() + " Relationship"); autoTerminatedEvents.add(dropEvent); } catch (final Exception e) { LOG.warn("Unable to generate Provenance Event for {} on behalf of {} due to {}", record.getCurrent(), connectableDescription, e); if (LOG.isDebugEnabled()) { LOG.warn("", e); record.setDestination(finalDestination.getFlowFileQueue()); incrementConnectionInputCounts(finalDestination, record); final FlowFileRecord currRec = record.getCurrent(); final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(currRec); builder.id(context.getNextFlowFileSequence()); final StandardRepositoryRecord newRecord = new StandardRepositoryRecord(destination.getFlowFileQueue()); provenanceReporter.clone(currRec, clone, false);
private void updateLastQueuedDate(final StandardRepositoryRecord record, final Long lastQueueDate) { final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()) .lastQueued(lastQueueDate, enqueuedIndex.getAndIncrement()).build(); record.setWorking(newFile); }
if (record.isMarkedForDelete()) { decrementClaimCount(record.getWorkingClaim()); if (record.getOriginalClaim() != null && !record.getOriginalClaim().equals(record.getWorkingClaim())) { decrementClaimCount(record.getOriginalClaim()); final FlowFileRecord flowFile = record.getCurrent(); final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate(); final Connectable connectable = context.getConnectable(); LOG.info("{} terminated by {}; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife}); } else if (record.isWorking() && record.getWorkingClaim() != record.getOriginalClaim()) { decrementClaimCount(record.getOriginalClaim()); if (record.isMarkedForAbort() || record.isMarkedForDelete()) { continue; // these don't need to be transferred if (record.getCurrent() != null) { Collection<FlowFileRecord> collection = recordMap.get(record.getDestination()); if (collection == null) { collection = new ArrayList<>(); recordMap.put(record.getDestination(), collection); collection.add(record.getCurrent());
@Override public ContentClaim getCurrentClaim() { return (getCurrent() == null) ? null : getCurrent().getContentClaim(); }
ensureNotAppending(record.getCurrentClaim()); claimCache.flush(record.getCurrentClaim()); } catch (final IOException e) { throw new FlowFileAccessException("Unable to read from source " + source + " due to " + e.toString(), e); try { newClaim = contentRepo.create(context.getConnectable().isLossTolerant()); claimLog.debug("Creating ContentClaim {} for 'merge' for {}", newClaim, destinationRecord.getCurrent()); } catch (final IOException e) { throw new FlowFileAccessException("Unable to create ContentClaim due to " + e.toString(), e); final StandardRepositoryRecord sourceRecord = getRecord(source); final long copied = contentRepo.exportTo(sourceRecord.getCurrentClaim(), out, sourceRecord.getCurrentClaimOffset(), source.getSize()); writtenCount += copied; readCount += copied; final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(destinationRecord.getCurrent()).contentClaim(newClaim).contentClaimOffset(0L).size(writtenCount).build(); destinationRecord.setWorking(newFile); return newFile;
private void handleContentNotFound(final ContentNotFoundException nfe, final StandardRepositoryRecord suspectRecord) { final ContentClaim registeredClaim = suspectRecord.getOriginalClaim(); final ContentClaim transientClaim = suspectRecord.getWorkingClaim(); final ContentClaim missingClaim = nfe.getMissingClaim(); final ProvenanceEventRecord dropEvent = provenanceReporter.drop(suspectRecord.getCurrent(), nfe.getMessage() == null ? "Content Not Found" : nfe.getMessage()); if (dropEvent != null) { context.getProvenanceRepository().registerEvent(dropEvent); } if (missingClaim == registeredClaim) { suspectRecord.markForAbort(); rollback(); throw new MissingFlowFileException("Unable to find content for FlowFile", nfe); } if (missingClaim == transientClaim) { rollback(); throw new MissingFlowFileException("Unable to find content for FlowFile", nfe); } }
final Set<StandardRepositoryRecord> transferRecords = new HashSet<>(); for (final StandardRepositoryRecord record : recordsToHandle) { if (record.isMarkedForAbort()) { decrementClaimCount(record.getWorkingClaim()); if (record.getCurrentClaim() != null && !record.getCurrentClaim().equals(record.getWorkingClaim())) { decrementClaimCount(record.getCurrentClaim()); if (record.getOriginal() != null) { final FlowFileQueue originalQueue = record.getOriginalQueue(); if (originalQueue != null) { if (penalize) { final long expirationEpochMillis = System.currentTimeMillis() + context.getConnectable().getPenalizationPeriod(TimeUnit.MILLISECONDS); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getOriginal()).penaltyExpirationTime(expirationEpochMillis).build(); originalQueue.put(newFile); } else { originalQueue.put(record.getOriginal()); .flatMap(record -> record.getTransientClaims().stream()) .collect(Collectors.toList());
@Override public FlowFile clone(FlowFile example, final long offset, final long size) { verifyTaskActive(); example = validateRecordState(example); final StandardRepositoryRecord exampleRepoRecord = getRecord(example); final FlowFileRecord currRec = exampleRepoRecord.getCurrent(); final ContentClaim claim = exampleRepoRecord.getCurrentClaim(); if (offset + size > example.getSize()) { throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + example.toString()); } final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(currRec); builder.id(context.getNextFlowFileSequence()); builder.contentClaimOffset(currRec.getContentClaimOffset() + offset); builder.size(size); final String newUuid = UUID.randomUUID().toString(); builder.addAttribute(CoreAttributes.UUID.key(), newUuid); final FlowFileRecord clone = builder.build(); if (claim != null) { context.getContentRepository().incrementClaimaintCount(claim); } final StandardRepositoryRecord record = new StandardRepositoryRecord(null); record.setWorking(clone, clone.getAttributes()); records.put(clone.getId(), record); if (offset == 0L && size == example.getSize()) { provenanceReporter.clone(example, clone); } else { registerForkEvent(example, clone); } return clone; }
final ContentClaim original = repoRecord.getOriginalClaim(); final ContentClaim current = repoRecord.getCurrentClaim(); final FlowFileRecord curFlowFile = repoRecord.getCurrent(); final String flowFileId = curFlowFile.getAttribute(CoreAttributes.UUID.key()); boolean eventAdded = false; final boolean newFlowFile = repoRecord.getOriginal() == null; if (contentChanged && !newFlowFile) { recordsToSubmit.add(provenanceReporter.build(curFlowFile, ProvenanceEventType.CONTENT_MODIFIED).build()); if (!eventAdded && !repoRecord.getUpdatedAttributes().isEmpty()) { final FlowFileRecord flowFile = repoRecord.getCurrent(); flowFileRecordMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile);
details.append(", "); if (repoRecord.getOriginalQueue() != null && repoRecord.getOriginalQueue().getIdentifier() != null) { details.append("queue=") .append(repoRecord.getOriginalQueue().getIdentifier()) .append("/"); .append(repoRecord.getCurrent().getAttribute(CoreAttributes.FILENAME.key())) .append("/uuid=") .append(repoRecord.getCurrent().getAttribute(CoreAttributes.UUID.key()));
@Override public void swapFlowFilesIn(final String swapLocation, final List<FlowFileRecord> swapRecords, final FlowFileQueue queue) throws IOException { final List<RepositoryRecord> repoRecords = new ArrayList<>(); for (final FlowFileRecord swapRecord : swapRecords) { final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord(queue, swapRecord); repoRecord.setSwapLocation(swapLocation); // set the swap file to indicate that it's being swapped in. repoRecord.setDestination(queue); repoRecords.add(repoRecord); } updateRepository(repoRecords, true); logger.info("Repository updated to reflect that {} FlowFiles were swapped in to {}", new Object[]{swapRecords.size(), queue}); }
final FlowFile flowFile = record.getCurrent(); final Relationship relationship = record.getTransferRelationship(); if (Relationship.SELF.equals(relationship)) { continue; largestTransferSetSize = Math.max(largestTransferSetSize, transferIds.size()); final ContentClaim workingClaim = record.getWorkingClaim(); if (workingClaim != null && workingClaim != record.getOriginalClaim() && record.getTransferRelationship() != null) { modifiedFlowFileIds.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
private FlowFile validateRecordState(final FlowFile flowFile, final boolean allowRecursiveRead) { if (!allowRecursiveRead && readRecursionSet.containsKey(flowFile)) { throw new IllegalStateException(flowFile + " already in use for an active callback or an InputStream created by ProcessSession.read(FlowFile) has not been closed"); } if (writeRecursionSet.contains(flowFile)) { throw new IllegalStateException(flowFile + " already in use for an active callback or an OutputStream created by ProcessSession.write(FlowFile) has not been closed"); } final StandardRepositoryRecord record = getRecord(flowFile); if (record == null) { rollback(); throw new FlowFileHandlingException(flowFile + " is not known in this session (" + toString() + ")"); } if (record.getTransferRelationship() != null) { rollback(); throw new FlowFileHandlingException(flowFile + " is already marked for transfer"); } if (record.isMarkedForDelete()) { rollback(); throw new FlowFileHandlingException(flowFile + " has already been marked for removal"); } return record.getCurrent(); }
@Override public void remove(FlowFile flowFile) { verifyTaskActive(); flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = getRecord(flowFile); record.markForDelete(); removedFlowFiles.add(flowFile.getAttribute(CoreAttributes.UUID.key())); // if original connection is null, the FlowFile was created in this session, so we // do not want to count it toward the removed count. if (record.getOriginalQueue() == null) { // if we've generated any Fork events, remove them because the FlowFile was created // and then removed in this session. generatedProvenanceEvents.remove(flowFile); removeForkEvents(flowFile); } else { removedCount++; removedBytes += flowFile.getSize(); provenanceReporter.drop(flowFile, flowFile.getAttribute(CoreAttributes.DISCARD_REASON.key())); } }
final FlowFileQueue inputQueue = repoRecord.getOriginalQueue(); if (inputQueue != null) { final String connectionId = inputQueue.getIdentifier(); incrementConnectionOutputCounts(connectionId, -1, -repoRecord.getOriginal().getSize()); newOwner.incrementConnectionOutputCounts(connectionId, 1, repoRecord.getOriginal().getSize()); if (repoRecord.getTransferRelationship() != null) { flowFilesOut--; contentSizeOut -= flowFile.getSize(); final ContentClaim currentClaim = repoRecord.getCurrentClaim(); if (currentClaim != null) { final ByteCountingOutputStream appendableStream = appendableStreams.remove(currentClaim);