@Override public FlowFile create() { verifyTaskActive(); final Map<String, String> attrs = new HashMap<>(); final String uuid = UUID.randomUUID().toString(); attrs.put(CoreAttributes.FILENAME.key(), uuid); attrs.put(CoreAttributes.PATH.key(), DEFAULT_FLOWFILE_PATH); attrs.put(CoreAttributes.UUID.key(), uuid); final FlowFileRecord fFile = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence()) .addAttributes(attrs) .build(); final StandardRepositoryRecord record = new StandardRepositoryRecord(null); record.setWorking(fFile, attrs); records.put(fFile.getId(), record); createdFlowFiles.add(fFile.getAttribute(CoreAttributes.UUID.key())); return fFile; }
private RemoteFlowFileRecord receiveFlowFile(final DataInputStream dis, final OutputStream out, final ContentClaim contentClaim, final long claimOffset, final int protocolVersion, final String peerDescription, final LoadBalanceCompression compression) throws IOException { final int metadataLength = dis.readInt(); DataInputStream metadataIn = new DataInputStream(new LimitingInputStream(dis, metadataLength)); if (compression != LoadBalanceCompression.DO_NOT_COMPRESS) { metadataIn = new DataInputStream(new GZIPInputStream(metadataIn)); } final Map<String, String> attributes = readAttributes(metadataIn); final String sourceSystemUuid = attributes.get(CoreAttributes.UUID.key()); logger.debug("Received Attributes {} from Peer {}", attributes, peerDescription); final long lineageStartDate = metadataIn.readLong(); final long entryDate = metadataIn.readLong(); final ContentClaimTriple contentClaimTriple = consumeContent(dis, out, contentClaim, claimOffset, peerDescription, compression == LoadBalanceCompression.COMPRESS_ATTRIBUTES_AND_CONTENT); final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() .id(flowFileRepository.getNextFlowFileSequence()) .addAttributes(attributes) .addAttribute(CoreAttributes.UUID.key(), UUID.randomUUID().toString()) .contentClaim(contentClaimTriple.getContentClaim()) .contentClaimOffset(contentClaimTriple.getClaimOffset()) .size(contentClaimTriple.getContentLength()) .entryDate(entryDate) .lineageStart(lineageStartDate, lineageStartIndex.getAndIncrement()) .build(); logger.debug("Received FlowFile {} with {} attributes and {} bytes of content", flowFileRecord, attributes.size(), contentClaimTriple.getContentLength()); return new RemoteFlowFileRecord(sourceSystemUuid, flowFileRecord); }
newAttributes.put(CoreAttributes.UUID.key(), uuid); final StandardFlowFileRecord.Builder fFileBuilder = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence());
final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() .fromFlowFile(record.getCurrent()) .contentClaim(newClaim)
newAttributes.put(CoreAttributes.UUID.key(), uuid); final FlowFileRecord fFile = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence()) .addAttributes(newAttributes) .lineageStart(lineageStartDate, lineageStartIndex)
final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() .fromFlowFile(record.getCurrent()) .contentClaim(newClaim)
@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 FlowFileRecord newFile = new StandardFlowFileRecord.Builder() .fromFlowFile(record.getCurrent()) .contentClaim(newClaim)
@Override public void close() throws IOException { if (closed) { return; } closed = true; writeRecursionSet.remove(sourceFlowFile); final long bytesWritten = countingOut.getBytesWritten(); StandardProcessSession.this.bytesWritten += bytesWritten; final OutputStream removed = openOutputStreams.remove(sourceFlowFile); if (removed == null) { LOG.error("Closed Session's OutputStream but there was no entry for it in the map; sourceFlowFile={}; map={}", sourceFlowFile, openOutputStreams); } flush(); removeTemporaryClaim(record); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder() .fromFlowFile(record.getCurrent()) .contentClaim(updatedClaim) .contentClaimOffset(Math.max(0, updatedClaim.getLength() - bytesWritten)) .size(bytesWritten) .build(); record.setWorking(newFile); } };
@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; }
@SuppressWarnings("unchecked") private StandardRepositoryRecord createRecord(final Record record) { final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder(); ffBuilder.id((Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID)); ffBuilder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE)); final Long lastQueueDate = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE); final Long queueDateIndex = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX); ffBuilder.lastQueued(lastQueueDate, queueDateIndex); final Long lineageStartDate = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE); final Long lineageStartIndex = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX); ffBuilder.lineageStart(lineageStartDate, lineageStartIndex); populateContentClaim(ffBuilder, record); ffBuilder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE)); ffBuilder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES)); final FlowFileRecord flowFileRecord = ffBuilder.build(); final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER); final FlowFileQueue queue = getFlowFileQueue(queueId); final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord(queue, flowFileRecord); requireFlowFileQueue(repoRecord, queueId); return repoRecord; }
@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; }
@SuppressWarnings("unchecked") public static FlowFileRecord getFlowFile(final Record record, final ResourceClaimManager claimManager) { final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder(); builder.id((Long) record.getFieldValue(FlowFileSchema.RECORD_ID)); builder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE)); builder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE)); builder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES)); builder.lineageStart((Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE), (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX)); builder.lastQueued((Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE), (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX)); final Record contentClaimRecord = (Record) record.getFieldValue(FlowFileSchema.CONTENT_CLAIM); if (contentClaimRecord != null) { final ContentClaim claim = ContentClaimFieldMap.getContentClaim(contentClaimRecord, claimManager); builder.contentClaim(claim); final Long offset = ContentClaimFieldMap.getContentClaimOffset(contentClaimRecord); if (offset != null) { builder.contentClaimOffset(offset); } } return builder.build(); } }
@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 RepositoryRecord deleteRecord(final Record record) { final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD); final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId); final FlowFileRecord flowFileRecord = ffBuilder.build(); final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord); repoRecord.markForDelete(); return repoRecord; }
@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; }
@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; }
private RepositoryRecord swapOutRecord(final Record record) { final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD); final String queueId = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE)); final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); final FlowFileQueue queue = getFlowFileQueue(queueId); final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() .id(recordId) .build(); return new StandardRepositoryRecord(queue, flowFileRecord, swapLocation); }
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); }
@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; }