private FlowFile getMostRecent(final FlowFile flowFile) { final StandardRepositoryRecord existingRecord = getRecord(flowFile); return existingRecord == null ? flowFile : existingRecord.getCurrent(); }
@Override public void transfer(FlowFile flowFile) { verifyTaskActive(); flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = getRecord(flowFile); if (record.getOriginalQueue() == null) { throw new IllegalArgumentException("Cannot transfer FlowFiles that are created in this Session back to self"); } record.setTransferRelationship(Relationship.SELF); updateLastQueuedDate(record); }
for (final FlowFile flowFile : flowFiles) { final FlowFileRecord flowFileRecord = (FlowFileRecord) flowFile; final StandardRepositoryRecord record = getRecord(flowFileRecord); record.setTransferRelationship(relationship); updateLastQueuedDate(record, queuedTime);
@Override public FlowFile removeAllAttributes(FlowFile flowFile, final Pattern keyPattern) { verifyTaskActive(); flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = getRecord(flowFile); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(keyPattern).build(); if (keyPattern == null) { record.setWorking(newFile); } else { final Map<String, String> curAttrs = record.getCurrent().getAttributes(); final Map<String, String> removed = new HashMap<>(); for (final String key : curAttrs.keySet()) { if (CoreAttributes.UUID.key().equals(key)) { continue; } if (keyPattern.matcher(key).matches()) { removed.put(key, null); } } record.setWorking(newFile, removed); } return newFile; }
@Override public FlowFile removeAllAttributes(FlowFile flowFile, final Set<String> keys) { verifyTaskActive(); flowFile = validateRecordState(flowFile); if (keys == null) { return flowFile; } final StandardRepositoryRecord record = getRecord(flowFile); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(keys).build(); final Map<String, String> updatedAttrs = new HashMap<>(); for (final String key : keys) { if (CoreAttributes.UUID.key().equals(key)) { continue; } updatedAttrs.put(key, null); } record.setWorking(newFile, updatedAttrs); return newFile; }
@Override public FlowFile putAllAttributes(FlowFile flowFile, final Map<String, String> attributes) { verifyTaskActive(); flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = getRecord(flowFile); final Map<String, String> updatedAttributes; if (attributes.containsKey(CoreAttributes.UUID.key())) { updatedAttributes = new HashMap<>(attributes); updatedAttributes.remove(CoreAttributes.UUID.key()); } else { updatedAttributes = attributes; } final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).addAttributes(updatedAttributes); final FlowFileRecord newFile = ffBuilder.build(); record.setWorking(newFile, updatedAttributes); return newFile; }
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 FlowFile removeAttribute(FlowFile flowFile, final String key) { verifyTaskActive(); flowFile = validateRecordState(flowFile); if (CoreAttributes.UUID.key().equals(key)) { return flowFile; } final StandardRepositoryRecord record = getRecord(flowFile); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(key).build(); record.setWorking(newFile, key, null); return newFile; }
@Override public FlowFile putAttribute(FlowFile flowFile, final String key, final String value) { verifyTaskActive(); flowFile = validateRecordState(flowFile); if (CoreAttributes.UUID.key().equals(key)) { return flowFile; } final StandardRepositoryRecord record = getRecord(flowFile); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).addAttribute(key, value).build(); record.setWorking(newFile, key, value); return newFile; }
final StandardRepositoryRecord record = getRecord(destination);
@Override public FlowFile penalize(FlowFile flowFile) { verifyTaskActive(); flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = getRecord(flowFile); final long expirationEpochMillis = System.currentTimeMillis() + context.getConnectable().getPenalizationPeriod(TimeUnit.MILLISECONDS); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).penaltyExpirationTime(expirationEpochMillis).build(); record.setWorking(newFile); return newFile; }
@Override public void transfer(FlowFile flowFile, final Relationship relationship) { verifyTaskActive(); flowFile = validateRecordState(flowFile); final int numDestinations = context.getConnections(relationship).size(); final int multiplier = Math.max(1, numDestinations); boolean autoTerminated = false; boolean selfRelationship = false; if (numDestinations == 0 && context.getConnectable().isAutoTerminated(relationship)) { // auto terminated. autoTerminated = true; } else if (numDestinations == 0 && relationship == Relationship.SELF) { selfRelationship = true; } else if (numDestinations == 0) { // the relationship specified is not known in this session/context throw new IllegalArgumentException("Relationship '" + relationship.getName() + "' is not known"); } final StandardRepositoryRecord record = getRecord(flowFile); record.setTransferRelationship(relationship); updateLastQueuedDate(record); if (autoTerminated) { removedCount += multiplier; removedBytes += flowFile.getSize(); } else if (!selfRelationship) { flowFilesOut += multiplier; contentSizeOut += flowFile.getSize() * multiplier; } }
@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())); } }
@Override public void remove(Collection<FlowFile> flowFiles) { verifyTaskActive(); flowFiles = validateRecordState(flowFiles); for (final FlowFile flowFile : flowFiles) { 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) { generatedProvenanceEvents.remove(flowFile); removeForkEvents(flowFile); } else { removedCount++; removedBytes += flowFile.getSize(); provenanceReporter.drop(flowFile, flowFile.getAttribute(CoreAttributes.DISCARD_REASON.key())); } } }
@Override public void exportTo(FlowFile source, final Path destination, final boolean append) { verifyTaskActive(); source = validateRecordState(source); final StandardRepositoryRecord record = getRecord(source); try { ensureNotAppending(record.getCurrentClaim()); claimCache.flush(record.getCurrentClaim()); final long copyCount = context.getContentRepository().exportTo(record.getCurrentClaim(), destination, append, record.getCurrentClaimOffset(), source.getSize()); bytesRead += copyCount; bytesWritten += copyCount; } catch (final ContentNotFoundException nfe) { handleContentNotFound(nfe, record); } catch (final Throwable t) { throw new FlowFileAccessException("Failed to export " + source + " to " + destination + " due to " + t.toString(), t); } }
private void registerForkEvent(final FlowFile parent, final FlowFile child) { ProvenanceEventBuilder eventBuilder = forkEventBuilders.get(parent); if (eventBuilder == null) { eventBuilder = context.getProvenanceRepository().eventBuilder(); eventBuilder.setEventType(ProvenanceEventType.FORK); eventBuilder.setFlowFileEntryDate(parent.getEntryDate()); eventBuilder.setLineageStartDate(parent.getLineageStartDate()); eventBuilder.setFlowFileUUID(parent.getAttribute(CoreAttributes.UUID.key())); eventBuilder.setComponentId(context.getConnectable().getIdentifier()); final Connectable connectable = context.getConnectable(); final String processorType = connectable.getComponentType(); eventBuilder.setComponentType(processorType); eventBuilder.addParentFlowFile(parent); updateEventContentClaims(eventBuilder, parent, getRecord(parent)); forkEventBuilders.put(parent, eventBuilder); } eventBuilder.addChildFlowFile(child); }
final StandardRepositoryRecord record = getRecord(destination); ContentClaim newClaim = null; final long claimOffset = 0L;
@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; }
verifyTaskActive(); source = validateRecordState(source); final StandardRepositoryRecord record = getRecord(source);
verifyTaskActive(); final StandardRepositoryRecord repoRecord = getRecord(flowFile); if (repoRecord == null) { throw new FlowFileHandlingException(flowFile + " is not known in this session (" + toString() + ")");