2929import org .apache .nifi .annotation .lifecycle .OnStopped ;
3030import org .apache .nifi .components .DescribedValue ;
3131import org .apache .nifi .components .PropertyDescriptor ;
32+ import org .apache .nifi .components .Validator ;
3233import org .apache .nifi .controller .NodeTypeProvider ;
3334import org .apache .nifi .flowfile .FlowFile ;
3435import org .apache .nifi .logging .ComponentLog ;
9394import java .time .Instant ;
9495import java .util .Date ;
9596import java .util .List ;
97+ import java .util .Map ;
9698import java .util .Optional ;
9799import java .util .Set ;
98100import java .util .UUID ;
102104import java .util .concurrent .TimeoutException ;
103105import java .util .concurrent .atomic .AtomicBoolean ;
104106
107+ import static java .nio .charset .StandardCharsets .UTF_8 ;
105108import static java .util .concurrent .TimeUnit .NANOSECONDS ;
106109import static java .util .concurrent .TimeUnit .SECONDS ;
107110import static org .apache .nifi .processors .aws .kinesis .ConsumeKinesisAttributes .APPROXIMATE_ARRIVAL_TIMESTAMP ;
@@ -246,6 +249,19 @@ Ensure that the credentials provided have access to Kinesis, DynamoDB and (optio
246249 .allowableValues (OutputStrategy .class )
247250 .build ();
248251
252+ static final PropertyDescriptor MESSAGE_DEMARCATOR = new PropertyDescriptor .Builder ()
253+ .name ("Message Demarcator" )
254+ .description ("""
255+ Specifies the string (interpreted as UTF-8) to use for demarcating multiple Kinesis messages
256+ within a single FlowFile. If not specified, the content of the messages will be concatenated
257+ without any delimiter.
258+ To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS.
259+ """ )
260+ .required (false )
261+ .addValidator (Validator .VALID )
262+ .dependsOn (PROCESSING_STRATEGY , ProcessingStrategy .DEMARCATOR )
263+ .build ();
264+
249265 static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor .Builder ()
250266 .name ("Initial Stream Position" )
251267 .description ("The position in the stream where the processor should start reading." )
@@ -309,6 +325,7 @@ Ensure that the credentials provided have access to Kinesis, DynamoDB and (optio
309325 RECORD_READER ,
310326 RECORD_WRITER ,
311327 OUTPUT_STRATEGY ,
328+ MESSAGE_DEMARCATOR ,
312329 INITIAL_STREAM_POSITION ,
313330 STREAM_POSITION_TIMESTAMP ,
314331 MAX_BYTES_TO_BUFFER ,
@@ -339,6 +356,7 @@ Ensure that the credentials provided have access to Kinesis, DynamoDB and (optio
339356 private volatile RecordBuffer .ForProcessor <Lease > recordBuffer ;
340357
341358 private volatile @ Nullable ReaderRecordProcessor readerRecordProcessor ;
359+ private volatile @ Nullable byte [] demarcatorValue ;
342360
343361 private volatile Future <InitializationResult > initializationResultFuture ;
344362 private final AtomicBoolean initialized = new AtomicBoolean ();
@@ -360,7 +378,7 @@ public void migrateProperties(final PropertyConfiguration config) {
360378 @ Override
361379 public Set <Relationship > getRelationships () {
362380 return switch (processingStrategy ) {
363- case FLOW_FILE -> RAW_FILE_RELATIONSHIPS ;
381+ case FLOW_FILE , DEMARCATOR -> RAW_FILE_RELATIONSHIPS ;
364382 case RECORD -> RECORD_FILE_RELATIONSHIPS ;
365383 };
366384 }
@@ -375,9 +393,16 @@ public void onPropertyModified(final PropertyDescriptor descriptor, final String
375393 @ OnScheduled
376394 public void setup (final ProcessContext context ) {
377395 readerRecordProcessor = switch (processingStrategy ) {
378- case FLOW_FILE -> null ;
396+ case FLOW_FILE , DEMARCATOR -> null ;
379397 case RECORD -> createReaderRecordProcessor (context );
380398 };
399+ demarcatorValue = switch (processingStrategy ) {
400+ case FLOW_FILE , RECORD -> null ;
401+ case DEMARCATOR -> {
402+ final String demarcatorValue = context .getProperty (MESSAGE_DEMARCATOR ).getValue ();
403+ yield demarcatorValue != null ? demarcatorValue .getBytes (UTF_8 ) : new byte [0 ];
404+ }
405+ };
381406
382407 final Region region = RegionUtil .getRegion (context );
383408 final AwsCredentialsProvider credentialsProvider = context .getProperty (AWS_CREDENTIALS_PROVIDER_SERVICE )
@@ -593,6 +618,7 @@ private void cleanUpState() {
593618
594619 recordBuffer = null ;
595620 readerRecordProcessor = null ;
621+ demarcatorValue = null ;
596622 }
597623
598624 private void shutdownScheduler () {
@@ -680,11 +706,10 @@ private void processRecordsFromBuffer(final ProcessSession session, final Lease
680706 }
681707
682708 final String shardId = lease .shardId ();
683- final ReaderRecordProcessor processor = readerRecordProcessor ;
684- if (processor != null ) {
685- processRecordsWithReader (processor , session , shardId , records );
686- } else {
687- processRecordsAsRaw (session , shardId , records );
709+ switch (processingStrategy ) {
710+ case FLOW_FILE -> processRecordsAsRaw (session , shardId , records );
711+ case RECORD -> processRecordsWithReader (session , shardId , records );
712+ case DEMARCATOR -> processRecordsAsDemarcated (session , shardId , records );
688713 }
689714
690715 session .adjustCounter ("Records Processed" , records .size (), false );
@@ -732,13 +757,48 @@ private void processRecordsAsRaw(final ProcessSession session, final String shar
732757 }
733758 }
734759
735- private void processRecordsWithReader (final ReaderRecordProcessor recordProcessor , final ProcessSession session , final String shardId , final List <KinesisClientRecord > records ) {
760+ private void processRecordsWithReader (final ProcessSession session , final String shardId , final List <KinesisClientRecord > records ) {
761+ final ReaderRecordProcessor recordProcessor = readerRecordProcessor ;
762+ if (recordProcessor == null ) {
763+ throw new IllegalStateException ("RecordProcessor has not been initialized" );
764+ }
765+
736766 final ProcessingResult result = recordProcessor .processRecords (session , streamName , shardId , records );
737767
738768 session .transfer (result .successFlowFiles (), REL_SUCCESS );
739769 session .transfer (result .parseFailureFlowFiles (), REL_PARSE_FAILURE );
740770 }
741771
772+ private void processRecordsAsDemarcated (final ProcessSession session , final String shardId , final List <KinesisClientRecord > records ) {
773+ final byte [] demarcator = demarcatorValue ;
774+ if (demarcator == null ) {
775+ throw new IllegalStateException ("Demarcator has not been initialized" );
776+ }
777+
778+ FlowFile flowFile = session .create ();
779+
780+ final Map <String , String > attributes = ConsumeKinesisAttributes .fromKinesisRecords (streamName , shardId , records .getFirst (), records .getLast ());
781+ attributes .put (RECORD_COUNT , String .valueOf (records .size ()));
782+ flowFile = session .putAllAttributes (flowFile , attributes );
783+
784+ flowFile = session .write (flowFile , out -> {
785+ try (final WritableByteChannel channel = Channels .newChannel (out )) {
786+ boolean writtenData = false ;
787+ for (final KinesisClientRecord record : records ) {
788+ if (writtenData ) {
789+ out .write (demarcator );
790+ }
791+ channel .write (record .data ());
792+ writtenData = true ;
793+ }
794+ }
795+ });
796+
797+ session .getProvenanceReporter ().receive (flowFile , ProvenanceTransitUriFormat .toTransitUri (streamName , shardId ));
798+
799+ session .transfer (flowFile , REL_SUCCESS );
800+ }
801+
742802 /**
743803 * An adapter between Kinesis Consumer Library and {@link RecordBuffer}.
744804 */
@@ -859,7 +919,8 @@ public String getDescription() {
859919
860920 enum ProcessingStrategy implements DescribedValue {
861921 FLOW_FILE ("Write one FlowFile for each consumed Kinesis Record" ),
862- RECORD ("Write one FlowFile containing multiple consumed Kinesis Records processed with Record Reader and Record Writer" );
922+ RECORD ("Write one FlowFile containing multiple consumed Kinesis Records processed with Record Reader and Record Writer" ),
923+ DEMARCATOR ("Write one FlowFile containing multiple consumed Kinesis Records separated by a configurable demarcator" );
863924
864925 private final String description ;
865926
0 commit comments