exceptionfactory commented on code in PR #10077: URL: https://github.com/apache/nifi/pull/10077#discussion_r2301406760
########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis-nar/src/main/resources/META-INF/NOTICE: ########## @@ -0,0 +1,391 @@ +nifi-aws-kinesis-3-nar +Copyright 2015-2025 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +****************** +Apache Software License v2 +****************** + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Amazon Event Stream + The following NOTICE information applies: + AWS EventStream for Java + Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Amazon Kinesis Client Library + The following NOTICE information applies: + AmazonKinesisClientLibrary + Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Apache Commons Collections + The following NOTICE information applies: + Apache Commons Collections + Copyright 2001-2015 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + (ASLv2) Apache Commons Collections 4 + The following NOTICE information applies: + Apache Commons Collections + Copyright 2001-2025 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (https://www.apache.org/). + + (ASLv2) AWS SDK for Java V2 + The following NOTICE information applies: + AWS SDK for Java 2.0 + Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. + + This product includes software developed by + Amazon Technologies, Inc (http://www.amazon.com/). + + ********************** + THIRD PARTY COMPONENTS + ********************** + This software includes third party software subject to the following copyrights: + - XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty. + - PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. + - Apache Commons Lang - https://github.com/apache/commons-lang + - Netty Reactive Streams - https://github.com/playframework/netty-reactive-streams + - Jackson-core - https://github.com/FasterXML/jackson-core + - Jackson-dataformat-cbor - https://github.com/FasterXML/jackson-dataformats-binary + + The licenses for these third party components are included in LICENSE.txt + + - For Apache Commons Lang see also this required NOTICE: + Apache Commons Lang + Copyright 2001-2020 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (https://www.apache.org/). + + (ASLv2) DynamoDB Lock Client + The following NOTICE information applies: + DynamoDB Lock Client + Copyright 2013 - 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Google Guava + The following NOTICE information applies: + Google Guava + Copyright (C) 2024 The Guava Authors + + (ASLv2) JSpecify + The following NOTICE information applies: + JSpecify + Copyright 2022 The JSpecify Authors + + (ASLv2) FindBugs JSR305 + The following NOTICE information applies: + FindBugs JSR305 + Copyright 2017 The FindBugs JSR305 Authors + + (ASLv2) Netty Review Comment: Is Netty included? It should be part of the shared NAR, unless a specific codec library is included from Netty. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis-nar/src/main/resources/META-INF/LICENSE: ########## @@ -0,0 +1,232 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +APACHE NIFI SUBCOMPONENTS: + +The Apache NiFi project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + + The binary distribution of this product bundles 'Bouncy Castle JDK 1.5' + under an MIT style license. + + Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.org) + + Permission is hereby granted, free of charge, to any person obtaining a copy + of this software and associated documentation files (the "Software"), to deal + in the Software without restriction, including without limitation the rights + to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + copies of the Software, and to permit persons to whom the Software is + furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in + all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + THE SOFTWARE. Review Comment: This section should be removed because the Bouncy Castle library should not be included in the NAR. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor: ########## @@ -12,6 +12,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. + Review Comment: This empty line can be reverted. ########## nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java: ########## @@ -109,11 +110,16 @@ public MockProcessSession(final SharedSessionState sharedState, final Processor public MockProcessSession(final SharedSessionState sharedState, final Processor processor, final boolean enforceStreamsClosed, final StateManager stateManager, final boolean allowSynchronousCommits) { - this(sharedState, processor, enforceStreamsClosed, stateManager, allowSynchronousCommits, false); + this(sharedState, processor, enforceStreamsClosed, stateManager, allowSynchronousCommits, false, false); Review Comment: These `nifi-mock` changes should be considered in a separate pull request, since they can impact the behavior of all tests. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ReaderRecordProcessor.java: ########## @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.aws.kinesis; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.WriteResult; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.List; + +import static java.util.Collections.emptyMap; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE; + +final class ReaderRecordProcessor { + + private final RecordReaderFactory recordReaderFactory; + private final RecordSetWriterFactory recordWriterFactory; + private final ComponentLog logger; + + ReaderRecordProcessor( + final RecordReaderFactory recordReaderFactory, + final RecordSetWriterFactory recordWriterFactory, + final ComponentLog logger) { + this.recordReaderFactory = recordReaderFactory; + this.recordWriterFactory = recordWriterFactory; + this.logger = logger; + } + + ProcessingResult processRecords( + final ProcessSession session, + final String streamName, + final String shardId, + final List<KinesisClientRecord> records) { + final List<FlowFile> successFlowFiles = new ArrayList<>(); + final List<FlowFile> failureFlowFiles = new ArrayList<>(); + + ActiveFlowFile activeFlowFile = null; + + for (final KinesisClientRecord kinesisRecord : records) { + final byte[] data = new byte[kinesisRecord.data().remaining()]; + kinesisRecord.data().get(data); + + try (final InputStream in = new ByteArrayInputStream(data); + final RecordReader reader = recordReaderFactory.createRecordReader(emptyMap(), in, data.length, logger)) { + + Record record; + while ((record = reader.nextRecord()) != null) { + final RecordSchema writeSchema = recordWriterFactory.getSchema(emptyMap(), record.getSchema()); + + if (activeFlowFile == null) { + activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId); + } else if (!writeSchema.equals(activeFlowFile.schema())) { + // If the write schema has changed, we need to complete the current FlowFile and start a new one. + final FlowFile completedFlowFile = activeFlowFile.complete(); + successFlowFiles.add(completedFlowFile); + + activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId); + } + + activeFlowFile.writeRecord(record, kinesisRecord); + } + } catch (final IOException | MalformedRecordException | SchemaNotFoundException e) { + logger.error("Failed to parse record from Kinesis stream using configured Record Reader", e); + final FlowFile failureFlowFile = createParseFailureFlowFile(session, streamName, shardId, kinesisRecord, e); + failureFlowFiles.add(failureFlowFile); + } + } + + if (activeFlowFile != null) { + final FlowFile completedFlowFile = activeFlowFile.complete(); + successFlowFiles.add(completedFlowFile); + } + + return new ProcessingResult(successFlowFiles, failureFlowFiles); + } + + private static FlowFile createParseFailureFlowFile( + final ProcessSession session, + final String streamName, + final String shardId, + final KinesisClientRecord record, + final Exception e) { + FlowFile flowFile = session.create(); + + record.data().rewind(); + flowFile = session.write(flowFile, out -> Channels.newChannel(out).write(record.data())); + + flowFile = session.putAllAttributes(flowFile, ConsumeKinesisAttributes.fromKinesisRecord(streamName, shardId, record)); + + final Throwable cause = e.getCause() != null ? e.getCause() : e; + final String errorMessage = cause.getLocalizedMessage() != null ? cause.getLocalizedMessage() : cause.getClass().getCanonicalName() + " thrown"; Review Comment: Using the full Java class name in an error message is not ideal because it exposes implementation details. I recommend using a placeholder message instead. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis-nar/src/main/resources/META-INF/NOTICE: ########## @@ -0,0 +1,391 @@ +nifi-aws-kinesis-3-nar +Copyright 2015-2025 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +****************** +Apache Software License v2 +****************** + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Amazon Event Stream + The following NOTICE information applies: + AWS EventStream for Java + Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Amazon Kinesis Client Library + The following NOTICE information applies: + AmazonKinesisClientLibrary + Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Apache Commons Collections + The following NOTICE information applies: + Apache Commons Collections + Copyright 2001-2015 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + (ASLv2) Apache Commons Collections 4 + The following NOTICE information applies: + Apache Commons Collections + Copyright 2001-2025 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (https://www.apache.org/). + + (ASLv2) AWS SDK for Java V2 + The following NOTICE information applies: + AWS SDK for Java 2.0 + Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. + + This product includes software developed by + Amazon Technologies, Inc (http://www.amazon.com/). + + ********************** + THIRD PARTY COMPONENTS + ********************** + This software includes third party software subject to the following copyrights: + - XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty. + - PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. + - Apache Commons Lang - https://github.com/apache/commons-lang + - Netty Reactive Streams - https://github.com/playframework/netty-reactive-streams + - Jackson-core - https://github.com/FasterXML/jackson-core + - Jackson-dataformat-cbor - https://github.com/FasterXML/jackson-dataformats-binary + + The licenses for these third party components are included in LICENSE.txt + + - For Apache Commons Lang see also this required NOTICE: + Apache Commons Lang + Copyright 2001-2020 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (https://www.apache.org/). + + (ASLv2) DynamoDB Lock Client + The following NOTICE information applies: + DynamoDB Lock Client + Copyright 2013 - 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Google Guava + The following NOTICE information applies: + Google Guava + Copyright (C) 2024 The Guava Authors + + (ASLv2) JSpecify + The following NOTICE information applies: + JSpecify + Copyright 2022 The JSpecify Authors + + (ASLv2) FindBugs JSR305 + The following NOTICE information applies: + FindBugs JSR305 + Copyright 2017 The FindBugs JSR305 Authors Review Comment: This library should not be included in the NAR. If it is, the dependencies should be updated to exclude it. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis-nar/src/main/resources/META-INF/NOTICE: ########## @@ -0,0 +1,391 @@ +nifi-aws-kinesis-3-nar +Copyright 2015-2025 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +****************** +Apache Software License v2 +****************** + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Amazon Event Stream + The following NOTICE information applies: + AWS EventStream for Java + Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Amazon Kinesis Client Library + The following NOTICE information applies: + AmazonKinesisClientLibrary + Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + (ASLv2) Apache Commons Collections + The following NOTICE information applies: + Apache Commons Collections + Copyright 2001-2015 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). Review Comment: Is the historical Commons Collection library actually included? ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ConsumeKinesis.java: ########## @@ -0,0 +1,607 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.aws.kinesis; + +import jakarta.annotation.Nullable; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.SystemResource; +import org.apache.nifi.annotation.behavior.SystemResourceConsideration; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.configuration.DefaultSettings; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService; +import org.apache.nifi.processors.aws.kinesis.ReaderRecordProcessor.ProcessingResult; +import org.apache.nifi.processors.aws.kinesis.RecordBuffer.ShardBufferId; +import org.apache.nifi.processors.aws.kinesis.RecordBuffer.ShardBufferLease; +import org.apache.nifi.processors.aws.region.RegionUtilV2; +import org.apache.nifi.proxy.ProxyConfiguration; +import org.apache.nifi.proxy.ProxyConfigurationService; +import org.apache.nifi.proxy.ProxySpec; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.http.Protocol; +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.http.nio.netty.Http2Configuration; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder; +import software.amazon.kinesis.common.ConfigsBuilder; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.coordinator.Scheduler; +import software.amazon.kinesis.lifecycle.events.InitializationInput; +import software.amazon.kinesis.lifecycle.events.LeaseLostInput; +import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; +import software.amazon.kinesis.lifecycle.events.ShardEndedInput; +import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; +import software.amazon.kinesis.processor.ShardRecordProcessor; +import software.amazon.kinesis.processor.ShardRecordProcessorFactory; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.net.URI; +import java.nio.channels.Channels; +import java.time.Duration; +import java.time.Instant; +import java.util.Date; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; + +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.APPROXIMATE_ARRIVAL_TIMESTAMP; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.MIME_TYPE; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.PARTITION_KEY; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.SEQUENCE_NUMBER; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.SHARD_ID; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.STREAM_NAME; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.SUB_SEQUENCE_NUMBER; + +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +@Tags({"amazon", "aws", "kinesis", "consume", "stream", "record"}) +@CapabilityDescription(""" + Consumes data from the specified AWS Kinesis stream and outputs a FlowFile for every processed Record (raw) \ + or a FlowFile for a batch of processed records if a Record Reader and Record Writer are configured. + AWS Kinesis Client Library can take a few minutes on the first start and several seconds on subsequent start \ + to initialise before starting to fetch data. + Uses DynamoDB for check pointing and coordination, and CloudWatch (optional) for metrics. + Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.""") +@WritesAttributes({ + @WritesAttribute(attribute = STREAM_NAME, + description = "The name of the Kinesis Stream from which all Kinesis Records in the Flow File were read"), + @WritesAttribute(attribute = SHARD_ID, + description = "Shard ID from which all Kinesis Records in the Flow File were read"), + @WritesAttribute(attribute = PARTITION_KEY, + description = "Partition key of the last Kinesis Record in the Flow File"), + @WritesAttribute(attribute = SEQUENCE_NUMBER, + description = "A Sequence Number of the last Kinesis Record in the Flow File"), + @WritesAttribute(attribute = SUB_SEQUENCE_NUMBER, + description = "A SubSequence Number of the last Kinesis Record in the Flow File. Generated by KPL when aggregating records into a single Kinesis Record"), + @WritesAttribute(attribute = APPROXIMATE_ARRIVAL_TIMESTAMP, + description = "Approximate arrival timestamp of the last Kinesis Record in the Flow File"), + @WritesAttribute(attribute = MIME_TYPE, + description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer (if configured)"), + @WritesAttribute(attribute = RECORD_COUNT, + description = "Number of records written to the FlowFiles by the Record Writer (if configured)"), + @WritesAttribute(attribute = RECORD_ERROR_MESSAGE, + description = "This attribute provides on failure the error message encountered by the Record Reader or Record Writer (if configured)") +}) +@DefaultSettings(yieldDuration = "100 millis") +@SystemResourceConsideration(resource = SystemResource.CPU, description = """ + Kinesis Client Library is used to create a Worker thread for consumption of Kinesis Records. + The Worker is initialised and started when this Processor has been scheduled. It runs continually, spawning Kinesis Record Processors as required \ + to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) are not controlled by the normal NiFi scheduler as part of the \ + Concurrent Thread pool and are not released until this processor is stopped.""") +@SystemResourceConsideration(resource = SystemResource.NETWORK, description = """ + Kinesis Client Library will continually poll for new Records, \ + requesting up to a maximum number of Records/bytes per call. This can result in sustained network usage.""") +@SystemResourceConsideration(resource = SystemResource.MEMORY, description = """ + ConsumeKinesis buffers Kinesis Records in memory until they can be processed. + The maximum size of the buffer is controlled by the 'Max Bytes to Buffer' property. + In addition, Kinesis Client Library may cache some amount of data for each shard when the processor's buffer is full.""") +public class ConsumeKinesis extends AbstractProcessor { + + private static final Duration HTTP_CLIENTS_CONNECTION_TIMEOUT = Duration.ofSeconds(30); + private static final Duration HTTP_CLIENTS_READ_TIMEOUT = Duration.ofMinutes(3); + + /** + * Best balance between throughput and CPU usage by KCL. + */ + private static final int KINESIS_HTTP_CLIENT_CONCURRENCY_PER_TASK = 16; + private static final int KINESIS_HTTP_CLIENT_WINDOW_SIZE = 512 * 1024; // 512 KiB + private static final Duration KINESIS_HTTP_HEALTH_CHECK_PERIOD = Duration.ofMinutes(1); + + static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder() + .name("Amazon Kinesis Stream Name") + .description("The name of the Kinesis stream to consume from.") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder() + .name("Application Name") + .description("The name of the Kinesis application. This is used for DynamoDB table naming and worker coordination.") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder() + .name("AWS Credentials Provider service") + .displayName("AWS Credentials Provider Service") + .description("The Controller Service that is used to obtain AWS credentials provider.") + .required(true) + .identifiesControllerService(AWSCredentialsProviderService.class) + .build(); + + static final PropertyDescriptor REGION = new PropertyDescriptor.Builder() + .name("Region") + .description("AWS Region in which the Kinesis stream is located.") + .required(true) + .allowableValues(RegionUtilV2.getAvailableRegions()) + .defaultValue(RegionUtilV2.createAllowableValue(Region.US_WEST_2).getValue()) + .build(); + + static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder() + .name("Record Reader") + .description(""" + The Record Reader to use for parsing the data received from Kinesis. + If not set, the records are written one per FlowFile without parsing.""") + .required(false) + .identifiesControllerService(RecordReaderFactory.class) + .build(); + + static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder() + .name("Record Writer") + .description("The Record Writer to use for serializing records.") + .required(true) + .dependsOn(RECORD_READER) + .identifiesControllerService(RecordSetWriterFactory.class) + .build(); + + static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder() + .name("Initial Stream Position") + .description("The position in the stream where the processor should start reading.") + .required(true) + .allowableValues(InitialPosition.class) + .defaultValue(InitialPosition.TRIM_HORIZON) + .build(); + + static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder() + .name("Stream Position Timestamp") + .description("Timestamp position in stream from which to start reading Kinesis Records. The timestamp must be in ISO 8601 format.") + .addValidator(StandardValidators.ISO8601_INSTANT_VALIDATOR) + .dependsOn(INITIAL_STREAM_POSITION, InitialPosition.AT_TIMESTAMP) + .required(true) + .build(); + + static final PropertyDescriptor MAX_BYTES_TO_BUFFER = new PropertyDescriptor.Builder() + .name("Max Bytes to Buffer") + .description(""" + The maximum size of Kinesis Records that can be buffered in memory before being processed by NiFi. + If the buffer size exceeds the limit, the KCL will stop consuming new records until free space is available. + + Using a larger value may improve throughput, but will do so at the expense of using additional heap. + Using a smaller value may back off the Kinesis Client Library (KCL) from consuming records if the buffer is full, which may result in lower throughput. + """) + .required(true) + .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .defaultValue("100 MB") + .build(); + + static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder() + .name("Checkpoint Interval") + .description(""" + Interval between checkpointing consumed Kinesis records. To checkpoint records on each NiFi session commit, set this value to 0 seconds. + + Checkpointing too frequently may result in performance degradation and higher DynamoDB costs. + Checkpointing too rarely may result in duplicated records whenever a Shard lease is lost or NiFi server restarts. + """) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .defaultValue("5 sec") + .required(true) + .build(); + + static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder() Review Comment: Are there any other metrics reporting options? If so, it would be better to make this an allowed value setting, with options including `NONE` and `CLOUDWATCH`. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ReaderRecordProcessor.java: ########## @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.aws.kinesis; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.WriteResult; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.List; + +import static java.util.Collections.emptyMap; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE; + +final class ReaderRecordProcessor { + + private final RecordReaderFactory recordReaderFactory; + private final RecordSetWriterFactory recordWriterFactory; + private final ComponentLog logger; + + ReaderRecordProcessor( + final RecordReaderFactory recordReaderFactory, + final RecordSetWriterFactory recordWriterFactory, + final ComponentLog logger) { + this.recordReaderFactory = recordReaderFactory; + this.recordWriterFactory = recordWriterFactory; + this.logger = logger; + } + + ProcessingResult processRecords( + final ProcessSession session, + final String streamName, + final String shardId, + final List<KinesisClientRecord> records) { + final List<FlowFile> successFlowFiles = new ArrayList<>(); + final List<FlowFile> failureFlowFiles = new ArrayList<>(); + + ActiveFlowFile activeFlowFile = null; + + for (final KinesisClientRecord kinesisRecord : records) { + final byte[] data = new byte[kinesisRecord.data().remaining()]; + kinesisRecord.data().get(data); + + try (final InputStream in = new ByteArrayInputStream(data); + final RecordReader reader = recordReaderFactory.createRecordReader(emptyMap(), in, data.length, logger)) { + + Record record; + while ((record = reader.nextRecord()) != null) { + final RecordSchema writeSchema = recordWriterFactory.getSchema(emptyMap(), record.getSchema()); + + if (activeFlowFile == null) { + activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId); + } else if (!writeSchema.equals(activeFlowFile.schema())) { + // If the write schema has changed, we need to complete the current FlowFile and start a new one. + final FlowFile completedFlowFile = activeFlowFile.complete(); + successFlowFiles.add(completedFlowFile); + + activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId); + } + + activeFlowFile.writeRecord(record, kinesisRecord); + } + } catch (final IOException | MalformedRecordException | SchemaNotFoundException e) { + logger.error("Failed to parse record from Kinesis stream using configured Record Reader", e); + final FlowFile failureFlowFile = createParseFailureFlowFile(session, streamName, shardId, kinesisRecord, e); + failureFlowFiles.add(failureFlowFile); + } + } + + if (activeFlowFile != null) { + final FlowFile completedFlowFile = activeFlowFile.complete(); + successFlowFiles.add(completedFlowFile); + } + + return new ProcessingResult(successFlowFiles, failureFlowFiles); + } + + private static FlowFile createParseFailureFlowFile( + final ProcessSession session, + final String streamName, + final String shardId, + final KinesisClientRecord record, + final Exception e) { + FlowFile flowFile = session.create(); + + record.data().rewind(); + flowFile = session.write(flowFile, out -> Channels.newChannel(out).write(record.data())); + + flowFile = session.putAllAttributes(flowFile, ConsumeKinesisAttributes.fromKinesisRecord(streamName, shardId, record)); + + final Throwable cause = e.getCause() != null ? e.getCause() : e; + final String errorMessage = cause.getLocalizedMessage() != null ? cause.getLocalizedMessage() : cause.getClass().getCanonicalName() + " thrown"; + flowFile = session.putAttribute(flowFile, RECORD_ERROR_MESSAGE, errorMessage); + + return flowFile; + } + + record ProcessingResult(List<FlowFile> successFlowFiles, List<FlowFile> parseFailureFlowFiles) { + } + + private static final class ActiveFlowFile { + + private final ComponentLog logger; + + private final ProcessSession session; + private final FlowFile flowFile; + private final RecordSetWriter writer; + private final RecordSchema schema; + + private final String streamName; + private final String shardId; + + private KinesisClientRecord lastRecord; + + private ActiveFlowFile( + final ComponentLog logger, + final ProcessSession session, + final FlowFile flowFile, + final RecordSetWriter writer, + final RecordSchema schema, + final String streamName, + final String shardId) { + this.logger = logger; + this.session = session; + this.flowFile = flowFile; + this.writer = writer; + this.schema = schema; + this.streamName = streamName; + this.shardId = shardId; + } + + static ActiveFlowFile startNewFile( + final ComponentLog logger, + final ProcessSession session, + final RecordSetWriterFactory recordWriterFactory, + final RecordSchema writeSchema, + final String streamName, + final String shardId) throws SchemaNotFoundException { + final FlowFile flowFile = session.create(); + final OutputStream outputStream = session.write(flowFile); + + try { + final RecordSetWriter writer = recordWriterFactory.createWriter(logger, writeSchema, outputStream, flowFile); + writer.beginRecordSet(); + + return new ActiveFlowFile(logger, session, flowFile, writer, writeSchema, streamName, shardId); + + } catch (final SchemaNotFoundException e) { + logger.debug("Failed to find writeSchema for Kinesis stream record", e); + try { + outputStream.close(); + } catch (final IOException ioe) { + logger.warn("Failed to close FlowFile output stream", ioe); Review Comment: If the exception is suppressed, it seems better to log it later. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ConsumeKinesis.java: ########## @@ -0,0 +1,607 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.aws.kinesis; + +import jakarta.annotation.Nullable; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.SystemResource; +import org.apache.nifi.annotation.behavior.SystemResourceConsideration; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.configuration.DefaultSettings; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService; +import org.apache.nifi.processors.aws.kinesis.ReaderRecordProcessor.ProcessingResult; +import org.apache.nifi.processors.aws.kinesis.RecordBuffer.ShardBufferId; +import org.apache.nifi.processors.aws.kinesis.RecordBuffer.ShardBufferLease; +import org.apache.nifi.processors.aws.region.RegionUtilV2; +import org.apache.nifi.proxy.ProxyConfiguration; +import org.apache.nifi.proxy.ProxyConfigurationService; +import org.apache.nifi.proxy.ProxySpec; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.http.Protocol; +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.http.nio.netty.Http2Configuration; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder; +import software.amazon.kinesis.common.ConfigsBuilder; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.coordinator.Scheduler; +import software.amazon.kinesis.lifecycle.events.InitializationInput; +import software.amazon.kinesis.lifecycle.events.LeaseLostInput; +import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; +import software.amazon.kinesis.lifecycle.events.ShardEndedInput; +import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; +import software.amazon.kinesis.processor.ShardRecordProcessor; +import software.amazon.kinesis.processor.ShardRecordProcessorFactory; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.net.URI; +import java.nio.channels.Channels; +import java.time.Duration; +import java.time.Instant; +import java.util.Date; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; + +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.APPROXIMATE_ARRIVAL_TIMESTAMP; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.MIME_TYPE; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.PARTITION_KEY; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.SEQUENCE_NUMBER; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.SHARD_ID; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.STREAM_NAME; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.SUB_SEQUENCE_NUMBER; + +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN) +@Tags({"amazon", "aws", "kinesis", "consume", "stream", "record"}) +@CapabilityDescription(""" + Consumes data from the specified AWS Kinesis stream and outputs a FlowFile for every processed Record (raw) \ + or a FlowFile for a batch of processed records if a Record Reader and Record Writer are configured. + AWS Kinesis Client Library can take a few minutes on the first start and several seconds on subsequent start \ + to initialise before starting to fetch data. + Uses DynamoDB for check pointing and coordination, and CloudWatch (optional) for metrics. + Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.""") +@WritesAttributes({ + @WritesAttribute(attribute = STREAM_NAME, + description = "The name of the Kinesis Stream from which all Kinesis Records in the Flow File were read"), + @WritesAttribute(attribute = SHARD_ID, + description = "Shard ID from which all Kinesis Records in the Flow File were read"), + @WritesAttribute(attribute = PARTITION_KEY, + description = "Partition key of the last Kinesis Record in the Flow File"), + @WritesAttribute(attribute = SEQUENCE_NUMBER, + description = "A Sequence Number of the last Kinesis Record in the Flow File"), + @WritesAttribute(attribute = SUB_SEQUENCE_NUMBER, + description = "A SubSequence Number of the last Kinesis Record in the Flow File. Generated by KPL when aggregating records into a single Kinesis Record"), + @WritesAttribute(attribute = APPROXIMATE_ARRIVAL_TIMESTAMP, + description = "Approximate arrival timestamp of the last Kinesis Record in the Flow File"), + @WritesAttribute(attribute = MIME_TYPE, + description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer (if configured)"), + @WritesAttribute(attribute = RECORD_COUNT, + description = "Number of records written to the FlowFiles by the Record Writer (if configured)"), + @WritesAttribute(attribute = RECORD_ERROR_MESSAGE, + description = "This attribute provides on failure the error message encountered by the Record Reader or Record Writer (if configured)") +}) +@DefaultSettings(yieldDuration = "100 millis") +@SystemResourceConsideration(resource = SystemResource.CPU, description = """ + Kinesis Client Library is used to create a Worker thread for consumption of Kinesis Records. + The Worker is initialised and started when this Processor has been scheduled. It runs continually, spawning Kinesis Record Processors as required \ + to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) are not controlled by the normal NiFi scheduler as part of the \ + Concurrent Thread pool and are not released until this processor is stopped.""") +@SystemResourceConsideration(resource = SystemResource.NETWORK, description = """ + Kinesis Client Library will continually poll for new Records, \ + requesting up to a maximum number of Records/bytes per call. This can result in sustained network usage.""") +@SystemResourceConsideration(resource = SystemResource.MEMORY, description = """ + ConsumeKinesis buffers Kinesis Records in memory until they can be processed. + The maximum size of the buffer is controlled by the 'Max Bytes to Buffer' property. + In addition, Kinesis Client Library may cache some amount of data for each shard when the processor's buffer is full.""") +public class ConsumeKinesis extends AbstractProcessor { + + private static final Duration HTTP_CLIENTS_CONNECTION_TIMEOUT = Duration.ofSeconds(30); + private static final Duration HTTP_CLIENTS_READ_TIMEOUT = Duration.ofMinutes(3); + + /** + * Best balance between throughput and CPU usage by KCL. + */ + private static final int KINESIS_HTTP_CLIENT_CONCURRENCY_PER_TASK = 16; + private static final int KINESIS_HTTP_CLIENT_WINDOW_SIZE = 512 * 1024; // 512 KiB + private static final Duration KINESIS_HTTP_HEALTH_CHECK_PERIOD = Duration.ofMinutes(1); + + static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder() + .name("Amazon Kinesis Stream Name") + .description("The name of the Kinesis stream to consume from.") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder() + .name("Application Name") + .description("The name of the Kinesis application. This is used for DynamoDB table naming and worker coordination.") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder() + .name("AWS Credentials Provider service") + .displayName("AWS Credentials Provider Service") + .description("The Controller Service that is used to obtain AWS credentials provider.") + .required(true) + .identifiesControllerService(AWSCredentialsProviderService.class) + .build(); + + static final PropertyDescriptor REGION = new PropertyDescriptor.Builder() + .name("Region") + .description("AWS Region in which the Kinesis stream is located.") + .required(true) + .allowableValues(RegionUtilV2.getAvailableRegions()) + .defaultValue(RegionUtilV2.createAllowableValue(Region.US_WEST_2).getValue()) + .build(); + + static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder() + .name("Record Reader") + .description(""" + The Record Reader to use for parsing the data received from Kinesis. + If not set, the records are written one per FlowFile without parsing.""") + .required(false) + .identifiesControllerService(RecordReaderFactory.class) + .build(); + + static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder() + .name("Record Writer") + .description("The Record Writer to use for serializing records.") + .required(true) + .dependsOn(RECORD_READER) + .identifiesControllerService(RecordSetWriterFactory.class) + .build(); + + static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder() + .name("Initial Stream Position") + .description("The position in the stream where the processor should start reading.") + .required(true) + .allowableValues(InitialPosition.class) + .defaultValue(InitialPosition.TRIM_HORIZON) + .build(); + + static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder() + .name("Stream Position Timestamp") + .description("Timestamp position in stream from which to start reading Kinesis Records. The timestamp must be in ISO 8601 format.") + .addValidator(StandardValidators.ISO8601_INSTANT_VALIDATOR) + .dependsOn(INITIAL_STREAM_POSITION, InitialPosition.AT_TIMESTAMP) + .required(true) + .build(); + + static final PropertyDescriptor MAX_BYTES_TO_BUFFER = new PropertyDescriptor.Builder() + .name("Max Bytes to Buffer") + .description(""" + The maximum size of Kinesis Records that can be buffered in memory before being processed by NiFi. + If the buffer size exceeds the limit, the KCL will stop consuming new records until free space is available. + + Using a larger value may improve throughput, but will do so at the expense of using additional heap. + Using a smaller value may back off the Kinesis Client Library (KCL) from consuming records if the buffer is full, which may result in lower throughput. + """) + .required(true) + .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .defaultValue("100 MB") + .build(); + + static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder() + .name("Checkpoint Interval") + .description(""" + Interval between checkpointing consumed Kinesis records. To checkpoint records on each NiFi session commit, set this value to 0 seconds. + + Checkpointing too frequently may result in performance degradation and higher DynamoDB costs. + Checkpointing too rarely may result in duplicated records whenever a Shard lease is lost or NiFi server restarts. + """) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .defaultValue("5 sec") + .required(true) + .build(); + + static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder() + .name("Report Metrics to CloudWatch") + .description("Whether to report Kinesis usage metrics to CloudWatch.") + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .allowableValues("true", "false") + .defaultValue("false") + .required(true) + .build(); + + static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(ProxySpec.HTTP, ProxySpec.HTTP_AUTH); + + private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = List.of( + KINESIS_STREAM_NAME, + APPLICATION_NAME, + AWS_CREDENTIALS_PROVIDER_SERVICE, + REGION, + RECORD_READER, + RECORD_WRITER, + INITIAL_STREAM_POSITION, + STREAM_POSITION_TIMESTAMP, + MAX_BYTES_TO_BUFFER, + CHECKPOINT_INTERVAL, + PROXY_CONFIGURATION_SERVICE, + REPORT_CLOUDWATCH_METRICS + ); + + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles that are created when records are successfully read from Kinesis and parsed.") + .build(); + + static final Relationship REL_PARSE_FAILURE = new Relationship.Builder() + .name("parse.failure") + .description("FlowFiles that failed to parse using the configured Record Reader.") + .build(); + + private static final Set<Relationship> RAW_FILE_RELATIONSHIPS = Set.of(REL_SUCCESS); + private static final Set<Relationship> RECORD_FILE_RELATIONSHIPS = Set.of(REL_SUCCESS, REL_PARSE_FAILURE); + + private volatile DynamoDbAsyncClient dynamoDbClient; + private volatile CloudWatchAsyncClient cloudWatchClient; + private volatile KinesisAsyncClient kinesisClient; + private volatile Scheduler kinesisScheduler; + + private volatile String streamName; + private volatile RecordBuffer.ForProcessor recordBuffer; + + private volatile boolean useReader = false; + private volatile Optional<ReaderRecordProcessor> readerRecordProcessor = Optional.empty(); + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + return PROPERTY_DESCRIPTORS; + } + + @Override + public Set<Relationship> getRelationships() { + return useReader ? RECORD_FILE_RELATIONSHIPS : RAW_FILE_RELATIONSHIPS; + } + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + if (descriptor.equals(RECORD_READER)) { + useReader = newValue != null; + } + } + + @OnScheduled + public void setup(final ProcessContext context) { + final RecordReaderFactory recordReaderFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); + if (recordReaderFactory != null) { + final RecordSetWriterFactory recordWriterFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class); + readerRecordProcessor = Optional.of(new ReaderRecordProcessor(recordReaderFactory, recordWriterFactory, getLogger())); + } + + final Region region = Region.of(context.getProperty(REGION).getValue()); + final AwsCredentialsProvider credentialsProvider = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE) + .asControllerService(AWSCredentialsProviderService.class).getAwsCredentialsProvider(); + + kinesisClient = KinesisAsyncClient.builder() + .region(region) + .credentialsProvider(credentialsProvider) + .endpointOverride(getKinesisEndpointOverride()) + .httpClient(createKinesisHttpClient(context)) + .build(); + + dynamoDbClient = DynamoDbAsyncClient.builder() + .region(region) + .credentialsProvider(credentialsProvider) + .endpointOverride(getDynamoDbEndpointOverride()) + .httpClient(createHttpClientBuilder(context).build()) + .build(); + + cloudWatchClient = CloudWatchAsyncClient.builder() + .region(region) + .credentialsProvider(credentialsProvider) + .endpointOverride(getCloudwatchEndpointOverride()) + .httpClient(createHttpClientBuilder(context).build()) + .build(); + + streamName = context.getProperty(KINESIS_STREAM_NAME).getValue(); + final InitialPositionInStreamExtended initialPositionExtended = getInitialPosition(context); + final SingleStreamTracker streamTracker = new SingleStreamTracker(streamName, initialPositionExtended); + + final long maxBytesToBuffer = context.getProperty(MAX_BYTES_TO_BUFFER).asDataSize(DataUnit.B).longValue(); + final Duration checkpointInterval = context.getProperty(CHECKPOINT_INTERVAL).asDuration(); + final MemoryBoundRecordBuffer memoryBoundRecordBuffer = new MemoryBoundRecordBuffer(getLogger(), maxBytesToBuffer, checkpointInterval); + recordBuffer = memoryBoundRecordBuffer; + final ShardRecordProcessorFactory recordProcessorFactory = () -> new ConsumeKinesisRecordProcessor(memoryBoundRecordBuffer); + + final String applicationName = context.getProperty(APPLICATION_NAME).getValue(); + final String workerId = UUID.randomUUID().toString(); Review Comment: I recommend using the Component ID as the Worker ID. ########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ReaderRecordProcessor.java: ########## @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.aws.kinesis; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.RecordSetWriter; +import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.WriteResult; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.List; + +import static java.util.Collections.emptyMap; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_COUNT; +import static org.apache.nifi.processors.aws.kinesis.ConsumeKinesisAttributes.RECORD_ERROR_MESSAGE; + +final class ReaderRecordProcessor { + + private final RecordReaderFactory recordReaderFactory; + private final RecordSetWriterFactory recordWriterFactory; + private final ComponentLog logger; + + ReaderRecordProcessor( + final RecordReaderFactory recordReaderFactory, + final RecordSetWriterFactory recordWriterFactory, + final ComponentLog logger) { + this.recordReaderFactory = recordReaderFactory; + this.recordWriterFactory = recordWriterFactory; + this.logger = logger; + } + + ProcessingResult processRecords( + final ProcessSession session, + final String streamName, + final String shardId, + final List<KinesisClientRecord> records) { + final List<FlowFile> successFlowFiles = new ArrayList<>(); + final List<FlowFile> failureFlowFiles = new ArrayList<>(); + + ActiveFlowFile activeFlowFile = null; + + for (final KinesisClientRecord kinesisRecord : records) { + final byte[] data = new byte[kinesisRecord.data().remaining()]; + kinesisRecord.data().get(data); + + try (final InputStream in = new ByteArrayInputStream(data); + final RecordReader reader = recordReaderFactory.createRecordReader(emptyMap(), in, data.length, logger)) { + + Record record; + while ((record = reader.nextRecord()) != null) { + final RecordSchema writeSchema = recordWriterFactory.getSchema(emptyMap(), record.getSchema()); + + if (activeFlowFile == null) { + activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId); + } else if (!writeSchema.equals(activeFlowFile.schema())) { + // If the write schema has changed, we need to complete the current FlowFile and start a new one. + final FlowFile completedFlowFile = activeFlowFile.complete(); + successFlowFiles.add(completedFlowFile); + + activeFlowFile = ActiveFlowFile.startNewFile(logger, session, recordWriterFactory, writeSchema, streamName, shardId); + } + + activeFlowFile.writeRecord(record, kinesisRecord); + } + } catch (final IOException | MalformedRecordException | SchemaNotFoundException e) { + logger.error("Failed to parse record from Kinesis stream using configured Record Reader", e); + final FlowFile failureFlowFile = createParseFailureFlowFile(session, streamName, shardId, kinesisRecord, e); + failureFlowFiles.add(failureFlowFile); + } + } + + if (activeFlowFile != null) { + final FlowFile completedFlowFile = activeFlowFile.complete(); + successFlowFiles.add(completedFlowFile); + } + + return new ProcessingResult(successFlowFiles, failureFlowFiles); + } + + private static FlowFile createParseFailureFlowFile( + final ProcessSession session, + final String streamName, + final String shardId, + final KinesisClientRecord record, + final Exception e) { + FlowFile flowFile = session.create(); + + record.data().rewind(); + flowFile = session.write(flowFile, out -> Channels.newChannel(out).write(record.data())); + + flowFile = session.putAllAttributes(flowFile, ConsumeKinesisAttributes.fromKinesisRecord(streamName, shardId, record)); + + final Throwable cause = e.getCause() != null ? e.getCause() : e; + final String errorMessage = cause.getLocalizedMessage() != null ? cause.getLocalizedMessage() : cause.getClass().getCanonicalName() + " thrown"; + flowFile = session.putAttribute(flowFile, RECORD_ERROR_MESSAGE, errorMessage); + + return flowFile; + } + + record ProcessingResult(List<FlowFile> successFlowFiles, List<FlowFile> parseFailureFlowFiles) { + } + + private static final class ActiveFlowFile { + + private final ComponentLog logger; + + private final ProcessSession session; + private final FlowFile flowFile; + private final RecordSetWriter writer; + private final RecordSchema schema; + + private final String streamName; + private final String shardId; + + private KinesisClientRecord lastRecord; + + private ActiveFlowFile( + final ComponentLog logger, + final ProcessSession session, + final FlowFile flowFile, + final RecordSetWriter writer, + final RecordSchema schema, + final String streamName, + final String shardId) { + this.logger = logger; + this.session = session; + this.flowFile = flowFile; + this.writer = writer; + this.schema = schema; + this.streamName = streamName; + this.shardId = shardId; + } + + static ActiveFlowFile startNewFile( + final ComponentLog logger, + final ProcessSession session, + final RecordSetWriterFactory recordWriterFactory, + final RecordSchema writeSchema, + final String streamName, + final String shardId) throws SchemaNotFoundException { + final FlowFile flowFile = session.create(); + final OutputStream outputStream = session.write(flowFile); + + try { + final RecordSetWriter writer = recordWriterFactory.createWriter(logger, writeSchema, outputStream, flowFile); + writer.beginRecordSet(); + + return new ActiveFlowFile(logger, session, flowFile, writer, writeSchema, streamName, shardId); + + } catch (final SchemaNotFoundException e) { + logger.debug("Failed to find writeSchema for Kinesis stream record", e); + try { + outputStream.close(); + } catch (final IOException ioe) { + logger.warn("Failed to close FlowFile output stream", ioe); + e.addSuppressed(ioe); + } + throw e; + + } catch (final IOException e) { + final ProcessException processException = new ProcessException("Failed to create a writer for a FlowFile", e); + + logger.debug("Failed to create a writer for a FlowFile. Stopping Kinesis records processing", e); + try { + outputStream.close(); + } catch (final IOException ioe) { + logger.warn("Failed to close FlowFile output stream", ioe); + processException.addSuppressed(ioe); + } + throw processException; + } + } + + RecordSchema schema() { + return schema; + } + + void writeRecord(final Record record, final KinesisClientRecord kinesisRecord) { + try { + writer.write(record); + } catch (final IOException e) { + logger.debug("Failed to write to a FlowFile. Stopping Kinesis records processing", e); Review Comment: Logging at the debug level and throwing an exception is not ideal, instead, the caller should catch and log the exception if necessary. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
