dariuszseweryn commented on code in PR #10077: URL: https://github.com/apache/nifi/pull/10077#discussion_r2297478119
########## nifi-extension-bundles/nifi-aws-bundle/nifi-aws-kinesis/src/main/java/org/apache/nifi/processors/aws/kinesis/ConsumeKinesis.java: ########## @@ -0,0 +1,616 @@ +/* + * 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.expression.ExpressionLanguageScope; +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.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.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 several seconds 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 = 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.") Review Comment: It is worth emphasizing that this only affects the buffer size. KCL may buffer more data for each shard that will wait for the `RecordBuffer` to accept them. -- 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]
