/** * Will remove FAILED_* attributes if FlowFile is no longer considered a * failed FlowFile * * @see #isFailedFlowFile(FlowFile) */ private FlowFile cleanUpFlowFileIfNecessary(FlowFile flowFile, ProcessSession session) { if (this.isFailedFlowFile(flowFile)) { Set<String> keysToRemove = new HashSet<>(); keysToRemove.add(FAILED_DELIMITER_ATTR); keysToRemove.add(FAILED_KEY_ATTR); keysToRemove.add(FAILED_TOPIC_ATTR); keysToRemove.add(FAILED_PROC_ID_ATTR); keysToRemove.add(FAILED_LAST_ACK_IDX); flowFile = session.removeAllAttributes(flowFile, keysToRemove); } return flowFile; }
/** * Will rendezvous with {@link KafkaPublisher} after building * {@link PublishingContext} and will produce the resulting {@link FlowFile}. * The resulting FlowFile contains all required information to determine * if message publishing originated from the provided FlowFile has actually * succeeded fully, partially or failed completely (see * {@link #isFailedFlowFile(FlowFile)}. */ private FlowFile doRendezvousWithKafka(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) { final AtomicReference<KafkaPublisherResult> publishResultRef = new AtomicReference<>(); session.read(flowFile, new InputStreamCallback() { @Override public void process(InputStream contentStream) throws IOException { PublishingContext publishingContext = PublishKafka.this.buildPublishingContext(flowFile, context, contentStream); KafkaPublisherResult result = PublishKafka.this.kafkaResource.publish(publishingContext); publishResultRef.set(result); } }); FlowFile resultFile = publishResultRef.get().isAllAcked() ? this.cleanUpFlowFileIfNecessary(flowFile, session) : session.putAllAttributes(flowFile, this.buildFailedFlowFileAttributes(publishResultRef.get().getLastMessageAcked(), flowFile, context)); if (!this.isFailedFlowFile(resultFile)) { resultFile = session.putAttribute(resultFile, MSG_COUNT, String.valueOf(publishResultRef.get().getMessagesSent())); } return resultFile; }
/** * Will rendezvous with Kafka if {@link ProcessSession} contains {@link FlowFile} * producing a result {@link FlowFile}. * <br> * The result {@link FlowFile} that is successful is then transfered to {@link #REL_SUCCESS} * <br> * The result {@link FlowFile} that is failed is then transfered to {@link #REL_FAILURE} * */ @Override protected boolean rendezvousWithKafka(ProcessContext context, ProcessSession session){ FlowFile flowFile = session.get(); if (flowFile != null) { long start = System.nanoTime(); flowFile = this.doRendezvousWithKafka(flowFile, context, session); Relationship relationship = REL_SUCCESS; if (!this.isFailedFlowFile(flowFile)) { String topic = context.getProperty(TOPIC).evaluateAttributeExpressions(flowFile).getValue(); long executionDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); String transitUri = this.buildTransitURI(context.getProperty(SECURITY_PROTOCOL).getValue(), this.brokers, topic); session.getProvenanceReporter().send(flowFile, transitUri, "Sent " + flowFile.getAttribute(MSG_COUNT) + " Kafka messages", executionDuration); this.getLogger().info("Successfully sent {} to Kafka as {} message(s) in {} millis", new Object[] { flowFile, flowFile.getAttribute(MSG_COUNT), executionDuration }); } else { relationship = REL_FAILURE; flowFile = session.penalize(flowFile); } session.transfer(flowFile, relationship); } return flowFile != null; }
byte[] delimiterBytes = null; int lastAckedMessageIndex = -1; if (this.isFailedFlowFile(flowFile)) { lastAckedMessageIndex = Integer.valueOf(flowFile.getAttribute(FAILED_LAST_ACK_IDX)); topicName = flowFile.getAttribute(FAILED_TOPIC_ATTR);