You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/02/11 17:22:49 UTC

[GitHub] [nifi] ChrisSamo632 opened a new pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

ChrisSamo632 opened a new pull request #4822:
URL: https://github.com/apache/nifi/pull/4822


   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   Enables fetching data from AWS Kinesis Data Streams.
   
   Builds upon the [original PR](https://github.com/apache/nifi/pull/3582/files) from @udaykale (and comments there by @turcsanyip and @jaynpearl).
   
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [x] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [x] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [x] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [x] Have you verified that the full build is successful on JDK 11?
   - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - ~[ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?~
   - ~[ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?~
   - [x] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - ~[ ] Have you ensured that format looks appropriate for the output in which it is rendered?~
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-804674166


   > @ChrisSamo632 Any timelines for merging/releasing it out?
   > We have a use-case and wanted to use it in NiFi production.
   
   It's ready to go as far as I'm concerned, but it needs someone to review and then a committer to merge it (there's a big backlog of PRs, I've no idea when anyone will get to this... much as I'd like it in too because we want to use it in production)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628034759



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            final String workerId = generateWorkerId();
+            getLogger().info("Starting Kinesis Worker {}", workerId);
+            worker = prepareWorker(context, sessionFactory, workerId);
+            new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            final boolean success = shutdownWorker();
+            worker = null;
+            readerFactory = null;
+            writerFactory = null;
+
+            if (!success) {
+                getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                context.yield();
+            }
+        }
+    }
+
+    private Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final String appName = getApplicationName(context);
+        final String streamName = getStreamName(context);
+        final long checkpointIntervalMillis = getCheckpointIntervalMillis(context);
+        final String kinesisEndpoint = getKinesisOverride(context).orElse(null);
+
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(sessionFactory, streamName,
+                checkpointIntervalMillis, kinesisEndpoint);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = prepareKinesisClientLibConfiguration(
+                context, appName, streamName, workerId, kinesisEndpoint);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(kinesisClientLibConfiguration, factory, context);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                appName, streamName, workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessSessionFactory sessionFactory, final String streamName,
+                                                                  final long checkpointIntervalMillis, final String kinesisEndpoint) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter, readerFactory, writerFactory
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String appName,
+                                                                       final String streamName, final String workerId,
+                                                                       final String kinesisEndpoint) {
+        final AWSCredentialsProvider credentialsProvider = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).isSet()

Review comment:
       `AWSCredentialsProviderControllerService` has a `Use Anonymous Credentials` option (true/false property). I haven't used it yet but based on its name it should provide the same functionality.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r629955372



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.MockProcessSession;
+import org.apache.nifi.util.SharedSessionState;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.time.format.DateTimeFormatter;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+public class TestAbstractKinesisRecordProcessor {
+    private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS");
+
+    private final TestRunner runner = TestRunners.newTestRunner(ConsumeKinesisStream.class);
+
+    @Mock
+    private ProcessSessionFactory processSessionFactory;
+
+    private final MockProcessSession session = new MockProcessSession(new SharedSessionState(runner.getProcessor(), new AtomicLong(0)), runner.getProcessor());
+
+    private IRecordProcessor fixture;
+
+    @Mock
+    private IRecordProcessorCheckpointer checkpointer;
+
+    @Mock
+    private Record kinesisRecord;
+
+    @Before
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+
+        when(processSessionFactory.createSession()).thenReturn(session);
+
+        // default test fixture will try operations twice with very little wait in between
+        fixture = new MockKinesisRecordProcessor(processSessionFactory, runner.getLogger(), "kinesis-test",
+                "endpoint-prefix", null, 10_000L, 1L, 2, DATE_TIME_FORMATTER);

Review comment:
       Think I'd originally expected to use multiple instances of the mock class, but you're right it doesn't make sense any longer so removed




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 edited a comment on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 edited a comment on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-804689864


   > @ChrisSamo632 - out of curiosity, did you consider the addition of a record reader/writer like with ConsumeKafkaRecord processors? At the moment, as far as I can tell, we would have one record per flow file. Using the records abstraction would provide options such as schema validation, format transformation, as well as having multiple records in one flow file (which is greatly improving the performances in case there is a high number of messages per second).
   
   @pvillard31 think I considered this in our original Slack conversation (or was that with @bbende?), but thought I'd leave out record writers at the minute for simplicity and to better understand how the Kinesis message (vs. Record) structure works
   
   With the KCL worker and multi-threaded consumer approach taken by the KCL library, we'd need a way of combining the records in the processor too (how do we combine records from different consumers? How about where the consumers are reading from different shards and/or multiple consumers from the same shard?)
   
   So I thought a record writer may be a sensible extension (fully agree it would be good to include from a nifi perspective) once there's more understanding of how people use the processor and how it works with different kinesis setups (so far I've only really tested it with simple streams)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4822:
URL: https://github.com/apache/nifi/pull/4822


   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] auyer commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
auyer commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-795907028


   I've been testing this for a few days, and had no issues so far. I should say that I'm not running NiFi in production yet, but plan to do so soon, and this feature helps a lot !
   If anyone else wants to try it in Docker, I've built a image following the instructions above, and published it publicly in [DockeHub](https://hub.docker.com/repository/docker/auyer/apache-nifi-with-get-kinesis). 
   
   Note: I will remove this repo after this PR gets into the next release.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-818492989


   @bhaveshpatelh the AbstractProcessor class has been changed as part of this PR (see earlier comments), so you need to build that from this branch in order for the AWS bundle to compile
   
   The easiest (but probably not fastest) thing is probably to just build the whole of nifi


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r626765126



##########
File path: nifi-api/src/main/java/org/apache/nifi/processor/AbstractProcessor.java
##########
@@ -21,7 +21,7 @@
 public abstract class AbstractProcessor extends AbstractSessionFactoryProcessor {
 
     @Override
-    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {

Review comment:
       @ChrisSamo632 Thank you for the review changes and apologies for my late reply!
   
   Regarding this particular question of accessing `ProcessSessionFactory` in a processor that comes from `AbstactProcessor` (indirectly):
   Basically, we have the following options (as you already mentioned):
   
   1. Remove the processor from the AWS abstract processors hierarchy and extend `AbstractProcessSessionFactory` processor directly (actually, I would not be against it because the code of the AWS abstract processors is quite complicated/deprecated and the new `ConsumeKinesisStream` does not really need the AWS client object provided by the abstract classes)
   2. Inherit `AbstractAWSProcessor` from `AbstractSessionFactoryProcessor`, copy this `onTrigger(ProcessContext, ProcessSessionFactory)` implementation (without the `final` modifier) to the AWS base class and override the method where custom handling of `ProcessSessionFactory` is needed (it would be code duplication and I agree with you that it is not really nice)
   3. Remove `final` modifier in `AbstractProcessor` (it looks OK to me but I'm a bit uncertain about it because it would affect the base class of almost all processors and I believe it was a design decision to make the `AbstractProcessor`'s API as simple as possible)
   
   @markap14 What is your opinion on option 3? Can the `final` be removed or it should rather be untouched?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r629954459



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.MockProcessSession;
+import org.apache.nifi.util.SharedSessionState;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.time.format.DateTimeFormatter;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+public class TestAbstractKinesisRecordProcessor {
+    private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS");
+
+    private final TestRunner runner = TestRunners.newTestRunner(ConsumeKinesisStream.class);
+
+    @Mock
+    private ProcessSessionFactory processSessionFactory;
+
+    private final MockProcessSession session = new MockProcessSession(new SharedSessionState(runner.getProcessor(), new AtomicLong(0)), runner.getProcessor());
+
+    private IRecordProcessor fixture;

Review comment:
       Probably a hangover from before the abstract class existed and the updates to the tests when I introduced the Record Reader/Writer didn't change the variable type, but it does make sense so easily changed




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r580963934



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/GetKinesisStream.java
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.lang3.StringUtils;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.kinesis.stream.record.KinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import javax.annotation.Nonnull;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Stream;
+
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads data from the specified AWS Kinesis stream and outputs a FlowFile for every processed Record. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY, description = "Partition from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER, description = "The unique identifier of the Kinesis Record within its shard."),
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_SHARD_ID, description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP, description = "Approximate arrival timestamp of the Record in the Kinesis stream")
+})
+@SeeAlso(PutKinesisStream.class)
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SuppressWarnings("java:S110")
+public class GetKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (initialise, process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, REGION, ENDPOINT_OVERRIDE, DYNAMODB_ENDPOINT_OVERRIDE,
+                    INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    private long retryWaitMillis;
+    private int numRetries;
+    private InitialPositionInStream initialPositionInStream;
+    private DateTimeFormatter dateTimeFormatter;
+    private Date startStreamPositionTimestamp;
+    private AWSCredentials awsCredentials;
+
+    private volatile ExecutorService executorService;
+    private Map<Worker, Future<?>> workers;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    protected AmazonKinesisClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
+        awsCredentials = credentialsProvider.getCredentials();
+        return super.createClient(context, credentialsProvider, config);
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        super.onScheduled(context);
+
+        workers = new ConcurrentHashMap<>(context.getMaxConcurrentTasks());
+        executorService = Executors.newFixedThreadPool(context.getMaxConcurrentTasks(), new ThreadFactory() {
+            private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+
+            @Override
+            public Thread newThread(@Nonnull final Runnable r) {
+                final Thread t = defaultFactory.newThread(r);
+                t.setName("GetKinesisStream " + getIdentifier() + " Task");
+                return t;
+            }
+        });
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        final Worker worker = prepareWorker(context, session);
+        final Future<?> workerFuture = executorService.submit(worker);
+
+        if (isScheduled()) {
+            workers.put(worker, workerFuture);
+
+            try {
+                getLogger().debug("Starting Kinesis Worker");
+                workerFuture.get(); // blocking Thread, completes when processor unscheduled and stopConsuming called
+                getLogger().debug("Kinesis Worker finished");
+            } catch (@SuppressWarnings("java:S2142") InterruptedException e) {
+                getLogger().warn("Interrupted while executing Kinesis Worker", e);
+            } catch (ExecutionException e) {
+                getLogger().error("Exception executing Kinesis Worker", e);
+                context.yield();
+            }
+        }
+    }
+
+    @OnUnscheduled
+    public void stopConsuming(final ProcessContext context) {
+        final AtomicBoolean success = new AtomicBoolean(true);
+        final Stream<Map.Entry<Worker, Future<?>>> workerStream;
+        if (workers.size() > 1) {
+            workerStream = workers.entrySet().parallelStream();
+        } else {
+            workerStream = workers.entrySet().stream();
+        }
+        workerStream.forEach(entry -> {
+            if (!shutdownWorker(entry.getKey(), entry.getValue())) {
+                success.set(false);
+            }
+        });
+
+        executorService.shutdown();
+        workers = null;
+
+        if (!success.get()) {
+            getLogger().warn("One of more problems while shutting down Kinesis Workers, see logs for details");
+            context.yield();
+        }
+    }
+
+    private Worker prepareWorker(final ProcessContext context, final ProcessSession session) {
+        final String appName = getApplicationName(context);
+        final String streamName = getStreamName(context);
+        final String workerId = generateWorkerId();
+        final long checkpointIntervalMillis = getCheckpointIntervalMillis(context);
+
+        final KinesisRecordProcessorFactory factory = new KinesisRecordProcessorFactory(session, getLogger(),
+                getClient().getEndpointPrefix(), checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = prepareKinesisClientLibConfiguration(
+                context, appName, streamName, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(kinesisClientLibConfiguration, factory, context);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                appName, streamName, workerId);
+
+        return workerBuilder.build();
+    }
+
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String appName,
+                                                                       final String streamName, final String workerId) {
+        final AWSCredentialsProvider credentialsProvider = new AWSStaticCredentialsProvider(awsCredentials);
+
+        @SuppressWarnings({"deprecated", "java:S1874"})
+        // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                new KinesisClientLibConfiguration(appName, streamName, credentialsProvider, workerId)
+                        .withCommonClientConfig(getClient().getClientConfiguration());
+
+        kinesisClientLibConfiguration.withRegionName(getRegion().getName())
+                .withMaxInitializationAttempts(numRetries);

Review comment:
       Second commit includes the update to allow dynamic property use for configuring the KCL




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-808687351


   @turcsanyip thanks for the comments. I'd been a bit confused/concerned about the thread handling and tried (but failed) to find a suitable example, what you suggest looks like a sensible approach on first inspection - I'll try to make the changes sooner than later
   
   Good spot with the KCL issue, will try the downgrade as you suggest (unless a fix comes from AWS in the meantime)
   
   @pvillard31 I'll look again at using Record Reader/Writer at the same time in a fashion similar to the processors suggested above (although I may still suggest it as a future improvement ticket)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628370800



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))

Review comment:
       Looking at this again, I think the options are:
   * stick with `"KCL property with name {}"` and drop the `with`
   * change to `"KCL method with name{}"` and retain the `with`
   These would be more accurate... so think I'll drop the `with` as you suggested and ensure the `prepareKinesisClientLibConfiguration` logs/errors are consistent




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628358283



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))

Review comment:
       The Error message relates to the KCL property though, not the Dynamic Property on the processor, so I'd say this error message is accurate (and the fact it's tied to the affected dynamic property in the NiFi UI should be indication enough of which thing is wrong)




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r626770932



##########
File path: nifi-api/src/main/java/org/apache/nifi/processor/AbstractProcessor.java
##########
@@ -21,7 +21,7 @@
 public abstract class AbstractProcessor extends AbstractSessionFactoryProcessor {
 
     @Override
-    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {

Review comment:
       RE option 1 - I considered this but decided against it because the new processor does use *some* of the existing abstract AWS processor logic (e.g. Proxy config & Auth). I didn't much fancy re-implementing that separately and also it could be difficult to keep the refactored processor closely aligned enough to future AWS Processor bundle updates in future (people easily not realising that they needed to make the same changes more than once within the bundle).
   
   I think there should be a separate Jira ticket to refactor the processors (or this could be considered as part of the migration to AWS SDK 2.x, a ticket for which I've previously raised) - the AWS bundle could certainly stand some rework/simplkfication, but this PR is already big enough IMO
   
   I've implemented option 3 currently in this PR, reworking to option 2 shouldn't be a big change if decided that's to preferred way forward.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r630459168



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,704 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "consume", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")

Review comment:
       Raised as NIFI-5832




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r629644567



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})

Review comment:
       @ChrisSamo632 As there is no Sonar check in the CI build, these SuppressWarnings may not be updated / maintained in future commits (contributed by other developers not running Sonar locally). For this reason, I would remove 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] bhaveshpatelh commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
bhaveshpatelh commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-779624802


   Thanks! Looking forward to this piece going out 


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] markap14 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
markap14 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r629677526



##########
File path: nifi-api/src/main/java/org/apache/nifi/processor/AbstractProcessor.java
##########
@@ -21,7 +21,7 @@
 public abstract class AbstractProcessor extends AbstractSessionFactoryProcessor {
 
     @Override
-    public final void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {

Review comment:
       Yeah I think #2 is the way to go. The point of AbstractProcessor is to take the ProcessSessionFactory and manage the session for you to ensure that the session is properly committed/rolled back. By making that `final` we ensure that if you extend `AbstractProcessor` that is being taken care of for you. If we were to remove the `final` there it means that you could potentially have a Processor that inherits from `AbstractProcessor` but where the session is not properly managed, which would lead to a lot of confusion.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628245431



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})

Review comment:
       "get" needs to be updated to "consume"

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {

Review comment:
       I would allow capitalized property names too (and also capitalize the input for sure).
   
   Error messages returned by `KinesisClientLibConfiguration` contain the capitalized version.
   Eg. configure `maxRecords` to `foo` (good property name but wrong value type)
   The error message (from `InvocationTargetException`):
   `"Value of MaxRecords should be positive, but current value is 0"`
   So referring the property name in the capitalized form seems to me valid.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))

Review comment:
       I would omit the "with-" from the property names in these error messages because it is not part of the name.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");

Review comment:
       I don't think the `AtomicBoolean` trick is not needed here but `ProcessException` can simply be thrown from the `catch` above.
   It would log only the first error but practically cannot be any error here because dynamic properties have already checked by the validator.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);

Review comment:
       Could not `ifPresent()` be used as above in case of `getDynamoDBOverride()`?

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");
+        }
+
+        return kinesisClientLibConfiguration;
+    }
+
+    Worker.Builder prepareWorkerBuilder(final ProcessContext context, final KinesisClientLibConfiguration kinesisClientLibConfiguration,
+                                        final IRecordProcessorFactory factory) {
+        final Worker.Builder workerBuilder = new Worker.Builder()
+                .config(kinesisClientLibConfiguration)
+                .kinesisClient(getClient())
+                .recordProcessorFactory(factory);
+
+        if (!isReportCloudWatchMetrics(context)) {
+            workerBuilder.metricsFactory(new NullMetricsFactory());
+        }
+
+        return workerBuilder;
+    }
+
+    private boolean shutdownWorker(final ProcessContext context) {
+        boolean success = true;
+        try {
+            if (!worker.hasGracefulShutdownStarted()) {
+                getLogger().info("Requesting Kinesis Worker shutdown");
+                final Future<Boolean> shutdown = worker.startGracefulShutdown();
+                // allow 2 seconds longer than the graceful period for shutdown before cancelling the task
+                if (Boolean.FALSE.equals(shutdown.get(getGracefulShutdownMillis(context) + 2_000L, TimeUnit.MILLISECONDS))) {
+                    getLogger().warn("Kinesis Worker shutdown did not complete in time, cancelling");
+                    success = false;
+                } else {
+                    getLogger().info("Kinesis Worker shutdown");
+                }
+            }
+        } catch (@SuppressWarnings("java:S2142") InterruptedException | TimeoutException | ExecutionException e) {
+            getLogger().warn("Problem while shutting down Kinesis Worker: {}", e.getLocalizedMessage(), e);
+            success = false;
+        }
+        return success;
+    }
+
+    private String generateWorkerId() {
+        try {
+            return InetAddress.getLocalHost().getCanonicalHostName() + ":" + UUID.randomUUID();
+        } catch (UnknownHostException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    private String getApplicationName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(APPLICATION_NAME).getValue());
+    }
+
+    private String getStreamName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(KINESIS_STREAM_NAME).getValue());
+    }
+
+    private long getFailoverTimeMillis(final PropertyContext context) {
+        return context.getProperty(FAILOVER_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getGracefulShutdownMillis(final PropertyContext context) {
+        return context.getProperty(GRACEFUL_SHUTDOWN_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getCheckpointIntervalMillis(final PropertyContext context) {
+        return context.getProperty(CHECKPOINT_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private int getNumRetries(final PropertyContext context) {
+        return context.getProperty(NUM_RETRIES).asInteger();
+    }
+
+    private long getRetryWaitMillis(final PropertyContext context) {
+        return context.getProperty(RETRY_WAIT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private boolean isReportCloudWatchMetrics(final PropertyContext context) {
+        return context.getProperty(REPORT_CLOUDWATCH_METRICS).asBoolean();
+    }
+
+    private Optional<String> getKinesisEndpoint(final PropertyContext context) {
+        return context.getProperty(ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private Optional<String> getDynamoDBOverride(final PropertyContext context) {
+        return context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private RecordReaderFactory getReaderFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+    }
+
+    private RecordSetWriterFactory getWriterFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+    }
+
+    private InitialPositionInStream getInitialPositionInStream(final PropertyContext context) {
+        return InitialPositionInStream.valueOf(StringUtils.trimToEmpty(context.getProperty(INITIAL_STREAM_POSITION).getValue()));
+    }
+
+    private DateTimeFormatter getDateTimeFormatter(final PropertyContext context) {
+        return DateTimeFormatter.ofPattern(context.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context) {
+        return getStartStreamTimestamp(context, getDateTimeFormatter(context));
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context, final DateTimeFormatter dateTimeFormatter) {
+        final String streamTimestamp = context.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+        return new Date(LocalDateTime.parse(streamTimestamp, dateTimeFormatter).toInstant(ZoneOffset.UTC).toEpochMilli());

Review comment:
       "Stream Position Timestamp" property is expected to be configured in UTC now, which is not really straightforward on the UI because the "aws.kinesis.approximate.arrival.timestamp" FF attribute is written in local TZ (NiFi server's system TZ).
   I think local system TZ should be used for this property too (and convert it to UTC for Kinesis in the code) or it should be documented in the property's description that UTC is expected.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627175281



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})

Review comment:
       They are, I have a `SonarLint` plugin installed in my IDEs and in the habit of fixing/supressing warnings in my normal projects, so just done the same here.
   
   Happy to remove the supressions (or add appropriate comments) if that's preferred - realise adding these will be different to many other parts of the existing code base




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627510454



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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 are 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()

Review comment:
       Good shout, I only learned about the `asTimePeriod` method after writing this processor and din't think to go back and change things... done




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-837296967


   > @ChrisSamo632 NIFI-8431 (Redundant validation of Dynamic Properties) has been merged. Could you please rebase your PR onto main and update `AbstractConfigurableComponent` according to that?
   > 
   > Please also find my comments regarding tests.
   
   @turcsanyip yep I'd planned to as soon as I could after the PR that replicated my fix was merged (earlier today). Will include a part of the next commit


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 edited a comment on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 edited a comment on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-804674166


   > @ChrisSamo632 Any timelines for merging/releasing it out?
   > We have a use-case and wanted to use it in NiFi production.
   
   @bhaveshpatelh it's ready to go as far as I'm concerned, but it needs someone to review and then a committer to merge it (there's a big backlog of PRs, I've no idea when anyone will get to this... much as I'd like it in too because we want to use it in production)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628366658



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");

Review comment:
       Guess I was thinking that this approach would at least log all the issues if there were multiple... but you're right, the `customValidate` should catch any such issues and have invalidated the processor beofre this code could be executed, so no need for this approach and the simpler "throw on first error" can be used




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-834187070


   > @ChrisSamo632 It seems to me that the checkpoint at shutdown does not work. I have a few uncheckpointed messages that get processed again and again when I restart the processor.
   > 
   > I don't really understand how the graceful shutdown works in KCL, but it seems to me we have no chance to exit cleanly (I am always seeing ZOMBIE state in the record processor's shutdown callback).
   > Could you please double check it?
   > 
   > ...
   
   @turcsanyip I'd noticed ZOMBIE logs a few times and thought it seemed a "normal" thing for KCL (from what I could tell, but note I'm hardly a Kinesis/KCL expert). Taking a look around on the internet, there are many threads about people seeing this kind of behaviour in many different versions of the KCL, for example:
   * https://github.com/awslabs/amazon-kinesis-client/issues/108
   * https://github.com/awslabs/amazon-kinesis-client/issues/677
   * https://github.com/amazon-archives/amazon-kinesis-connectors/issues/10
   
   A common thing between these seems to be that the KCL settings likely need tweaking depending upon one's setup. With the ability to configure most KCL settings via Dynamic Properties on the processor, it's largely open for users to figure this out I guess depending upon their setup.
   
   That said, a common theme seems to be the need to increase the `failoverTimeMillis` and/or `gracefulShutdownMillis` KCL settings, I've therefore added those to the `ConsumeKinesisStream` processor's properties to encourage users to investigate/tweak as required.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628017393



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            final String workerId = generateWorkerId();
+            getLogger().info("Starting Kinesis Worker {}", workerId);
+            worker = prepareWorker(context, sessionFactory, workerId);
+            new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            final boolean success = shutdownWorker();
+            worker = null;
+            readerFactory = null;
+            writerFactory = null;
+
+            if (!success) {
+                getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                context.yield();
+            }
+        }
+    }
+
+    private Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final String appName = getApplicationName(context);
+        final String streamName = getStreamName(context);
+        final long checkpointIntervalMillis = getCheckpointIntervalMillis(context);
+        final String kinesisEndpoint = getKinesisOverride(context).orElse(null);
+
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(sessionFactory, streamName,
+                checkpointIntervalMillis, kinesisEndpoint);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = prepareKinesisClientLibConfiguration(
+                context, appName, streamName, workerId, kinesisEndpoint);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(kinesisClientLibConfiguration, factory, context);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                appName, streamName, workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessSessionFactory sessionFactory, final String streamName,
+                                                                  final long checkpointIntervalMillis, final String kinesisEndpoint) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter, readerFactory, writerFactory
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String appName,
+                                                                       final String streamName, final String workerId,
+                                                                       final String kinesisEndpoint) {
+        final AWSCredentialsProvider credentialsProvider = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).isSet()

Review comment:
       The potential problem there would be if someone had a Kinesis stream (AWS or other variant with the same API) that wasn't configured to require security - they'd have no way of using the processor against such a setup
   
   Such a setup is not something that anyone would recommend. Public/anonymous access to a service, particularly one like this, is rarely a good idea... but disallowing it completely in the processor doesn't seem the right thing to do and all other AWS processors allow for anonymous access too (I found the `AnonymousAWSCredentials` use in the `AbstractAWSProcessor` and reproduced it from there as this code now avoids using the Credentials from the inherritted method after a previous suggested refactoring of the code)




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-808751275


   @turcsanyip started to have a look at making these changes. Using KCL 1.13.3 instead of 1.14.x appears to be straight forward (i.e. no big API changes).
   
   The refactoring to use a single Worker per processor and yield once the Worker has been setup looks fairly straight forward and I agree a simpler implementation *except* that if I am to follow the `ConsumeAzureEventHub` approach and use the `ProcessSessionFactory` version of `onTrigger`, I have to extend the `AbstractSessionFactoryProcessor` base class, but the existing AWS processors have a fairly long chain that extends from `AbstractProcessor` - I don't want to unmarry this processor from all the existing AWS processors and moving all AWS processors to use the different base class feels like a big change and not something we'd want to do.
   
   Is there a straight forward way around this do you think? OIne concern I had with the original implementation was the fact that I was holding on to a single `ProcessSession` and comitting it multiple times (i.e. after every set of messages had been processed by the KCL RecordProcessor) - using the ProcessSessionFactory approach to create a new session every time a new set of Kinesis messages are received would seem better... aside for the above issue of the different abstract processors.
   
   I'll look again at this further, but any guidance is welcome!
   
   
   @pvillard31 having looked again at the Record processing now I've the Azure processor to compare with, I think includiung them here should be fairly straight forward (famous last words).


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 edited a comment on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 edited a comment on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-818492989


   @bhaveshpatelh the `AbstractProcessor` class has been changed as part of this PR (see earlier comments), so you need to build that from this branch in order for the AWS bundle to compile.
   
   The easiest (but probably not fastest) thing is probably to just build the whole of nifi. For a faster build, you could try:
   ```bash
   cd nifi-api
   mvn clean install
   
   cd ../nifi-nar-bundles/nifi-aws-bundle
   mvn -T 2.0C clean install
   ```
   (Note: I've not tried this, so no guarantees it will work)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627456319



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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 are 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getName() + "-" + Worker.class.getName() + "-";

Review comment:
       Darn, I'd intended to use `getSimpleName()` from the off... good catch!




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] bhaveshpatelh commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
bhaveshpatelh commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-818477923


   
   @ChrisSamo632 When I tried following commands to build the processor, it's throwing an error.
   can you guide me with the next steps?
   
   ```
   # clone the code
   git clone https://github.com/ChrisSamo632/nifi.git
   git checkout NIFI-2892
   
   # build the code
   cd nifi/nifi-nar-bundles/nifi-aws-bundle
   mvn -T 2.0C clean install
   ```
   
   ```
   [INFO] -------------------------------------------------------------
   [WARNING] COMPILATION WARNING : 
   [INFO] -------------------------------------------------------------
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AnonymousCredentialsStrategy.java:[26,29] [deprecation] StaticCredentialsProvider in com.amazonaws.internal has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java:[22,32] [deprecation] AmazonS3EncryptionClient in com.amazonaws.services.s3 has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java:[24,32] [deprecation] AmazonS3EncryptionClient in com.amazonaws.services.s3 has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java:[25,38] [deprecation] CryptoConfiguration in com.amazonaws.services.s3.model has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AccessKeyPairCredentialsStrategy.java:[26,29] [deprecation] StaticCredentialsProvider in com.amazonaws.internal has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/wag/InvokeAWSGatewayApi.java:[272,73] [deprecation] getHeaders() in HttpResponse has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AnonymousCredentialsStrategy.java:[44,19] [deprecation] StaticCredentialsProvider in com.amazonaws.internal has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AssumeRoleCredentialsStrategy.java:[163,55] [deprecation] AWSSecurityTokenServiceClient(AWSCredentialsProvider,ClientConfiguration) in AWSSecurityTokenServiceClient has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AssumeRoleCredentialsStrategy.java:[165,32] [deprecation] setEndpoint(String) in AWSSecurityTokenService has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java:[59,8] [deprecation] AmazonS3EncryptionClient in com.amazonaws.services.s3 has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java:[59,46] [deprecation] AmazonS3EncryptionClient in com.amazonaws.services.s3 has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideCEncryptionStrategy.java:[59,42] [deprecation] AmazonS3EncryptionClient(AWSCredentialsProvider,EncryptionMaterialsProvider) in AmazonS3EncryptionClient has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/cloudwatch/PutCloudWatchMetric.java:[251,15] [deprecation] AmazonCloudWatchClient(AWSCredentialsProvider,ClientConfiguration) in AmazonCloudWatchClient has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java:[51,8] [deprecation] CryptoConfiguration in com.amazonaws.services.s3.model has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java:[51,47] [deprecation] CryptoConfiguration in com.amazonaws.services.s3.model has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java:[57,8] [deprecation] AmazonS3EncryptionClient in com.amazonaws.services.s3 has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java:[57,46] [deprecation] AmazonS3EncryptionClient in com.amazonaws.services.s3 has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/s3/encryption/ClientSideKMSEncryptionStrategy.java:[57,42] [deprecation] AmazonS3EncryptionClient(AWSCredentialsProvider,EncryptionMaterialsProvider,CryptoConfiguration) in AmazonS3EncryptionClient has been deprecated
   [WARNING] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/credentials/provider/factory/strategies/AccessKeyPairCredentialsStrategy.java:[49,19] [deprecation] StaticCredentialsProvider in com.amazonaws.internal has been deprecated
   [INFO] 19 warnings 
   [INFO] -------------------------------------------------------------
   [INFO] -------------------------------------------------------------
   [ERROR] COMPILATION ERROR : 
   [INFO] -------------------------------------------------------------
   [ERROR] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java:[486,16] error: onTrigger(ProcessContext,ProcessSessionFactory) in ConsumeKinesisStream cannot override onTrigger(ProcessContext,ProcessSessionFactory) in AbstractProcessor
     overridden method is final
   [INFO] 1 error
   [INFO] -------------------------------------------------------------
   [INFO] ------------------------------------------------------------------------
   [INFO] Reactor Summary for nifi-aws-bundle 1.14.0-SNAPSHOT:
   [INFO] 
   [INFO] nifi-aws-bundle .................................... SUCCESS [  2.902 s]
   [INFO] nifi-aws-service-api ............................... SUCCESS [  5.103 s]
   [INFO] nifi-aws-abstract-processors ....................... SUCCESS [  8.258 s]
   [INFO] nifi-aws-processors ................................ FAILURE [ 30.808 s]
   [INFO] nifi-aws-service-api-nar ........................... SUCCESS [ 15.162 s]
   [INFO] nifi-aws-nar ....................................... SKIPPED
   [INFO] ------------------------------------------------------------------------
   [INFO] BUILD FAILURE
   [INFO] ------------------------------------------------------------------------
   [INFO] Total time:  50.014 s (Wall Clock)
   [INFO] Finished at: 2021-04-13T06:27:18Z
   [INFO] ------------------------------------------------------------------------
   [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile (default-compile) on project nifi-aws-processors: Compilation failure
   [ERROR] /home/ubuntu/nifi/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java:[486,16] error: onTrigger(ProcessContext,ProcessSessionFactory) in ConsumeKinesisStream cannot override onTrigger(ProcessContext,ProcessSessionFactory) in AbstractProcessor
   [ERROR]   overridden method is final
   [ERROR] 
   [ERROR] -> [Help 1]
   [ERROR] 
   [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.
   [ERROR] Re-run Maven using the -X switch to enable full debug logging.
   [ERROR] 
   [ERROR] For more information about the errors and possible solutions, please read the following articles:
   [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException
   [ERROR] 
   [ERROR] After correcting the problems, you can resume the build with the command
   [ERROR]   mvn <goals> -rf :nifi-aws-processors
   
   ```


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r626809145



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,

Review comment:
       `KINESIS_STREAM_NAME` property (defined in `AbstractKinesisStreamProcessor`) supports FlowFile attributes. It may work for the Put processor but is not applicable in case of Consume (as the stream name passed in the Worker once at the beginning).

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(

Review comment:
       Please use upper-case names for static constants.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/KinesisRecordProcessorRaw.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.util.StopWatch;
+
+import java.time.format.DateTimeFormatter;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class KinesisRecordProcessorRaw extends AbstractKinesisRecordProcessor {
+    @SuppressWarnings("java:S107")
+    public KinesisRecordProcessorRaw(final ProcessSessionFactory sessionFactory, final ComponentLog log, final String streamName,
+                                     final String endpointPrefix, final long checkpointIntervalMillis, final long retryWaitMillis,
+                                     final int numRetries, final DateTimeFormatter dateTimeFormatter) {
+        super(sessionFactory, log, streamName, endpointPrefix, checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter);
+    }
+
+    @Override
+    void processRecord(final List<FlowFile> flowFiles, final Record record, final boolean lastRecord,
+                       final ProcessSession session, final StopWatch stopWatch) {
+        final String partitionKey = record.getPartitionKey();
+        final String sequenceNumber = record.getSequenceNumber();
+        final Date approximateArrivalTimestamp = record.getApproximateArrivalTimestamp();
+        final byte[] data = record.getData() != null ? record.getData().array() : new byte[0];
+
+        FlowFile flowFile = session.create();
+        session.write(flowFile, out -> out.write(data));
+
+        if (log.isDebugEnabled()) {
+            log.debug("Sequence No: {}, Partition Key: {}, Data: {}", sequenceNumber, partitionKey, BASE_64_ENCODER.encodeToString(data));
+        }
+
+        session.getProvenanceReporter().receive(flowFile, String.format("http://%s.amazonaws.com/%s/%s#%s", endpointPrefix, kinesisShardId,

Review comment:
       I'm not familiar with alternative endpoints but my understanding is that a full URL can be configured via the processor's property.
   Is it OK to use `.amazonaws.com` suffix in that case?

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/AbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.StopWatch;
+
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor {
+    public static final String AWS_KINESIS_SHARD_ID = "aws.kinesis.shard.id";
+
+    public static final String AWS_KINESIS_SEQUENCE_NUMBER = "aws.kinesis.sequence.number";
+
+    public static final String AWS_KINESIS_PARTITION_KEY = "aws.kinesis.partition.key";
+
+    public static final String AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP = "aws.kinesis.approximate.arrival.timestamp";
+
+    public static final String KINESIS_RECORD_SCHEMA_KEY = "kinesis.name";
+
+    static final Base64.Encoder BASE_64_ENCODER = Base64.getEncoder();
+
+    final ProcessSessionFactory sessionFactory;
+    final ComponentLog log;
+    final String streamName;
+    final String endpointPrefix;
+
+    final long checkpointIntervalMillis;
+    final long retryWaitMillis;
+    final int numRetries;
+    final DateTimeFormatter dateTimeFormatter;
+
+    String kinesisShardId;
+    long nextCheckpointTimeInMillis;
+
+    boolean processingRecords = false;

Review comment:
       These fields could be `private` or `protected`.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),

Review comment:
       The description is a bit misleading for me: `Partition key` is not the ID of a "partition" but belong to the message and decides which shard it goes to (and then read from).

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +

Review comment:
       I think the "NiFi Concurrent Thread is permanently assigned" part is not true any more (because a custom thread is created for the Worker).

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD

Review comment:
       This is a new processor and we don't need to support the legacy, processor level config of the AWS credentials and the proxy. I would add only the Controller Service way for these.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,

Review comment:
       I would move it up between partition key and sequence number (to follow the logical order: partition key decides shard which has a range of sequence numbers).

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})

Review comment:
       Are these Sonar rule suppressions? Do you run it locally?

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", KINESIS_STREAM_NAME);
+    }};
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+    private volatile AWSCredentials awsCredentials;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    protected AmazonKinesisClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
+        awsCredentials = credentialsProvider.getCredentials();
+        return super.createClient(context, credentialsProvider, config);
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            getLogger().debug("Starting Kinesis Worker");
+            worker = prepareWorker(context, sessionFactory);
+            new Thread(worker).start();

Review comment:
       I would strongly recommend to give a custom name to the thread that makes the troubleshooting easier in the future.
   Eg. `"ConsumeKinesisStream-Worker-" + getIdentifier()`

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", KINESIS_STREAM_NAME);
+    }};
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+    private volatile AWSCredentials awsCredentials;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    protected AmazonKinesisClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
+        awsCredentials = credentialsProvider.getCredentials();

Review comment:
       The `AWSCredentialsProvider` object is needed later on so it should be stored instead of the `AWSCredentials`.
   
   Furthermore, I would suggest storing as few instance fields as possible in the processor. `AWSCredentialsProvider` can be retrieved directly from `ProcessContext` in `prepareKinesisClientLibConfiguration()`.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/AbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.StopWatch;
+
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor {
+    public static final String AWS_KINESIS_SHARD_ID = "aws.kinesis.shard.id";
+
+    public static final String AWS_KINESIS_SEQUENCE_NUMBER = "aws.kinesis.sequence.number";
+
+    public static final String AWS_KINESIS_PARTITION_KEY = "aws.kinesis.partition.key";
+
+    public static final String AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP = "aws.kinesis.approximate.arrival.timestamp";
+
+    public static final String KINESIS_RECORD_SCHEMA_KEY = "kinesis.name";
+
+    static final Base64.Encoder BASE_64_ENCODER = Base64.getEncoder();
+
+    final ProcessSessionFactory sessionFactory;
+    final ComponentLog log;
+    final String streamName;
+    final String endpointPrefix;
+
+    final long checkpointIntervalMillis;
+    final long retryWaitMillis;
+    final int numRetries;
+    final DateTimeFormatter dateTimeFormatter;
+
+    String kinesisShardId;
+    long nextCheckpointTimeInMillis;
+
+    boolean processingRecords = false;
+
+    @SuppressWarnings("java:S107")
+    AbstractKinesisRecordProcessor(final ProcessSessionFactory sessionFactory, final ComponentLog log, final String streamName,
+                                   final String endpointPrefix, final long checkpointIntervalMillis, final long retryWaitMillis,
+                                   final int numRetries, final DateTimeFormatter dateTimeFormatter) {
+        this.sessionFactory = sessionFactory;
+        this.log = log;
+        this.streamName = streamName;
+        this.endpointPrefix = endpointPrefix;
+        this.checkpointIntervalMillis = checkpointIntervalMillis;
+        this.retryWaitMillis = retryWaitMillis;
+        this.numRetries = numRetries;
+        this.dateTimeFormatter = dateTimeFormatter;
+    }
+
+    @Override
+    public void initialize(final InitializationInput initializationInput) {
+        if (initializationInput.getPendingCheckpointSequenceNumber() != null) {
+            log.warn("Initializing record processor for stream: {} / shard {}; from sequence number: {}; indicates previously uncheckpointed sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber(), initializationInput.getPendingCheckpointSequenceNumber());
+        } else {
+            log.debug("Initializing record processor for stream: {} / shard: {}; from sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber());
+        }
+
+        this.kinesisShardId = initializationInput.getShardId();
+
+        // ensure we don't immediately checkpoint
+        this.nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
+    }
+
+    @SuppressWarnings({"java:S3252"}) // ConsumeKinesisStream reference to REL_SUCCESS instead of deprecated AbstractAWSProcessor
+    @Override
+    public void processRecords(final ProcessRecordsInput processRecordsInput) {
+        log.debug("Processing {} records from {}; cache entry: {}; cache exit: {}; millis behind latest: {}",
+                processRecordsInput.getRecords().size(), kinesisShardId,
+                processRecordsInput.getCacheEntryTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheEntryTime().atZone(ZoneId.systemDefault())) : null,
+                processRecordsInput.getCacheExitTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheExitTime().atZone(ZoneId.systemDefault())) : null,
+                processRecordsInput.getMillisBehindLatest());

Review comment:
       I don't really like `if (log.isDebugEnabled()) { ... }` but in this case I would add it because `processRecords()` gets called frequently and there are date conversions in the debug call.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", KINESIS_STREAM_NAME);
+    }};
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+    private volatile AWSCredentials awsCredentials;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),

Review comment:
       Typo: parseable (?)




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628381717



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");
+        }
+
+        return kinesisClientLibConfiguration;
+    }
+
+    Worker.Builder prepareWorkerBuilder(final ProcessContext context, final KinesisClientLibConfiguration kinesisClientLibConfiguration,
+                                        final IRecordProcessorFactory factory) {
+        final Worker.Builder workerBuilder = new Worker.Builder()
+                .config(kinesisClientLibConfiguration)
+                .kinesisClient(getClient())
+                .recordProcessorFactory(factory);
+
+        if (!isReportCloudWatchMetrics(context)) {
+            workerBuilder.metricsFactory(new NullMetricsFactory());
+        }
+
+        return workerBuilder;
+    }
+
+    private boolean shutdownWorker(final ProcessContext context) {
+        boolean success = true;
+        try {
+            if (!worker.hasGracefulShutdownStarted()) {
+                getLogger().info("Requesting Kinesis Worker shutdown");
+                final Future<Boolean> shutdown = worker.startGracefulShutdown();
+                // allow 2 seconds longer than the graceful period for shutdown before cancelling the task
+                if (Boolean.FALSE.equals(shutdown.get(getGracefulShutdownMillis(context) + 2_000L, TimeUnit.MILLISECONDS))) {
+                    getLogger().warn("Kinesis Worker shutdown did not complete in time, cancelling");
+                    success = false;
+                } else {
+                    getLogger().info("Kinesis Worker shutdown");
+                }
+            }
+        } catch (@SuppressWarnings("java:S2142") InterruptedException | TimeoutException | ExecutionException e) {
+            getLogger().warn("Problem while shutting down Kinesis Worker: {}", e.getLocalizedMessage(), e);
+            success = false;
+        }
+        return success;
+    }
+
+    private String generateWorkerId() {
+        try {
+            return InetAddress.getLocalHost().getCanonicalHostName() + ":" + UUID.randomUUID();
+        } catch (UnknownHostException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    private String getApplicationName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(APPLICATION_NAME).getValue());
+    }
+
+    private String getStreamName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(KINESIS_STREAM_NAME).getValue());
+    }
+
+    private long getFailoverTimeMillis(final PropertyContext context) {
+        return context.getProperty(FAILOVER_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getGracefulShutdownMillis(final PropertyContext context) {
+        return context.getProperty(GRACEFUL_SHUTDOWN_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getCheckpointIntervalMillis(final PropertyContext context) {
+        return context.getProperty(CHECKPOINT_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private int getNumRetries(final PropertyContext context) {
+        return context.getProperty(NUM_RETRIES).asInteger();
+    }
+
+    private long getRetryWaitMillis(final PropertyContext context) {
+        return context.getProperty(RETRY_WAIT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private boolean isReportCloudWatchMetrics(final PropertyContext context) {
+        return context.getProperty(REPORT_CLOUDWATCH_METRICS).asBoolean();
+    }
+
+    private Optional<String> getKinesisEndpoint(final PropertyContext context) {
+        return context.getProperty(ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private Optional<String> getDynamoDBOverride(final PropertyContext context) {
+        return context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private RecordReaderFactory getReaderFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+    }
+
+    private RecordSetWriterFactory getWriterFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+    }
+
+    private InitialPositionInStream getInitialPositionInStream(final PropertyContext context) {
+        return InitialPositionInStream.valueOf(StringUtils.trimToEmpty(context.getProperty(INITIAL_STREAM_POSITION).getValue()));
+    }
+
+    private DateTimeFormatter getDateTimeFormatter(final PropertyContext context) {
+        return DateTimeFormatter.ofPattern(context.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context) {
+        return getStartStreamTimestamp(context, getDateTimeFormatter(context));
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context, final DateTimeFormatter dateTimeFormatter) {
+        final String streamTimestamp = context.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+        return new Date(LocalDateTime.parse(streamTimestamp, dateTimeFormatter).toInstant(ZoneOffset.UTC).toEpochMilli());

Review comment:
       Think I've done this (like most people... I dislike working with timezones, so easy to get things wrong!)




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628256440



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");

Review comment:
       I don't think the `AtomicBoolean` trick is needed here but `ProcessException` can simply be thrown from the `catch` above.
   It would log only the first error but practically there cannot be any error here because dynamic properties have already been checked by the validator.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] auyer commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
auyer commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-789986473


   Hi ! I'm interested in testing this out.
   I currently run Apache Nifi using the base Docker images. How can I proceed to include the Kinesis Get Processor into my images? 
   Thanks ! Hope I can help!


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628731292



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");
+        }
+
+        return kinesisClientLibConfiguration;
+    }
+
+    Worker.Builder prepareWorkerBuilder(final ProcessContext context, final KinesisClientLibConfiguration kinesisClientLibConfiguration,
+                                        final IRecordProcessorFactory factory) {
+        final Worker.Builder workerBuilder = new Worker.Builder()
+                .config(kinesisClientLibConfiguration)
+                .kinesisClient(getClient())
+                .recordProcessorFactory(factory);
+
+        if (!isReportCloudWatchMetrics(context)) {
+            workerBuilder.metricsFactory(new NullMetricsFactory());
+        }
+
+        return workerBuilder;
+    }
+
+    private boolean shutdownWorker(final ProcessContext context) {
+        boolean success = true;
+        try {
+            if (!worker.hasGracefulShutdownStarted()) {
+                getLogger().info("Requesting Kinesis Worker shutdown");
+                final Future<Boolean> shutdown = worker.startGracefulShutdown();
+                // allow 2 seconds longer than the graceful period for shutdown before cancelling the task
+                if (Boolean.FALSE.equals(shutdown.get(getGracefulShutdownMillis(context) + 2_000L, TimeUnit.MILLISECONDS))) {
+                    getLogger().warn("Kinesis Worker shutdown did not complete in time, cancelling");
+                    success = false;
+                } else {
+                    getLogger().info("Kinesis Worker shutdown");
+                }
+            }
+        } catch (@SuppressWarnings("java:S2142") InterruptedException | TimeoutException | ExecutionException e) {
+            getLogger().warn("Problem while shutting down Kinesis Worker: {}", e.getLocalizedMessage(), e);
+            success = false;
+        }
+        return success;
+    }
+
+    private String generateWorkerId() {
+        try {
+            return InetAddress.getLocalHost().getCanonicalHostName() + ":" + UUID.randomUUID();
+        } catch (UnknownHostException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    private String getApplicationName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(APPLICATION_NAME).getValue());
+    }
+
+    private String getStreamName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(KINESIS_STREAM_NAME).getValue());
+    }
+
+    private long getFailoverTimeMillis(final PropertyContext context) {
+        return context.getProperty(FAILOVER_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getGracefulShutdownMillis(final PropertyContext context) {
+        return context.getProperty(GRACEFUL_SHUTDOWN_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getCheckpointIntervalMillis(final PropertyContext context) {
+        return context.getProperty(CHECKPOINT_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private int getNumRetries(final PropertyContext context) {
+        return context.getProperty(NUM_RETRIES).asInteger();
+    }
+
+    private long getRetryWaitMillis(final PropertyContext context) {
+        return context.getProperty(RETRY_WAIT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private boolean isReportCloudWatchMetrics(final PropertyContext context) {
+        return context.getProperty(REPORT_CLOUDWATCH_METRICS).asBoolean();
+    }
+
+    private Optional<String> getKinesisEndpoint(final PropertyContext context) {
+        return context.getProperty(ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private Optional<String> getDynamoDBOverride(final PropertyContext context) {
+        return context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private RecordReaderFactory getReaderFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+    }
+
+    private RecordSetWriterFactory getWriterFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+    }
+
+    private InitialPositionInStream getInitialPositionInStream(final PropertyContext context) {
+        return InitialPositionInStream.valueOf(StringUtils.trimToEmpty(context.getProperty(INITIAL_STREAM_POSITION).getValue()));
+    }
+
+    private DateTimeFormatter getDateTimeFormatter(final PropertyContext context) {
+        return DateTimeFormatter.ofPattern(context.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context) {
+        return getStartStreamTimestamp(context, getDateTimeFormatter(context));
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context, final DateTimeFormatter dateTimeFormatter) {
+        final String streamTimestamp = context.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+        return new Date(LocalDateTime.parse(streamTimestamp, dateTimeFormatter).toInstant(ZoneOffset.UTC).toEpochMilli());

Review comment:
       I think I'll leave this as-is because it's a bit more explicit what's going on




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-809686439


   @turcsanyip updated per your suggestions.
   
   Note that this includes:
   - added (optional) Record Reader/Writer (@pvillard31 FYI)
   - better Thread/Session handling for the use of Kinesis Client Library
   - reverted to KCL verison 1.13.3 due to [amazon-kinesis-client#796](https://github.com/awslabs/amazon-kinesis-client/issues/796)
   - removed double-validation of dynamic properties (introduced by NIFI-8266 @mattyb149 FYI in case you see an issue with my change to `AbstractConfigurableComponent`)
   - change to `AbstractProcessor` to remove the `final` modifier of the `onTrigger` method that takes the `ProcessorSessionFactory` - by default processors extending this class will still need to implement the `onTrigger` method that takes the `ProcessSession`, but can now optionally override the former (this seemed a better option in the end than copying the contents of `AbstractProcessor` into `AbstractAWSProcessor`, but happy for that to be challenged)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r629948591



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,704 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "consume", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")

Review comment:
       @ChrisSamo632 I did some kind of performance tests with 1 Mio messages in 12 shards and running NiFi on my local machine (no cluster). It took 64-74 seconds with default settings (no additional dynamic properties).
   
   The interesting thing that I could not see any difference when I set Checkpoint Interval to 0 sec (that is checkpointing "synchronously" after each bunch of messages received in `IRecordProcessor.processRecords()` callback). It seems there is no significant overhead of checkpointing more frequently (and it has the advantage of having fewer duplicated messages in case of restart).
   
   It can be investigated further in a follow-up ticket with more sophisticated performance tests (nifi cluster, non-local machine, tuning KCL properties, etc) and the default can be adjusted if that is reasonable.
   
   




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628444152



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))

Review comment:
       It looks good to me now. Thanks!
   There is one more "with-" prefix left in `buildDynamicPropertyBeanValidationResult()`.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] auyer edited a comment on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
auyer edited a comment on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-795907028


   I've been testing this for a few days, and had no issues so far. I should say that I'm not running NiFi in production yet, but plan to do so soon, and this feature helps a lot !
   If anyone else wants to try it in Docker, I've built a image following the instructions above, and published it publicly in [DockeHub](https://hub.docker.com/r/auyer/apache-nifi-with-get-kinesis). 
   
   Note: I will remove this repo after this PR gets into the next release.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-808562542


   @ChrisSamo632 Thanks for picking up this task and implementing this non-trivial processor!
   
   I did not review the code in detail (yet) but spotted some issues regarding thread handling:
   
   The `onTrigger()` method should not block its thread. In NiFi, there is a shared thread pool for running all processors' `onTrigger()` methods. If some processors hold up execution threads, it can lead to starvation of other processors.
   I think a similar pattern could be used as in case of `ConsumeAzureEventHub`: start up the Worker at the first execution of `onTrigger()` and then just yield in the subsequent calls.
   
   I believe one Worker per processor should be enough and it is not necessary to maintain a pool of workers. A single Worker can run multiple threads for executing the Kinesis RecordProcessors. As far as I saw, the Worker spins up a thread for each assigned shard by default (RecordProcessor-xxxx threads). So the parallel processing is provided with one Worker too. The code is simpler in this way and there is less overhead at runtime (each Worker has its own "maintenance" threads like LeaseRenewer-xxxx, LeaseCoordinator-xxxx).
   
   The processor cannot stop cleanly due to a bug in KCL. Interestingly, it has just been reported to AWS by someone else: https://github.com/awslabs/amazon-kinesis-client/issues/796
   When the processor shuts down the Worker, the Worker leaves behind a thread running. Stopping/starting the processor multiple time would lead to thread leaking. Furthermore, the "zombie" thread(s) prevent NiFi to shut down properly:
   `2021-03-26 12:12:52,715 WARN [main] org.apache.nifi.bootstrap.Command NiFi has not finished shutting down after 20 seconds. Killing process.`
   For this reason, I think we need to downgrade the KCL version to 1.13.3 until the bug is fixed.
   
   And an idea: the way the processor receives data via KCL is quite similar to other Consume*** processors (like the already mentioned `ConsumeAzureEventHub` or `ConsumeAMQP`), so I would consider to name the processor `ConsumeKinesisStream` instead of `Get~` (actually there is a Get-like/polling API for Kinesis too and another processor may be implemented later using that API).
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627455807



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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 are are not released until this processor is stopped.")

Review comment:
       D'oh




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627181783



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/AbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.StopWatch;
+
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor {
+    public static final String AWS_KINESIS_SHARD_ID = "aws.kinesis.shard.id";
+
+    public static final String AWS_KINESIS_SEQUENCE_NUMBER = "aws.kinesis.sequence.number";
+
+    public static final String AWS_KINESIS_PARTITION_KEY = "aws.kinesis.partition.key";
+
+    public static final String AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP = "aws.kinesis.approximate.arrival.timestamp";
+
+    public static final String KINESIS_RECORD_SCHEMA_KEY = "kinesis.name";
+
+    static final Base64.Encoder BASE_64_ENCODER = Base64.getEncoder();
+
+    final ProcessSessionFactory sessionFactory;
+    final ComponentLog log;
+    final String streamName;
+    final String endpointPrefix;
+
+    final long checkpointIntervalMillis;
+    final long retryWaitMillis;
+    final int numRetries;
+    final DateTimeFormatter dateTimeFormatter;
+
+    String kinesisShardId;
+    long nextCheckpointTimeInMillis;
+
+    boolean processingRecords = false;
+
+    @SuppressWarnings("java:S107")
+    AbstractKinesisRecordProcessor(final ProcessSessionFactory sessionFactory, final ComponentLog log, final String streamName,
+                                   final String endpointPrefix, final long checkpointIntervalMillis, final long retryWaitMillis,
+                                   final int numRetries, final DateTimeFormatter dateTimeFormatter) {
+        this.sessionFactory = sessionFactory;
+        this.log = log;
+        this.streamName = streamName;
+        this.endpointPrefix = endpointPrefix;
+        this.checkpointIntervalMillis = checkpointIntervalMillis;
+        this.retryWaitMillis = retryWaitMillis;
+        this.numRetries = numRetries;
+        this.dateTimeFormatter = dateTimeFormatter;
+    }
+
+    @Override
+    public void initialize(final InitializationInput initializationInput) {
+        if (initializationInput.getPendingCheckpointSequenceNumber() != null) {
+            log.warn("Initializing record processor for stream: {} / shard {}; from sequence number: {}; indicates previously uncheckpointed sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber(), initializationInput.getPendingCheckpointSequenceNumber());
+        } else {
+            log.debug("Initializing record processor for stream: {} / shard: {}; from sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber());
+        }
+
+        this.kinesisShardId = initializationInput.getShardId();
+
+        // ensure we don't immediately checkpoint
+        this.nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
+    }
+
+    @SuppressWarnings({"java:S3252"}) // ConsumeKinesisStream reference to REL_SUCCESS instead of deprecated AbstractAWSProcessor
+    @Override
+    public void processRecords(final ProcessRecordsInput processRecordsInput) {
+        log.debug("Processing {} records from {}; cache entry: {}; cache exit: {}; millis behind latest: {}",
+                processRecordsInput.getRecords().size(), kinesisShardId,
+                processRecordsInput.getCacheEntryTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheEntryTime().atZone(ZoneId.systemDefault())) : null,
+                processRecordsInput.getCacheExitTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheExitTime().atZone(ZoneId.systemDefault())) : null,
+                processRecordsInput.getMillisBehindLatest());

Review comment:
       Tried to catch these, but evidently missed one... good spot




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628448909



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");
+        }
+
+        return kinesisClientLibConfiguration;
+    }
+
+    Worker.Builder prepareWorkerBuilder(final ProcessContext context, final KinesisClientLibConfiguration kinesisClientLibConfiguration,
+                                        final IRecordProcessorFactory factory) {
+        final Worker.Builder workerBuilder = new Worker.Builder()
+                .config(kinesisClientLibConfiguration)
+                .kinesisClient(getClient())
+                .recordProcessorFactory(factory);
+
+        if (!isReportCloudWatchMetrics(context)) {
+            workerBuilder.metricsFactory(new NullMetricsFactory());
+        }
+
+        return workerBuilder;
+    }
+
+    private boolean shutdownWorker(final ProcessContext context) {
+        boolean success = true;
+        try {
+            if (!worker.hasGracefulShutdownStarted()) {
+                getLogger().info("Requesting Kinesis Worker shutdown");
+                final Future<Boolean> shutdown = worker.startGracefulShutdown();
+                // allow 2 seconds longer than the graceful period for shutdown before cancelling the task
+                if (Boolean.FALSE.equals(shutdown.get(getGracefulShutdownMillis(context) + 2_000L, TimeUnit.MILLISECONDS))) {
+                    getLogger().warn("Kinesis Worker shutdown did not complete in time, cancelling");
+                    success = false;
+                } else {
+                    getLogger().info("Kinesis Worker shutdown");
+                }
+            }
+        } catch (@SuppressWarnings("java:S2142") InterruptedException | TimeoutException | ExecutionException e) {
+            getLogger().warn("Problem while shutting down Kinesis Worker: {}", e.getLocalizedMessage(), e);
+            success = false;
+        }
+        return success;
+    }
+
+    private String generateWorkerId() {
+        try {
+            return InetAddress.getLocalHost().getCanonicalHostName() + ":" + UUID.randomUUID();
+        } catch (UnknownHostException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    private String getApplicationName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(APPLICATION_NAME).getValue());
+    }
+
+    private String getStreamName(final PropertyContext context) {
+        return StringUtils.trimToEmpty(context.getProperty(KINESIS_STREAM_NAME).getValue());
+    }
+
+    private long getFailoverTimeMillis(final PropertyContext context) {
+        return context.getProperty(FAILOVER_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getGracefulShutdownMillis(final PropertyContext context) {
+        return context.getProperty(GRACEFUL_SHUTDOWN_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private long getCheckpointIntervalMillis(final PropertyContext context) {
+        return context.getProperty(CHECKPOINT_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private int getNumRetries(final PropertyContext context) {
+        return context.getProperty(NUM_RETRIES).asInteger();
+    }
+
+    private long getRetryWaitMillis(final PropertyContext context) {
+        return context.getProperty(RETRY_WAIT).asTimePeriod(TimeUnit.MILLISECONDS);
+    }
+
+    private boolean isReportCloudWatchMetrics(final PropertyContext context) {
+        return context.getProperty(REPORT_CLOUDWATCH_METRICS).asBoolean();
+    }
+
+    private Optional<String> getKinesisEndpoint(final PropertyContext context) {
+        return context.getProperty(ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private Optional<String> getDynamoDBOverride(final PropertyContext context) {
+        return context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).isSet()
+                ? Optional.of(StringUtils.trimToEmpty(context.getProperty(DYNAMODB_ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()))
+                : Optional.empty();
+    }
+
+    private RecordReaderFactory getReaderFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+    }
+
+    private RecordSetWriterFactory getWriterFactory(final PropertyContext context) {
+        return context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+    }
+
+    private InitialPositionInStream getInitialPositionInStream(final PropertyContext context) {
+        return InitialPositionInStream.valueOf(StringUtils.trimToEmpty(context.getProperty(INITIAL_STREAM_POSITION).getValue()));
+    }
+
+    private DateTimeFormatter getDateTimeFormatter(final PropertyContext context) {
+        return DateTimeFormatter.ofPattern(context.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context) {
+        return getStartStreamTimestamp(context, getDateTimeFormatter(context));
+    }
+
+    private Date getStartStreamTimestamp(final PropertyContext context, final DateTimeFormatter dateTimeFormatter) {
+        final String streamTimestamp = context.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+        return new Date(LocalDateTime.parse(streamTimestamp, dateTimeFormatter).toInstant(ZoneOffset.UTC).toEpochMilli());

Review comment:
       It works properly now with local system time zone. Thanks
   
   A minor note: `withZoneSameInstant(ZoneOffset.UTC)` is not necessarily needed, because in the end we need the Instant and this method does not change the Instant (and for this reason it does not cause any problem either).




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627477091



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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 are 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()

Review comment:
       Similar to "Communications Timeout" property, "Checkpoint Interval" and "Retry Wait" should support other time periods too and it would be enough to convert the period to millisec in the code where it is used.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip edited a comment on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip edited a comment on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-808562542


   @ChrisSamo632 Thanks for picking up this task and implementing this non-trivial processor!
   
   I did not review the code in detail (yet) but spotted some issues regarding thread handling:
   
   The `onTrigger()` method should not block its thread. In NiFi, there is a shared thread pool for executing all processors' `onTrigger()` methods. If some processors held up execution threads, it could lead to starvation of other processors.
   I think a similar pattern could be used as in case of `ConsumeAzureEventHub`: start up the Worker at the first execution of `onTrigger()` and then just yield in the subsequent calls.
   
   I believe one Worker per processor should be enough and it is not necessary to maintain a pool of workers. A single Worker can run multiple threads for executing the Kinesis RecordProcessors. As far as I saw, the Worker spins up a thread for each assigned shard by default (RecordProcessor-xxxx threads). So the parallel processing is provided with one Worker too. The code is simpler in this way and there is less overhead at runtime (each Worker has its own "maintenance" threads like LeaseRenewer-xxxx, LeaseCoordinator-xxxx).
   
   The processor cannot stop cleanly due to a bug in KCL. Interestingly, it has just been reported to AWS by someone else: https://github.com/awslabs/amazon-kinesis-client/issues/796
   When the processor shuts down the Worker, the Worker leaves behind a thread running. Stopping/starting the processor multiple times would lead to thread leaking. Furthermore, the "zombie" thread(s) prevent NiFi to shut down properly:
   `2021-03-26 12:12:52,715 WARN [main] org.apache.nifi.bootstrap.Command NiFi has not finished shutting down after 20 seconds. Killing process.`
   For this reason, I think we need to downgrade the KCL version to 1.13.3 until the bug is fixed.
   
   And an idea: the way the processor receives data via KCL is quite similar to other Consume*** processors (like the already mentioned `ConsumeAzureEventHub` or `ConsumeAMQP`), so I would consider to name the processor `ConsumeKinesisStream` instead of `Get~` (actually there is a Get-like/polling API for Kinesis too and another processor may be implemented later using that API).
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627997264



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {

Review comment:
       Think I added the booleans after writing this validation logic and didn't think to go back later, good spot




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r626386412



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings("java:S110")
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)

Review comment:
       Added




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628080211



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            final String workerId = generateWorkerId();
+            getLogger().info("Starting Kinesis Worker {}", workerId);
+            worker = prepareWorker(context, sessionFactory, workerId);
+            new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            final boolean success = shutdownWorker();
+            worker = null;
+            readerFactory = null;
+            writerFactory = null;
+
+            if (!success) {
+                getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                context.yield();
+            }
+        }
+    }
+
+    private Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final String appName = getApplicationName(context);
+        final String streamName = getStreamName(context);
+        final long checkpointIntervalMillis = getCheckpointIntervalMillis(context);
+        final String kinesisEndpoint = getKinesisOverride(context).orElse(null);
+
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(sessionFactory, streamName,
+                checkpointIntervalMillis, kinesisEndpoint);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = prepareKinesisClientLibConfiguration(
+                context, appName, streamName, workerId, kinesisEndpoint);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(kinesisClientLibConfiguration, factory, context);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                appName, streamName, workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessSessionFactory sessionFactory, final String streamName,
+                                                                  final long checkpointIntervalMillis, final String kinesisEndpoint) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter, readerFactory, writerFactory
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String appName,
+                                                                       final String streamName, final String workerId,
+                                                                       final String kinesisEndpoint) {
+        final AWSCredentialsProvider credentialsProvider = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).isSet()

Review comment:
       So it does, you're quite right - and it does, indeed, result in the same thing




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627252972



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/AbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.StopWatch;
+
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor {
+    public static final String AWS_KINESIS_SHARD_ID = "aws.kinesis.shard.id";
+
+    public static final String AWS_KINESIS_SEQUENCE_NUMBER = "aws.kinesis.sequence.number";
+
+    public static final String AWS_KINESIS_PARTITION_KEY = "aws.kinesis.partition.key";
+
+    public static final String AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP = "aws.kinesis.approximate.arrival.timestamp";
+
+    public static final String KINESIS_RECORD_SCHEMA_KEY = "kinesis.name";
+
+    static final Base64.Encoder BASE_64_ENCODER = Base64.getEncoder();
+
+    final ProcessSessionFactory sessionFactory;
+    final ComponentLog log;
+    final String streamName;
+    final String endpointPrefix;
+
+    final long checkpointIntervalMillis;
+    final long retryWaitMillis;
+    final int numRetries;
+    final DateTimeFormatter dateTimeFormatter;
+
+    String kinesisShardId;
+    long nextCheckpointTimeInMillis;
+
+    boolean processingRecords = false;

Review comment:
       Think I'd missed updating some of these from a previous incarnation.
   
   I thought default (package-private) would be better than `protected` because everything that uses these properties are within the same package and I don't (currently) anticipate any classes to extend this abstract class in other packages (without also updating `ConsumeKinesisStream` itself and also then they might as well add `protected` modifier here).
   
   Not `private` because the fields are used by the concrete classes (and unit tests)... but I was being lazy and not generating getter/setter methods, so I've made things `private` and added appropriate getter/setters where needed.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627679111



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);

Review comment:
       Why is `streamTimestamp` parsed twice? I believe this line is unnecessary.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)

Review comment:
       `streamTimestamp` variable could be used instead of getting the value from the context again.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/AbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor {
+    public static final String AWS_KINESIS_SHARD_ID = "aws.kinesis.shard.id";
+
+    public static final String AWS_KINESIS_SEQUENCE_NUMBER = "aws.kinesis.sequence.number";
+
+    public static final String AWS_KINESIS_PARTITION_KEY = "aws.kinesis.partition.key";
+
+    public static final String AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP = "aws.kinesis.approximate.arrival.timestamp";
+
+    public static final String KINESIS_RECORD_SCHEMA_KEY = "kinesis.name";
+
+    static final Base64.Encoder BASE_64_ENCODER = Base64.getEncoder();
+
+    private final ProcessSessionFactory sessionFactory;
+    private final ComponentLog log;
+    private final String streamName;
+    private final String transitUriPrefix;
+
+    private final long checkpointIntervalMillis;
+    private final long retryWaitMillis;
+    private final int numRetries;
+    private final DateTimeFormatter dateTimeFormatter;
+
+    private String kinesisShardId;
+    private long nextCheckpointTimeInMillis;
+
+    private boolean processingRecords = false;
+
+    @SuppressWarnings("java:S107")
+    AbstractKinesisRecordProcessor(final ProcessSessionFactory sessionFactory, final ComponentLog log, final String streamName,
+                                   final String endpointPrefix, final String kinesisEndpoint,
+                                   final long checkpointIntervalMillis, final long retryWaitMillis,
+                                   final int numRetries, final DateTimeFormatter dateTimeFormatter) {
+        this.sessionFactory = sessionFactory;
+        this.log = log;
+        this.streamName = streamName;
+        this.checkpointIntervalMillis = checkpointIntervalMillis;
+        this.retryWaitMillis = retryWaitMillis;
+        this.numRetries = numRetries;
+        this.dateTimeFormatter = dateTimeFormatter;
+
+        this.transitUriPrefix = StringUtils.isBlank(kinesisEndpoint) ? String.format("http://%s.amazonaws.com", endpointPrefix) : kinesisEndpoint;
+    }
+
+    @Override
+    public void initialize(final InitializationInput initializationInput) {
+        if (initializationInput.getPendingCheckpointSequenceNumber() != null) {
+            log.warn("Initializing record processor for stream: {} / shard {}; from sequence number: {}; indicates previously uncheckpointed sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber(), initializationInput.getPendingCheckpointSequenceNumber());
+        } else {
+            log.debug("Initializing record processor for stream: {} / shard: {}; from sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber());
+        }
+
+        this.kinesisShardId = initializationInput.getShardId();
+
+        // ensure we don't immediately checkpoint
+        this.nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
+    }
+
+    @SuppressWarnings({"java:S3252"}) // ConsumeKinesisStream reference to REL_SUCCESS instead of deprecated AbstractAWSProcessor
+    @Override
+    public void processRecords(final ProcessRecordsInput processRecordsInput) {
+        if (log.isDebugEnabled()) {
+            log.debug("Processing {} records from {}; cache entry: {}; cache exit: {}; millis behind latest: {}",
+                    processRecordsInput.getRecords().size(), kinesisShardId,
+                    processRecordsInput.getCacheEntryTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheEntryTime().atZone(ZoneId.systemDefault())) : null,
+                    processRecordsInput.getCacheExitTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheExitTime().atZone(ZoneId.systemDefault())) : null,
+                    processRecordsInput.getMillisBehindLatest());
+        }
+
+        ProcessSession session = null;
+        try {
+            final List<Record> records = processRecordsInput.getRecords();
+            if (!records.isEmpty()) {
+                final List<FlowFile> flowFiles = new ArrayList<>(records.size());
+                final StopWatch stopWatch = new StopWatch(true);
+                session = sessionFactory.createSession();
+
+                startProcessingRecords();
+                final int recordsTransformed = processRecordsWithRetries(records, flowFiles, session, stopWatch);
+                transferTo(ConsumeKinesisStream.REL_SUCCESS, session, records.size(), recordsTransformed, flowFiles);
+                session.commit();
+                processingRecords = false;
+
+                // if creating an Kinesis checkpoint fails, then the same record(s) can be retrieved again
+                checkpointOnceEveryCheckpointInterval(processRecordsInput.getCheckpointer());
+            }
+        } catch (final Exception e) {
+            log.error("Unable to fully process received Kinesis record(s) due to {}", e.getLocalizedMessage(), e);
+            // FlowFiles that are already committed will not get rolled back
+            if (session != null) {
+                session.rollback();
+            }
+        }
+    }
+
+    void startProcessingRecords() {
+        processingRecords = true;
+    }
+
+    private int processRecordsWithRetries(final List<Record> records, final List<FlowFile> flowFiles,
+                                           final ProcessSession session, final StopWatch stopWatch) {
+        int recordsTransformed = 0;
+        for (int r = 0; r < records.size(); r++) {
+            final Record kinesisRecord = records.get(r);
+            boolean processedSuccessfully = false;
+            for (int i = 0; !processedSuccessfully && i < numRetries; i++) {
+                processedSuccessfully = attemptProcessRecord(flowFiles, kinesisRecord, r == records.size() - 1, session, stopWatch);
+            }
+
+            if (processedSuccessfully) {
+                recordsTransformed++;
+            } else {
+                log.error("Couldn't process Kinesis record {}, skipping.", kinesisRecord);
+            }
+        }
+
+        return recordsTransformed;
+    }
+
+    private boolean attemptProcessRecord(final List<FlowFile> flowFiles, final Record kinesisRecord, final boolean lastRecord,
+                                         final ProcessSession session, final StopWatch stopWatch) {
+        boolean processedSuccessfully = false;
+        try {
+            processRecord(flowFiles, kinesisRecord, lastRecord, session, stopWatch);
+            processedSuccessfully = true;
+        } catch (final Exception e) {
+            log.error("Caught Exception while processing Kinesis record {}", kinesisRecord, e);
+
+            // backoff if we encounter an exception.
+            try {
+                Thread.sleep(retryWaitMillis);
+            } catch (@SuppressWarnings("java:S2142") InterruptedException ie) {
+                log.debug("Interrupted sleep during record processing back-off", ie);
+            }
+        }
+
+        return processedSuccessfully;
+    }
+
+    /**
+     * Process an individual {@link Record} and serialise to {@link FlowFile}
+     *
+     * @param flowFiles {@link List} of {@link FlowFile}s to be output after all processing is complete
+     * @param kinesisRecord the Kinesis {@link Record} to be processed
+     * @param lastRecord whether this is the last {@link Record} to be processed in this batch
+     * @param session {@link ProcessSession} into which {@link FlowFile}s will be transferred
+     * @param stopWatch {@link StopWatch} tracking how much time has been spent processing the current batch
+     *
+     * @throws RuntimeException if there are any unhandled Exceptions that should be retried
+     */
+    abstract void processRecord(final List<FlowFile> flowFiles, final Record kinesisRecord, final boolean lastRecord,
+                                final ProcessSession session, final StopWatch stopWatch);
+
+    void reportProvenance(final ProcessSession session, final FlowFile flowFile, final String partitionKey,
+                 final String sequenceNumber, final StopWatch stopWatch) {
+        final String transitUri = StringUtils.isNotBlank(partitionKey) && StringUtils.isNotBlank(sequenceNumber)
+                ? String.format("%s/%s/%s#%s", transitUriPrefix, kinesisShardId, partitionKey, sequenceNumber)
+                : String.format("%s/%s", transitUriPrefix, kinesisShardId);
+
+        session.getProvenanceReporter().receive(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+    }
+
+    Map<String, String> getDefaultAttributes(final String sequenceNumber, final String partitionKey, final Date approximateArrivalTimestamp) {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put(AWS_KINESIS_SHARD_ID, kinesisShardId);
+        attributes.put(AWS_KINESIS_SEQUENCE_NUMBER, sequenceNumber);
+        attributes.put(AWS_KINESIS_PARTITION_KEY, partitionKey);
+        if (approximateArrivalTimestamp != null) {
+            attributes.put(AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                    dateTimeFormatter.format(approximateArrivalTimestamp.toInstant().atZone(ZoneId.systemDefault())));
+        }
+        return attributes;
+    }
+
+    void transferTo(final Relationship relationship, final ProcessSession session, final int recordsProcessed,
+                    final int recordsTransformed, final List<FlowFile> flowFiles) {
+        session.adjustCounter("Records Processed", recordsProcessed, false);
+        if (!flowFiles.isEmpty()) {
+            session.adjustCounter("Records Transformed", recordsTransformed, false);
+            session.transfer(flowFiles, relationship);
+        }
+    }
+
+    private void checkpointOnceEveryCheckpointInterval(final IRecordProcessorCheckpointer checkpointer) {
+        if (System.currentTimeMillis() > nextCheckpointTimeInMillis) {
+            checkpointWithRetries(checkpointer);
+            nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
+        }
+    }
+
+    @Override
+    public void shutdown(final ShutdownInput shutdownInput) {
+        log.debug("Shutting down record processor for shard: {} with reason: {}", kinesisShardId, shutdownInput.getShutdownReason());
+
+        // be sure to finish processing any records before shutdown on TERMINATE
+        if (ShutdownReason.TERMINATE == shutdownInput.getShutdownReason()) {
+            for (int i = 0; processingRecords && i < numRetries; i++) {
+                log.debug("Record Processor for shard {} still processing records, waiting before shutdown", kinesisShardId);
+                try {
+                    Thread.sleep(retryWaitMillis);
+                } catch (@SuppressWarnings("java:S2142") InterruptedException ie) {
+                    log.debug("Interrupted sleep while waiting for record processing to complete before shutdown (TERMINATE)", ie);
+                }
+            }

Review comment:
       I believe it should be logged when the `for` loop exited due to exhausting `numRetries` instead of `processingRecords` being `false`.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {

Review comment:
       `boolean` variables could simply be used because the controller service objects are not used for other purposes.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));

Review comment:
       Validate methods typically just validate the properties but do not have any side effects (I would not say it is a convention but never seen it).
   I would suggest to define getter methods for these properties too (like for the other ones) and call them from the validator and also where the property value is used (onTrigger or so).
   I believe `PropertyContext` can be used as parameter type of the getter and then both `ValidationContext` and `ProcessContext` arguments can be passed in.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/AbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor {
+    public static final String AWS_KINESIS_SHARD_ID = "aws.kinesis.shard.id";
+
+    public static final String AWS_KINESIS_SEQUENCE_NUMBER = "aws.kinesis.sequence.number";
+
+    public static final String AWS_KINESIS_PARTITION_KEY = "aws.kinesis.partition.key";
+
+    public static final String AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP = "aws.kinesis.approximate.arrival.timestamp";
+
+    public static final String KINESIS_RECORD_SCHEMA_KEY = "kinesis.name";
+
+    static final Base64.Encoder BASE_64_ENCODER = Base64.getEncoder();
+
+    private final ProcessSessionFactory sessionFactory;
+    private final ComponentLog log;
+    private final String streamName;
+    private final String transitUriPrefix;
+
+    private final long checkpointIntervalMillis;
+    private final long retryWaitMillis;
+    private final int numRetries;
+    private final DateTimeFormatter dateTimeFormatter;
+
+    private String kinesisShardId;
+    private long nextCheckpointTimeInMillis;
+
+    private boolean processingRecords = false;
+
+    @SuppressWarnings("java:S107")
+    AbstractKinesisRecordProcessor(final ProcessSessionFactory sessionFactory, final ComponentLog log, final String streamName,
+                                   final String endpointPrefix, final String kinesisEndpoint,
+                                   final long checkpointIntervalMillis, final long retryWaitMillis,
+                                   final int numRetries, final DateTimeFormatter dateTimeFormatter) {
+        this.sessionFactory = sessionFactory;
+        this.log = log;
+        this.streamName = streamName;
+        this.checkpointIntervalMillis = checkpointIntervalMillis;
+        this.retryWaitMillis = retryWaitMillis;
+        this.numRetries = numRetries;
+        this.dateTimeFormatter = dateTimeFormatter;
+
+        this.transitUriPrefix = StringUtils.isBlank(kinesisEndpoint) ? String.format("http://%s.amazonaws.com", endpointPrefix) : kinesisEndpoint;
+    }
+
+    @Override
+    public void initialize(final InitializationInput initializationInput) {
+        if (initializationInput.getPendingCheckpointSequenceNumber() != null) {
+            log.warn("Initializing record processor for stream: {} / shard {}; from sequence number: {}; indicates previously uncheckpointed sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber(), initializationInput.getPendingCheckpointSequenceNumber());
+        } else {
+            log.debug("Initializing record processor for stream: {} / shard: {}; from sequence number: {}",
+                    streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber());
+        }
+
+        this.kinesisShardId = initializationInput.getShardId();
+
+        // ensure we don't immediately checkpoint
+        this.nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
+    }
+
+    @SuppressWarnings({"java:S3252"}) // ConsumeKinesisStream reference to REL_SUCCESS instead of deprecated AbstractAWSProcessor
+    @Override
+    public void processRecords(final ProcessRecordsInput processRecordsInput) {
+        if (log.isDebugEnabled()) {
+            log.debug("Processing {} records from {}; cache entry: {}; cache exit: {}; millis behind latest: {}",
+                    processRecordsInput.getRecords().size(), kinesisShardId,
+                    processRecordsInput.getCacheEntryTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheEntryTime().atZone(ZoneId.systemDefault())) : null,
+                    processRecordsInput.getCacheExitTime() != null ? dateTimeFormatter.format(processRecordsInput.getCacheExitTime().atZone(ZoneId.systemDefault())) : null,
+                    processRecordsInput.getMillisBehindLatest());
+        }
+
+        ProcessSession session = null;
+        try {
+            final List<Record> records = processRecordsInput.getRecords();
+            if (!records.isEmpty()) {
+                final List<FlowFile> flowFiles = new ArrayList<>(records.size());
+                final StopWatch stopWatch = new StopWatch(true);
+                session = sessionFactory.createSession();
+
+                startProcessingRecords();
+                final int recordsTransformed = processRecordsWithRetries(records, flowFiles, session, stopWatch);
+                transferTo(ConsumeKinesisStream.REL_SUCCESS, session, records.size(), recordsTransformed, flowFiles);
+                session.commit();
+                processingRecords = false;
+
+                // if creating an Kinesis checkpoint fails, then the same record(s) can be retrieved again
+                checkpointOnceEveryCheckpointInterval(processRecordsInput.getCheckpointer());
+            }
+        } catch (final Exception e) {
+            log.error("Unable to fully process received Kinesis record(s) due to {}", e.getLocalizedMessage(), e);
+            // FlowFiles that are already committed will not get rolled back
+            if (session != null) {
+                session.rollback();
+            }
+        }
+    }
+
+    void startProcessingRecords() {
+        processingRecords = true;
+    }
+
+    private int processRecordsWithRetries(final List<Record> records, final List<FlowFile> flowFiles,
+                                           final ProcessSession session, final StopWatch stopWatch) {
+        int recordsTransformed = 0;
+        for (int r = 0; r < records.size(); r++) {
+            final Record kinesisRecord = records.get(r);
+            boolean processedSuccessfully = false;
+            for (int i = 0; !processedSuccessfully && i < numRetries; i++) {
+                processedSuccessfully = attemptProcessRecord(flowFiles, kinesisRecord, r == records.size() - 1, session, stopWatch);
+            }
+
+            if (processedSuccessfully) {
+                recordsTransformed++;
+            } else {
+                log.error("Couldn't process Kinesis record {}, skipping.", kinesisRecord);
+            }
+        }
+
+        return recordsTransformed;
+    }
+
+    private boolean attemptProcessRecord(final List<FlowFile> flowFiles, final Record kinesisRecord, final boolean lastRecord,
+                                         final ProcessSession session, final StopWatch stopWatch) {
+        boolean processedSuccessfully = false;
+        try {
+            processRecord(flowFiles, kinesisRecord, lastRecord, session, stopWatch);
+            processedSuccessfully = true;
+        } catch (final Exception e) {
+            log.error("Caught Exception while processing Kinesis record {}", kinesisRecord, e);
+
+            // backoff if we encounter an exception.
+            try {
+                Thread.sleep(retryWaitMillis);
+            } catch (@SuppressWarnings("java:S2142") InterruptedException ie) {
+                log.debug("Interrupted sleep during record processing back-off", ie);
+            }
+        }
+
+        return processedSuccessfully;
+    }
+
+    /**
+     * Process an individual {@link Record} and serialise to {@link FlowFile}
+     *
+     * @param flowFiles {@link List} of {@link FlowFile}s to be output after all processing is complete
+     * @param kinesisRecord the Kinesis {@link Record} to be processed
+     * @param lastRecord whether this is the last {@link Record} to be processed in this batch
+     * @param session {@link ProcessSession} into which {@link FlowFile}s will be transferred
+     * @param stopWatch {@link StopWatch} tracking how much time has been spent processing the current batch
+     *
+     * @throws RuntimeException if there are any unhandled Exceptions that should be retried
+     */
+    abstract void processRecord(final List<FlowFile> flowFiles, final Record kinesisRecord, final boolean lastRecord,
+                                final ProcessSession session, final StopWatch stopWatch);
+
+    void reportProvenance(final ProcessSession session, final FlowFile flowFile, final String partitionKey,
+                 final String sequenceNumber, final StopWatch stopWatch) {
+        final String transitUri = StringUtils.isNotBlank(partitionKey) && StringUtils.isNotBlank(sequenceNumber)
+                ? String.format("%s/%s/%s#%s", transitUriPrefix, kinesisShardId, partitionKey, sequenceNumber)
+                : String.format("%s/%s", transitUriPrefix, kinesisShardId);
+
+        session.getProvenanceReporter().receive(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
+    }
+
+    Map<String, String> getDefaultAttributes(final String sequenceNumber, final String partitionKey, final Date approximateArrivalTimestamp) {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put(AWS_KINESIS_SHARD_ID, kinesisShardId);
+        attributes.put(AWS_KINESIS_SEQUENCE_NUMBER, sequenceNumber);
+        attributes.put(AWS_KINESIS_PARTITION_KEY, partitionKey);
+        if (approximateArrivalTimestamp != null) {
+            attributes.put(AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                    dateTimeFormatter.format(approximateArrivalTimestamp.toInstant().atZone(ZoneId.systemDefault())));
+        }
+        return attributes;
+    }
+
+    void transferTo(final Relationship relationship, final ProcessSession session, final int recordsProcessed,
+                    final int recordsTransformed, final List<FlowFile> flowFiles) {
+        session.adjustCounter("Records Processed", recordsProcessed, false);
+        if (!flowFiles.isEmpty()) {
+            session.adjustCounter("Records Transformed", recordsTransformed, false);
+            session.transfer(flowFiles, relationship);
+        }
+    }
+
+    private void checkpointOnceEveryCheckpointInterval(final IRecordProcessorCheckpointer checkpointer) {
+        if (System.currentTimeMillis() > nextCheckpointTimeInMillis) {
+            checkpointWithRetries(checkpointer);
+            nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
+        }
+    }
+
+    @Override
+    public void shutdown(final ShutdownInput shutdownInput) {
+        log.debug("Shutting down record processor for shard: {} with reason: {}", kinesisShardId, shutdownInput.getShutdownReason());
+
+        // be sure to finish processing any records before shutdown on TERMINATE
+        if (ShutdownReason.TERMINATE == shutdownInput.getShutdownReason()) {
+            for (int i = 0; processingRecords && i < numRetries; i++) {
+                log.debug("Record Processor for shard {} still processing records, waiting before shutdown", kinesisShardId);
+                try {
+                    Thread.sleep(retryWaitMillis);
+                } catch (@SuppressWarnings("java:S2142") InterruptedException ie) {
+                    log.debug("Interrupted sleep while waiting for record processing to complete before shutdown (TERMINATE)", ie);
+                }
+            }
+        }
+        checkpointWithRetries(shutdownInput.getCheckpointer());
+    }
+
+    private void checkpointWithRetries(final IRecordProcessorCheckpointer checkpointer) {
+        log.debug("Checkpointing shard " + kinesisShardId);
+        try {
+            for (int i = 0; i < numRetries; i++) {
+                if (checkpoint(checkpointer, i)) {
+                    break;
+                }
+            }
+        } catch (ShutdownException se) {
+            // Ignore checkpoint if the processor instance has been shutdown (fail over).
+            log.info("Caught shutdown exception, skipping checkpoint.", se);
+        } catch (InvalidStateException e) {
+            // This indicates an issue with the DynamoDB table (check for table, provisioned IOPS).
+            log.error("Cannot save checkpoint to the DynamoDB table used by the Amazon Kinesis Client Library.", e);
+        }
+    }
+
+    private boolean checkpoint(final IRecordProcessorCheckpointer checkpointer, final int attempt) throws ShutdownException, InvalidStateException {

Review comment:
       I think it should be called `attemptCheckpoint()` similar to `attemptProcessRecord()` (the success / backoff logic follows the same pattern in these methods).

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);

Review comment:
       As there are getter methods to retrieve the property values, there is no need to store them but the getter can be called everywhere the property needed (as far as I can see the ProcessContext object is available in those places).
   
   The same getters can be defined for the record factories below and those fields can also be deleted.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            final String workerId = generateWorkerId();
+            getLogger().info("Starting Kinesis Worker {}", workerId);
+            worker = prepareWorker(context, sessionFactory, workerId);
+            new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            final boolean success = shutdownWorker();
+            worker = null;
+            readerFactory = null;
+            writerFactory = null;
+
+            if (!success) {
+                getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                context.yield();

Review comment:
       As the processor is stopped, `yield()` has no effect.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            final String workerId = generateWorkerId();
+            getLogger().info("Starting Kinesis Worker {}", workerId);
+            worker = prepareWorker(context, sessionFactory, workerId);
+            new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker

Review comment:
       For this reason, it will not make sense to configure this processor with Concurrent Tasks > 1 in practice.
   Please disable concurrent tasks via adding `@TriggerSerially` annotation on the processor.
   
   It will prevent users to misconfigure the processor (believing that higher concurrent tasks would increase the throughput).
   Furthermore, in that case the worker creation should be synchronized also.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            final String workerId = generateWorkerId();
+            getLogger().info("Starting Kinesis Worker {}", workerId);
+            worker = prepareWorker(context, sessionFactory, workerId);
+            new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            final boolean success = shutdownWorker();
+            worker = null;
+            readerFactory = null;
+            writerFactory = null;
+
+            if (!success) {
+                getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                context.yield();
+            }
+        }
+    }
+
+    private Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final String appName = getApplicationName(context);
+        final String streamName = getStreamName(context);
+        final long checkpointIntervalMillis = getCheckpointIntervalMillis(context);
+        final String kinesisEndpoint = getKinesisOverride(context).orElse(null);

Review comment:
       As there are getter methods to retrieve the property values, it would be nicer to simply pass the context object in (instead of the bunch of parameters) and every prepare* method would retrieve the properties they need.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        final ControllerService recordReader = validationContext.getProperty(RECORD_READER).isSet()
+                ? validationContext.getProperty(RECORD_READER).asControllerService()
+                : null;
+        final ControllerService recordWriter = validationContext.getProperty(RECORD_WRITER).isSet()
+                ? validationContext.getProperty(RECORD_WRITER).asControllerService()
+                : null;
+        if (recordReader != null && recordWriter == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (recordWriter != null && recordReader == null) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class);
+
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            final String workerId = generateWorkerId();
+            getLogger().info("Starting Kinesis Worker {}", workerId);
+            worker = prepareWorker(context, sessionFactory, workerId);
+            new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            final boolean success = shutdownWorker();
+            worker = null;
+            readerFactory = null;
+            writerFactory = null;
+
+            if (!success) {
+                getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                context.yield();
+            }
+        }
+    }
+
+    private Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final String appName = getApplicationName(context);
+        final String streamName = getStreamName(context);
+        final long checkpointIntervalMillis = getCheckpointIntervalMillis(context);
+        final String kinesisEndpoint = getKinesisOverride(context).orElse(null);
+
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(sessionFactory, streamName,
+                checkpointIntervalMillis, kinesisEndpoint);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = prepareKinesisClientLibConfiguration(
+                context, appName, streamName, workerId, kinesisEndpoint);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(kinesisClientLibConfiguration, factory, context);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                appName, streamName, workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessSessionFactory sessionFactory, final String streamName,
+                                                                  final long checkpointIntervalMillis, final String kinesisEndpoint) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter, readerFactory, writerFactory
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), streamName, getClient().getEndpointPrefix(), kinesisEndpoint,
+                        checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String appName,
+                                                                       final String streamName, final String workerId,
+                                                                       final String kinesisEndpoint) {
+        final AWSCredentialsProvider credentialsProvider = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).isSet()

Review comment:
       The Credentials Provider service could be mandatory now because the processor level credential properties have been removed.
   In that case it would not be needed to check if the property is configured and create an anonymous credential otherwise.
   What do you think?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627178322



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,

Review comment:
       Removed the expression language capability from the property in the abstract class and added a new version to the `PutKinesisStream` processor that adds the expression language capability for just that processor
   
   Could copy & paste the definition into each processor, but sharing the common property definitions seemed sensible still




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r580091888



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/GetKinesisStream.java
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.lang3.StringUtils;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.kinesis.stream.record.KinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import javax.annotation.Nonnull;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Stream;
+
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads data from the specified AWS Kinesis stream and outputs a FlowFile for every processed Record. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY, description = "Partition from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER, description = "The unique identifier of the Kinesis Record within its shard."),
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_SHARD_ID, description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = KinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP, description = "Approximate arrival timestamp of the Record in the Kinesis stream")
+})
+@SeeAlso(PutKinesisStream.class)
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SuppressWarnings("java:S110")
+public class GetKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (initialise, process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, REGION, ENDPOINT_OVERRIDE, DYNAMODB_ENDPOINT_OVERRIDE,
+                    INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    private long retryWaitMillis;
+    private int numRetries;
+    private InitialPositionInStream initialPositionInStream;
+    private DateTimeFormatter dateTimeFormatter;
+    private Date startStreamPositionTimestamp;
+    private AWSCredentials awsCredentials;
+
+    private volatile ExecutorService executorService;
+    private Map<Worker, Future<?>> workers;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return Collections.singleton(REL_SUCCESS);
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final List<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    protected AmazonKinesisClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
+        awsCredentials = credentialsProvider.getCredentials();
+        return super.createClient(context, credentialsProvider, config);
+    }
+
+    @Override
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        retryWaitMillis = getRetryWaitMillis(context);
+        numRetries = getNumRetries(context);
+
+        super.onScheduled(context);
+
+        workers = new ConcurrentHashMap<>(context.getMaxConcurrentTasks());
+        executorService = Executors.newFixedThreadPool(context.getMaxConcurrentTasks(), new ThreadFactory() {
+            private final ThreadFactory defaultFactory = Executors.defaultThreadFactory();
+
+            @Override
+            public Thread newThread(@Nonnull final Runnable r) {
+                final Thread t = defaultFactory.newThread(r);
+                t.setName("GetKinesisStream " + getIdentifier() + " Task");
+                return t;
+            }
+        });
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        final Worker worker = prepareWorker(context, session);
+        final Future<?> workerFuture = executorService.submit(worker);
+
+        if (isScheduled()) {
+            workers.put(worker, workerFuture);
+
+            try {
+                getLogger().debug("Starting Kinesis Worker");
+                workerFuture.get(); // blocking Thread, completes when processor unscheduled and stopConsuming called
+                getLogger().debug("Kinesis Worker finished");
+            } catch (@SuppressWarnings("java:S2142") InterruptedException e) {
+                getLogger().warn("Interrupted while executing Kinesis Worker", e);
+            } catch (ExecutionException e) {
+                getLogger().error("Exception executing Kinesis Worker", e);
+                context.yield();
+            }
+        }
+    }
+
+    @OnUnscheduled
+    public void stopConsuming(final ProcessContext context) {
+        final AtomicBoolean success = new AtomicBoolean(true);
+        final Stream<Map.Entry<Worker, Future<?>>> workerStream;
+        if (workers.size() > 1) {
+            workerStream = workers.entrySet().parallelStream();
+        } else {
+            workerStream = workers.entrySet().stream();
+        }
+        workerStream.forEach(entry -> {
+            if (!shutdownWorker(entry.getKey(), entry.getValue())) {
+                success.set(false);
+            }
+        });
+
+        executorService.shutdown();
+        workers = null;
+
+        if (!success.get()) {
+            getLogger().warn("One of more problems while shutting down Kinesis Workers, see logs for details");
+            context.yield();
+        }
+    }
+
+    private Worker prepareWorker(final ProcessContext context, final ProcessSession session) {
+        final String appName = getApplicationName(context);
+        final String streamName = getStreamName(context);
+        final String workerId = generateWorkerId();
+        final long checkpointIntervalMillis = getCheckpointIntervalMillis(context);
+
+        final KinesisRecordProcessorFactory factory = new KinesisRecordProcessorFactory(session, getLogger(),
+                getClient().getEndpointPrefix(), checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = prepareKinesisClientLibConfiguration(
+                context, appName, streamName, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(kinesisClientLibConfiguration, factory, context);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                appName, streamName, workerId);
+
+        return workerBuilder.build();
+    }
+
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String appName,
+                                                                       final String streamName, final String workerId) {
+        final AWSCredentialsProvider credentialsProvider = new AWSStaticCredentialsProvider(awsCredentials);
+
+        @SuppressWarnings({"deprecated", "java:S1874"})
+        // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                new KinesisClientLibConfiguration(appName, streamName, credentialsProvider, workerId)
+                        .withCommonClientConfig(getClient().getClientConfiguration());
+
+        kinesisClientLibConfiguration.withRegionName(getRegion().getName())
+                .withMaxInitializationAttempts(numRetries);

Review comment:
       Would be better to allow use of Dynamic Properties to set this and other Kinesis Client Library Configuration properties. There are a lot of KCL config settings that could be tweaked and allowing a user to set them (without increasing the processor's already rather large list of static properties) would seem sensible.
   
   I'll look to push a change for this in the near future.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] bhaveshpatelh commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
bhaveshpatelh commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-804672379


   @ChrisSamo632 Any timelines for merging/releasing it out?
   We have a use-case and wanted to use it in NiFi production.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r629647637



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.MockProcessSession;
+import org.apache.nifi.util.SharedSessionState;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.time.format.DateTimeFormatter;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+public class TestAbstractKinesisRecordProcessor {
+    private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS");
+
+    private final TestRunner runner = TestRunners.newTestRunner(ConsumeKinesisStream.class);
+
+    @Mock
+    private ProcessSessionFactory processSessionFactory;
+
+    private final MockProcessSession session = new MockProcessSession(new SharedSessionState(runner.getProcessor(), new AtomicLong(0)), runner.getProcessor());
+
+    private IRecordProcessor fixture;

Review comment:
       I don't really understand why the interface is used for the type of the fixture.
   `AbstractKinesisRecordProcessor` could be used and `((AbstractKinesisRecordProcessor) fixture)` type casts could be avoided in that way.
   Could you please check it in other tests too?

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/kinesis/stream/record/TestAbstractKinesisRecordProcessor.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason;
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
+import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
+import org.apache.nifi.util.MockProcessSession;
+import org.apache.nifi.util.SharedSessionState;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import java.time.format.DateTimeFormatter;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+public class TestAbstractKinesisRecordProcessor {
+    private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS");
+
+    private final TestRunner runner = TestRunners.newTestRunner(ConsumeKinesisStream.class);
+
+    @Mock
+    private ProcessSessionFactory processSessionFactory;
+
+    private final MockProcessSession session = new MockProcessSession(new SharedSessionState(runner.getProcessor(), new AtomicLong(0)), runner.getProcessor());
+
+    private IRecordProcessor fixture;
+
+    @Mock
+    private IRecordProcessorCheckpointer checkpointer;
+
+    @Mock
+    private Record kinesisRecord;
+
+    @Before
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+
+        when(processSessionFactory.createSession()).thenReturn(session);
+
+        // default test fixture will try operations twice with very little wait in between
+        fixture = new MockKinesisRecordProcessor(processSessionFactory, runner.getLogger(), "kinesis-test",
+                "endpoint-prefix", null, 10_000L, 1L, 2, DATE_TIME_FORMATTER);

Review comment:
       Test object could be defined inline and no Mock class would be needed.
   
   ```
           fixture = new AbstractKinesisRecordProcessor(processSessionFactory, runner.getLogger(), "kinesis-test",
                   "endpoint-prefix", null, 10_000L, 1L, 2, DATE_TIME_FORMATTER) {
               @Override
               void processRecord(List<FlowFile> flowFiles, Record kinesisRecord, boolean lastRecord, ProcessSession session, StopWatch stopWatch) {
                   // intentionally blank
               }
           };
   ```




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628017950



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 min")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));

Review comment:
       Yeah I agree this is better, hadn't checked/realised that Validation & Process Contexts were related with the super class, using the suggested methods is definitely cleaner




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-804689864


   > @ChrisSamo632 - out of curiosity, did you consider the addition of a record reader/writer like with ConsumeKafkaRecord processors? At the moment, as far as I can tell, we would have one record per flow file. Using the records abstraction would provide options such as schema validation, format transformation, as well as having multiple records in one flow file (which is greatly improving the performances in case there is a high number of messages per second).
   
   Think I considered this in our original slack conversation, but thought I'd leave it record writers at the minute for simplicity and to better understand how the Kinesis message (vs. Record) structure works
   
   With the KCL worker and multi-threaded consumer approach taken by the KCL library, we'd need a way of combining the records in the processor too (how do we combine records from different consumers? How about where the consumers are reading from different shards?)
   
   So I thought a record reader may be a sensible extension (fully agree it would be good to include from a nifi perspective) once there's more understanding of how people use the processor and how it works with different kinesis setups (so far I've only really tested it with simple streams)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628256440



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        }
+
+        // after a Worker is registered successfully, nothing has to be done at onTrigger
+        // because new sessions are created when new messages are consumed by the Worker
+        context.yield();
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // intentionally blank (using onTrigger with ProcessSessionFactory above instead)
+    }
+
+    @OnStopped
+    public void stopConsuming(final ProcessContext context) {
+        if (worker != null) {
+            synchronized (WORKER_LOCK) {
+                if (worker != null) {
+                    final boolean success = shutdownWorker(context);
+                    worker = null;
+
+                    if (!success) {
+                        getLogger().warn("One or more problems while shutting down Kinesis Worker, see logs for details");
+                    }
+                }
+            }
+        }
+    }
+
+    private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
+        final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
+
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration =
+                prepareKinesisClientLibConfiguration(context, workerId);
+
+        final Worker.Builder workerBuilder = prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
+
+        getLogger().info("Kinesis Worker prepared for application {} to process stream {} as worker ID {}...",
+                getApplicationName(context), getStreamName(context), workerId);
+
+        return workerBuilder.build();
+    }
+
+    private IRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        return () -> {
+            if (isRecordReaderSet && isRecordWriterSet) {
+                return new KinesisRecordProcessorRecord(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
+                        getReaderFactory(context), getWriterFactory(context)
+                );
+            } else {
+                return new KinesisRecordProcessorRaw(
+                        sessionFactory, getLogger(), getStreamName(context), getClient().getEndpointPrefix(),
+                        getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
+                        getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
+                );
+            }
+        };
+    }
+
+    /*
+     *  Developer note: if setting KCL configuration explicitly from processor properties, be sure to add them to the
+     *  DISALLOWED_DYNAMIC_KCL_PROPERTIES list so Dynamic Properties can't be used to override the static properties
+     */
+    KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
+        @SuppressWarnings({"deprecated", "java:S1874"}) // use most of the defaults in the constructor chain rather than the mammoth constructor here
+        final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
+                getApplicationName(context),
+                getStreamName(context),
+                getCredentialsProvider(context),
+                workerId
+        )
+                .withCommonClientConfig(getClient().getClientConfiguration())
+                .withRegionName(getRegion().getName())
+                .withFailoverTimeMillis(getFailoverTimeMillis(context))
+                .withShutdownGraceMillis(getGracefulShutdownMillis(context));
+
+        final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            kinesisClientLibConfiguration.withTimestampAtInitialPositionInStream(getStartStreamTimestamp(context));
+        } else {
+            kinesisClientLibConfiguration.withInitialPositionInStream(initialPositionInStream);
+        }
+
+        getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
+
+        final String kinesisEndpoint = getKinesisEndpoint(context).orElse(null);
+        if (StringUtils.isNotBlank(kinesisEndpoint)) {
+            kinesisClientLibConfiguration.withKinesisEndpoint(kinesisEndpoint);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+                .keySet()
+                .stream()
+                .filter(PropertyDescriptor::isDynamic)
+                .collect(Collectors.toList());
+
+        final AtomicBoolean dynamicPropertyFailure = new AtomicBoolean(false);
+        dynamicProperties.forEach(descriptor -> {
+            final String name = descriptor.getName();
+            final String value = context.getProperty(descriptor).getValue();
+            try {
+                BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, name, value);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+                getLogger().error("Unable to set Kinesis Client Library Configuration property for {} with value {}", name, value, e);
+                dynamicPropertyFailure.set(true);
+            }
+        });
+        if (dynamicPropertyFailure.get()) {
+            throw new ProcessException("Failed to set dynamic properties for the Kinesis Client Library (see logs for more details)");

Review comment:
       I don't think the `AtomicBoolean` trick is needed here but `ProcessException` can simply be thrown from the `catch` above.
   It would log only the first error but practically cannot be any error here because dynamic properties have already checked by the validator.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627213813



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/record/KinesisRecordProcessorRaw.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.stream.record;
+
+import com.amazonaws.services.kinesis.model.Record;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.util.StopWatch;
+
+import java.time.format.DateTimeFormatter;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class KinesisRecordProcessorRaw extends AbstractKinesisRecordProcessor {
+    @SuppressWarnings("java:S107")
+    public KinesisRecordProcessorRaw(final ProcessSessionFactory sessionFactory, final ComponentLog log, final String streamName,
+                                     final String endpointPrefix, final long checkpointIntervalMillis, final long retryWaitMillis,
+                                     final int numRetries, final DateTimeFormatter dateTimeFormatter) {
+        super(sessionFactory, log, streamName, endpointPrefix, checkpointIntervalMillis, retryWaitMillis, numRetries, dateTimeFormatter);
+    }
+
+    @Override
+    void processRecord(final List<FlowFile> flowFiles, final Record record, final boolean lastRecord,
+                       final ProcessSession session, final StopWatch stopWatch) {
+        final String partitionKey = record.getPartitionKey();
+        final String sequenceNumber = record.getSequenceNumber();
+        final Date approximateArrivalTimestamp = record.getApproximateArrivalTimestamp();
+        final byte[] data = record.getData() != null ? record.getData().array() : new byte[0];
+
+        FlowFile flowFile = session.create();
+        session.write(flowFile, out -> out.write(data));
+
+        if (log.isDebugEnabled()) {
+            log.debug("Sequence No: {}, Partition Key: {}, Data: {}", sequenceNumber, partitionKey, BASE_64_ENCODER.encodeToString(data));
+        }
+
+        session.getProvenanceReporter().receive(flowFile, String.format("http://%s.amazonaws.com/%s/%s#%s", endpointPrefix, kinesisShardId,

Review comment:
       I think you're right - the IT tests have an example of using `http://localhost:4566` to override the Kinesis endpoint, so using that as the `transitUri` prefix would make more sense (i.e. where an endpoint override is provided, we should use that, otherwise default to `http://*.amazonaws.com/...`)




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] pvillard31 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-832606983


   Hey @ChrisSamo632 - It seems there are some build issues with the unit tests.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r626384627



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings("java:S110")
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)

Review comment:
       Should limit allowedValues (true or false)




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627452012



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", KINESIS_STREAM_NAME);
+    }};
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+    private volatile AWSCredentials awsCredentials;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),

Review comment:
       Good ol' i18n - I'm in UK, guess you're US (or another English speaking locale)?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628163612



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,708 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_PERIOD = new PropertyDescriptor.Builder()
+            .displayName("Failover Time")
+            .name("amazon-kinesis-stream-failover-period")
+            .description("Kinesis Client Library failover period")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_PERIOD = new PropertyDescriptor.Builder()
+            .displayName("Failover Time")

Review comment:
       Copy-paste error: Graceful Shutdown Time
   (not sure but "Timeout" may be better, also in case of Failover Timeout)




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r630918349



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,726 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.WorkerStateChangeListener;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.AbstractAWSCredentialsProviderProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@TriggerSerially
+@Tags({"amazon", "aws", "kinesis", "consume", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (optional) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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.")
+@SeeAlso(PutKinesisStream.class)
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor FAILOVER_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Failover Timeout")
+            .name("amazon-kinesis-stream-failover-timeout")
+            .description("Kinesis Client Library failover timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("30 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor GRACEFUL_SHUTDOWN_TIMEOUT = new PropertyDescriptor.Builder()
+            .displayName("Graceful Shutdown Timeout")
+            .name("amazon-kinesis-stream-graceful-shutdown-timeout")
+            .description("Kinesis Client Library graceful shutdown timeout")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("20 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval between Kinesis checkpoints")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("3 secs")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval between Kinesis operation retries (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("1 sec")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
+            .required(true)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+        put("failoverTimeMillis", FAILOVER_TIMEOUT);
+        put("gracefulShutdownMillis", GRACEFUL_SHUTDOWN_TIMEOUT);
+    }};
+
+    private static final Object WORKER_LOCK = new Object();
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Worker.class.getSimpleName() + "-";
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private static final PropertyUtilsBean PROPERTY_UTILS_BEAN;
+    private static final BeanUtilsBean BEAN_UTILS_BEAN;
+    static {
+        PROPERTY_UTILS_BEAN = new PropertyUtilsBean();
+        PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        BEAN_UTILS_BEAN = new BeanUtilsBean(convertUtilsBean2, PROPERTY_UTILS_BEAN);
+    }
+
+    private volatile boolean isRecordReaderSet;
+    private volatile boolean isRecordWriterSet;
+
+    private volatile Worker worker;
+    final AtomicReference<WorkerStateChangeListener.WorkerState> workerState = new AtomicReference<>(null);
+    private final AtomicBoolean stopped = new AtomicBoolean(false);
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTY_DESCRIPTORS;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-zA-Z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            final String propName = StringUtils.uncapitalize(subject);
+            if (!PROPERTY_UTILS_BEAN.isWriteable(kclTemp, propName)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name %s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            BEAN_UTILS_BEAN.setProperty(kclTemp, propName, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name %s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name %s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        DateTimeFormatter dateTimeFormatter = null;
+        try {
+            dateTimeFormatter = getDateTimeFormatter(validationContext);
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == getInitialPositionInStream(validationContext)) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                try {
+                    // check the streamTimestamp can be formatted
+                    getStartStreamTimestamp(validationContext, dateTimeFormatter);
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                    TIMESTAMP_FORMAT.getDisplayName()))
+                            .build());
+                }
+            }
+        }
+
+        if (isRecordReaderSet && !isRecordWriterSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_WRITER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_WRITER.getDisplayName(), RECORD_READER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        } else if (isRecordWriterSet && !isRecordReaderSet) {
+            validationResults.add(new ValidationResult.Builder()
+                    .subject(RECORD_READER.getName())
+                    .explanation(String.format("%s must be set if %s is set in order to write FlowFiles as Records.",
+                            RECORD_READER.getDisplayName(), RECORD_WRITER.getDisplayName()))
+                    .valid(false)
+                    .build());
+        }
+
+        return validationResults;
+    }
+
+    @OnScheduled
+    @Override
+    public void onScheduled(ProcessContext context) {
+        stopped.set(false);
+        super.onScheduled(context);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+        if (worker == null) {
+            synchronized (WORKER_LOCK) {
+                if (worker == null) {
+                    final String workerId = generateWorkerId();
+                    getLogger().info("Starting Kinesis Worker {}", workerId);
+                    // create worker (WorkerState will be CREATED)
+                    worker = prepareWorker(context, sessionFactory, workerId);
+                    // initialise and start Worker (will set WorkerState to INITIALIZING and attempt to start)
+                    new Thread(worker, WORKER_THREAD_NAME_TEMPLATE + workerId).start();
+                }
+            }
+        } else {
+            if (!stopped.get() && WorkerStateChangeListener.WorkerState.SHUT_DOWN == workerState.get()) {
+                throw new ProcessException("Worker has shutdown unexpectedly, possibly due to a configuration issue; check logs for details");

Review comment:
       `context.yield()` needs to be called before throwing the exception (without yielding `onTrigger()` would be called again immediately and would dump the log with the error message in a loop).




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-839077520


   @turcsanyip I've implemented a check in `onTrigger` to ensure the `WorkerState` never reaches `SHUT_DOWN` without the `ConsumeKinesisStream` processor being Stopped in NiFi - this is the only way I can see of trying to check whether the Worker has failed to initialise (it would remain at `CREATED` or `INITIALIZING` state until finally shutting down due to a configuration error).
   
   Unit test added to show the behaviour (happily the unit tests don't connect to a Kinesis/DynamoDB instance so the Worker always fails eventually, now we throw a `ProcessException` to note that in the NiFi UI and allow a Bulletin to be generated).
   
   Newer KCL versions allow for better handling of this scenario, but we're limited to when we can do in KCL 1.13.3 (see NIFI-8531 for upgrading to KCL 2.x and a note about improving this scenario).


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 edited a comment on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 edited a comment on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-808751275


   @turcsanyip started to have a look at making these changes. Using KCL 1.13.3 instead of 1.14.x appears to be straight forward (i.e. no big API changes).
   
   The refactoring to use a single Worker per processor and yield once the Worker has been setup looks fairly straight forward and I agree a simpler implementation *except* that if I am to follow the `ConsumeAzureEventHub` approach and use the `ProcessSessionFactory` version of `onTrigger`, I have to extend the `AbstractSessionFactoryProcessor` base class, but the existing AWS processors have a fairly long chain that extends from `AbstractProcessor` - I don't want to unmarry this processor from all the existing AWS processors and moving all AWS processors to use the different base class feels like a big change and not something we'd want to do.
   
   Is there a straight forward way around this do you think? OIne concern I had with the original implementation was the fact that I was holding on to a single `ProcessSession` and comitting it multiple times (i.e. after every set of messages had been processed by the KCL RecordProcessor) - using the ProcessSessionFactory approach to create a new session every time a new set of Kinesis messages are received would seem better... aside for the above issue of the different abstract processors.
   
   I'll look again at this further, but any guidance is welcome!
   
   
   @pvillard31 having looked again at the Record processing now I've the Azure processor to compare with, I think includiung them here should be fairly straight forward (famous last words)
   
   
   EDIT: the Session Factory problem can be sorted by changing the base abstract AWS processor to extend `AbstractSessionFactoryProcessor` and copying the `AbstractProcessor` methods into there (but not making the overridden `onTrigger` method as `final`) - this retains the same functionality for most AWS processors but allows the `ConsumeKinesisStream` processor access to the `ProcessSessionFactory`. An akhenaten would be to remove the final modifier from the `AbstractProcessor` `onTrigger` that accepts the factory (but that impacts nearly all processors and the method is presumably `final` for good reason)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627180647



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", KINESIS_STREAM_NAME);
+    }};
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+    private volatile AWSCredentials awsCredentials;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),

Review comment:
       The oh-so-reliable [Wikipedia](https://en.wiktionary.org/wiki/parsable) suggests both versions of the word are correct and "without the e" generally prefered (in formal language at least) - IntelliJ complains at me if I add the `e` so I'll leave as-is unless you really object :slightly_smiling_face: 




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r628155029



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", KINESIS_STREAM_NAME);
+    }};
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+    private volatile AWSCredentials awsCredentials;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),

Review comment:
       Yes, the proofreading language has been set to English (USA) in my IntelliJ.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#discussion_r627415055



##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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 are are not released until this processor is stopped.")

Review comment:
       Typo: "and are not released..." (?)

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,680 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.services.kinesis.AmazonKinesisClient;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition from which the (last) Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. It runs continually, spawning Kinesis Record Processors as required " +
+        "to fetch Kinesis Records. The Worker Thread (and any child Record Processor threads) is not released until this processor is stopped. " +
+        "This means a NiFi Concurrent Thread is permanently assigned to this Processor while it is running and other threads will be created within the JVM " +
+        "that are not controlled by the normal NiFi scheduler.")
+@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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
+                    PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", KINESIS_STREAM_NAME);
+    }};
+
+    private static final Set<Relationship> RELATIONSHIPS = Collections.singleton(REL_SUCCESS);
+    private static final Set<Relationship> RECORD_RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(REL_SUCCESS, REL_PARSE_FAILURE)));
+
+    private final PropertyUtilsBean propertyUtilsBean;
+    private final BeanUtilsBean beanUtilsBean;
+
+    private volatile long retryWaitMillis;
+    private volatile int numRetries;
+    private volatile InitialPositionInStream initialPositionInStream;
+    private volatile DateTimeFormatter dateTimeFormatter;
+    private volatile Date startStreamPositionTimestamp;
+    private volatile AWSCredentials awsCredentials;
+
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordSetWriterFactory writerFactory;
+    private volatile boolean isRecordReaderSet = false;
+    private volatile boolean isRecordWriterSet = false;
+
+    private volatile Worker worker;
+
+    public ConsumeKinesisStream() {
+        propertyUtilsBean = new PropertyUtilsBean();
+        propertyUtilsBean.addBeanIntrospector(new FluentPropertyBeanIntrospector("with"));
+
+        final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
+            @SuppressWarnings("unchecked") // generic Enum conversion from String property values
+            @Override
+            public Object convert(final String value, final Class clazz) {
+                if (clazz.isEnum()) {
+                    return Enum.valueOf(clazz, value);
+                }else{
+                    return super.convert(value, clazz);
+                }
+            }
+        };
+
+        beanUtilsBean = new BeanUtilsBean(convertUtilsBean2, propertyUtilsBean);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return isRecordReaderSet && isRecordWriterSet ? RECORD_RELATIONSHIPS : RELATIONSHIPS;
+    }
+
+    @Override
+    public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) {
+        if (RECORD_READER.equals(descriptor)) {
+            isRecordReaderSet = StringUtils.isNotEmpty(newValue);
+        } else if (RECORD_WRITER.equals(descriptor)) {
+            isRecordWriterSet = StringUtils.isNotEmpty(newValue);
+        }
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
+                .addValidator(this::validateDynamicKCLConfigProperty)
+                .expressionLanguageSupported(ExpressionLanguageScope.NONE);
+
+        return builder.build();
+    }
+
+    private ValidationResult validateDynamicKCLConfigProperty(final String subject, final String input, final ValidationContext context) {
+        final ValidationResult.Builder validationResult = new ValidationResult.Builder().subject(subject).input(input);
+
+        if (!subject.matches("^(?!with)[a-z]\\w*$")) {
+            return validationResult
+                    .explanation("Property name must not have a prefix of \"with\", must start with a lowercase letter and contain only letters, numbers or underscores")
+                    .valid(false).build();
+        }
+
+        if (DISALLOWED_DYNAMIC_KCL_PROPERTIES.keySet().stream().anyMatch(k -> k.equalsIgnoreCase(subject))) {
+            return validationResult
+                    .explanation(String.format("Use \"%s\" instead of a dynamic property", DISALLOWED_DYNAMIC_KCL_PROPERTIES.get(subject).getDisplayName()))
+                    .valid(false).build();
+        }
+
+        @SuppressWarnings("java:S1192")
+        final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
+        try {
+            if (!propertyUtilsBean.isWriteable(kclTemp, subject)) {
+                return validationResult
+                        .explanation(String.format("Kinesis Client Library Configuration property with name with%s does not exist or is not writable", StringUtils.capitalize(subject)))
+                        .valid(false).build();
+            }
+            beanUtilsBean.setProperty(kclTemp, subject, input);
+        } catch (IllegalAccessException e) {
+            return validationResult
+                    .explanation(String.format("Kinesis Client Library Configuration property with name with%s is not accessible", StringUtils.capitalize(subject)))
+                    .valid(false).build();
+        } catch (InvocationTargetException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
+        } catch (IllegalArgumentException e) {
+            return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
+        }
+
+        return validationResult.valid(true).build();
+    }
+
+    private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
+                                                                      final String subject, final String input, final String message) {
+        return validationResult
+                .explanation(
+                        String.format("Kinesis Client Library Configuration property with name with%s cannot be used with value \"%s\" : %s",
+                                StringUtils.capitalize(subject), input, message)
+                )
+                .valid(false).build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final Set<ValidationResult> validationResults = new HashSet<>(super.customValidate(validationContext));
+
+        initialPositionInStream = InitialPositionInStream.valueOf(StringUtils.trimToEmpty(validationContext.getProperty(INITIAL_STREAM_POSITION).getValue()));
+        try {
+            dateTimeFormatter = DateTimeFormatter.ofPattern(validationContext.getProperty(TIMESTAMP_FORMAT).evaluateAttributeExpressions().getValue());
+        } catch (IllegalArgumentException iae) {
+            validationResults.add(new ValidationResult.Builder().valid(false)
+                    .subject(TIMESTAMP_FORMAT.getName())
+                    .explanation(String.format("%s must be a valid java.time.DateTimeFormatter format", TIMESTAMP_FORMAT.getDisplayName()))
+                    .build()
+            );
+        }
+
+        if (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream) {
+            if (!validationContext.getProperty(STREAM_POSITION_TIMESTAMP).isSet()) {
+                validationResults.add(new ValidationResult.Builder().valid(false)
+                        .subject(STREAM_POSITION_TIMESTAMP.getName())
+                        .explanation(String.format("%s must be provided when %s is %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),
+                                INITIAL_STREAM_POSITION.getDisplayName(), AT_TIMESTAMP.getDisplayName()))
+                        .build()
+                );
+            } else if (dateTimeFormatter != null) {
+                final String streamTimestamp = validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue();
+                try {
+                    dateTimeFormatter.parse(streamTimestamp);
+                    startStreamPositionTimestamp = new Date(
+                            LocalDateTime.parse(validationContext.getProperty(STREAM_POSITION_TIMESTAMP).getValue(), dateTimeFormatter)
+                                    .toInstant(ZoneOffset.UTC).toEpochMilli()
+                    );
+                } catch (Exception e) {
+                    validationResults.add(new ValidationResult.Builder().valid(false)
+                            .subject(STREAM_POSITION_TIMESTAMP.getName())
+                            .explanation(String.format("%s must be parsable by %s", STREAM_POSITION_TIMESTAMP.getDisplayName(),

Review comment:
       Interesting, my IntelliJ is complaining about "parsable" :)
   Let's leave it as it is now.

##########
File path: nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/stream/ConsumeKinesisStream.java
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.stream;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.AWSStaticCredentialsProvider;
+import com.amazonaws.auth.AnonymousAWSCredentials;
+import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessorFactory;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration;
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import org.apache.commons.beanutils.BeanUtilsBean;
+import org.apache.commons.beanutils.ConvertUtilsBean2;
+import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
+import org.apache.commons.beanutils.PropertyUtilsBean;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+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.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+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.kinesis.stream.record.AbstractKinesisRecordProcessor;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
+import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.RecordFieldType;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+@Tags({"amazon", "aws", "kinesis", "get", "stream"})
+@CapabilityDescription("Reads 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. " +
+        "At-least-once delivery of all Kinesis Records within the Stream while the processor is running. " +
+        "AWS Kinesis Client Library can take several seconds to initialise before starting to fetch data. " +
+        "Uses DynamoDB for check pointing and CloudWatch (optional) for metrics. " +
+        "Ensure that the credentials provided have access to DynamoDB and CloudWatch (if used) along with Kinesis.")
+@WritesAttributes({
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_PARTITION_KEY,
+                description = "Partition key of the (last) Kinesis Record read from the Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SHARD_ID,
+                description = "Shard ID from which the Kinesis Record was read"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_SEQUENCE_NUMBER,
+                description = "The unique identifier of the (last) Kinesis Record within its Shard"),
+        @WritesAttribute(attribute = AbstractKinesisRecordProcessor.AWS_KINESIS_APPROXIMATE_ARRIVAL_TIMESTAMP,
+                description = "Approximate arrival timestamp of the (last) Kinesis Record read from the stream"),
+        @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)")
+})
+@DynamicProperties({
+        @DynamicProperty(name="Kinesis Client Library (KCL) Configuration property name",
+                description="Override default KCL Configuration properties with required values. Supports setting of values via the \"with\" " +
+                        "methods on the KCL Configuration class. Specify the property to be set without the leading prefix, e.g. \"maxInitialisationAttempts\" " +
+                        "will call \"withMaxInitialisationAttempts\" and set the provided value. Only supports setting of simple property values, e.g. String, " +
+                        "int, long and boolean. Does not allow override of KCL Configuration settings handled by non-dynamic processor properties.",
+                expressionLanguageScope = ExpressionLanguageScope.NONE, value="Value to set in the KCL Configuration property")
+})
+@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 triggered. 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 are 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.")
+@SeeAlso(PutKinesisStream.class)
+@SuppressWarnings({"java:S110", "java:S2160", "java:S3077"})
+public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
+    static final AllowableValue TRIM_HORIZON = new AllowableValue(
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            InitialPositionInStream.TRIM_HORIZON.toString(),
+            "Start reading at the last untrimmed record in the shard in the system, which is the oldest data record in the shard."
+    );
+    static final AllowableValue LATEST = new AllowableValue(
+            InitialPositionInStream.LATEST.toString(),
+            InitialPositionInStream.LATEST.toString(),
+            "Start reading just after the most recent record in the shard, so that you always read the most recent data in the shard."
+    );
+    static final AllowableValue AT_TIMESTAMP = new AllowableValue(
+            InitialPositionInStream.AT_TIMESTAMP.toString(),
+            InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
+    );
+
+    public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
+            .displayName("Application Name")
+            .name("amazon-kinesis-stream-application-name")
+            .description("The Kinesis stream reader application name.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .required(true).build();
+
+    public static final PropertyDescriptor INITIAL_STREAM_POSITION = new PropertyDescriptor.Builder()
+            .displayName("Initial Stream Position")
+            .name("amazon-kinesis-stream-initial-position")
+            .description("Initial position to read Kinesis streams.")
+            .allowableValues(LATEST, TRIM_HORIZON, AT_TIMESTAMP)
+            .defaultValue(LATEST.getValue())
+            .required(true).build();
+
+    public static final PropertyDescriptor STREAM_POSITION_TIMESTAMP = new PropertyDescriptor.Builder()
+            .displayName("Stream Position Timestamp")
+            .name("amazon-kinesis-stream-position-timestamp")
+            .description("Timestamp position in stream from which to start reading Kinesis Records. " +
+                    "Required if " + INITIAL_STREAM_POSITION.getDescription() + " is " + AT_TIMESTAMP.getDisplayName() + ". " +
+                    "Uses the Timestamp Format to parse value into a Date.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR) // customValidate checks the value against TIMESTAMP_FORMAT
+            .dependsOn(INITIAL_STREAM_POSITION, AT_TIMESTAMP)
+            .required(false).build();
+
+    public static final PropertyDescriptor TIMESTAMP_FORMAT = new PropertyDescriptor.Builder()
+            .displayName("Timestamp Format")
+            .name("amazon-kinesis-stream-timestamp-format")
+            .description("Format to use for parsing the " + STREAM_POSITION_TIMESTAMP.getDisplayName() + " into a Date " +
+                    "and converting the Kinesis Record's Approximate Arrival Timestamp into a FlowFile attribute.")
+            .addValidator((subject, input, context) -> {
+                if (StringUtils.isNotBlank(input)) {
+                    try {
+                        DateTimeFormatter.ofPattern(input);
+                    } catch (Exception e) {
+                        return new ValidationResult.Builder().valid(false).subject(subject).input(input)
+                                .explanation("Must be a valid java.time.DateTimeFormatter pattern, e.g. " + RecordFieldType.TIMESTAMP.getDefaultFormat())
+                                .build();
+                    }
+                }
+                return new ValidationResult.Builder().valid(true).subject(subject).build();
+            })
+            .defaultValue(RecordFieldType.TIMESTAMP.getDefaultFormat())
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true).build();
+
+    public static final PropertyDescriptor CHECKPOINT_INTERVAL_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Checkpoint Interval")
+            .name("amazon-kinesis-stream-checkpoint-interval")
+            .description("Interval (milliseconds) between Kinesis checkpoints")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("60000")
+            .required(true).build();
+
+    public static final PropertyDescriptor NUM_RETRIES = new PropertyDescriptor.Builder()
+            .displayName("Retry Count")
+            .name("amazon-kinesis-stream-retry-count")
+            .description("Number of times to retry a Kinesis operation (process record, checkpoint, shutdown)")
+            .addValidator(StandardValidators.INTEGER_VALIDATOR)
+            .defaultValue("10")
+            .required(true).build();
+
+    public static final PropertyDescriptor RETRY_WAIT_MILLIS = new PropertyDescriptor.Builder()
+            .displayName("Retry Wait")
+            .name("amazon-kinesis-stream-retry-wait")
+            .description("Interval (milliseconds) between Kinesis operation retries (get records, checkpoint, shutdown)")
+            .addValidator(StandardValidators.LONG_VALIDATOR)
+            .defaultValue("1000")
+            .required(true).build();
+
+    public static final PropertyDescriptor DYNAMODB_ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
+            .displayName("DynamoDB Override")
+            .name("amazon-kinesis-stream-dynamodb-override")
+            .description("DynamoDB override to use non-AWS deployments")
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(false).build();
+
+    public static final PropertyDescriptor REPORT_CLOUDWATCH_METRICS = new PropertyDescriptor.Builder()
+            .displayName("Report Metrics to CloudWatch")
+            .name("amazon-kinesis-stream-cloudwatch-flag")
+            .description("Whether to report Kinesis usage metrics to CloudWatch.")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .required(true).build();
+
+    public static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-reader")
+            .displayName("Record Reader")
+            .description("The Record Reader to use for reading received messages." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder()
+            .name("amazon-kinesis-stream-record-writer")
+            .displayName("Record Writer")
+            .description("The Record Writer to use for serializing Records to an output FlowFile." +
+                    " The Kinesis Stream name can be referred to by Expression Language '${" +
+                    AbstractKinesisRecordProcessor.KINESIS_RECORD_SCHEMA_KEY + "}' to access a schema." +
+                    " If Record Reader/Writer are not specified, each Kinesis Record will create a FlowFile.")
+            .identifiesControllerService(RecordSetWriterFactory.class)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .required(false)
+            .build();
+
+    public static final Relationship REL_PARSE_FAILURE = new Relationship.Builder()
+            .name("parse.failure")
+            .description("If a message from Kinesis cannot be parsed using the configured Record Reader" +
+                    " or failed to be written by the configured Record Writer," +
+                    " the contents of the message will be routed to this Relationship as its own individual FlowFile.")
+            .build();
+
+    public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
+            Arrays.asList(
+                    // Kinesis Stream specific properties
+                    KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
+                    DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
+                    CHECKPOINT_INTERVAL_MILLIS, NUM_RETRIES, RETRY_WAIT_MILLIS, REPORT_CLOUDWATCH_METRICS,
+                    // generic AWS processor properties
+                    TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
+            )
+    );
+
+    @SuppressWarnings({"java:S3599", "java:S1171"})
+    private static final Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<String, PropertyDescriptor>(){{
+        put("regionName", REGION);
+        put("timestampAtInitialPositionInStream", STREAM_POSITION_TIMESTAMP);
+        put("initialPositionInStream", INITIAL_STREAM_POSITION);
+        put("dynamoDBEndpoint", DYNAMODB_ENDPOINT_OVERRIDE);
+        put("kinesisEndpoint", ENDPOINT_OVERRIDE);
+    }};
+
+    private static final String WORKER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getName() + "-" + Worker.class.getName() + "-";

Review comment:
       The thread name with the fully qualified class names is really long. I would consider to use `Class.getSimpleName()` instead.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] pvillard31 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-804680493


   @ChrisSamo632 - out of curiosity, did you consider the addition of a record reader/writer like with ConsumeKafkaRecord processors? At the moment, as far as I can tell, we would have one record per flow file. Using the records abstraction would provide options such as schema validation, format transformation, as well as having multiple records in one flow file (which is greatly improving the performances in case there is a high number of messages per second).


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-790017429


   > Hi ! I'm interested in testing this out.
   > I currently run Apache Nifi using the base Docker images. How can I proceed to include the Kinesis Get Processor into my images?
   > Thanks ! Hope I can help!
   
   @auyer I think you'd need to download the source from my branch, [build it locally](https://github.com/apache/nifi#getting-started) and then copy the new NARs into your image in the `lib/` folder (overwrite the existing AWS NARs) - you can probably build just the AWS NARs (rather than the whole of NiFi) to speed up the process... if you build the whole of NiFi, you might as well just run the compiled version natively rather than using the Docker Image.
   
   So, something like:
   ```bash
   # clone the code
   git clone https://github.com/ChrisSamo632/nifi.git
   git checkout NIFI-2892
   
   # build the code
   cd nifi/nifi-nar-bundles/nifi-aws-bundle
   mvn -T 2.0C clean install
   
   # get the NARs
   find . -name "*.nar" -exec {} /tmp
   ```
   
   Copy these into a custom Docker Image with something like the following:
   ```dockerfile
   FROM apache/nifi:1.13.0
   
   RUN rm lib/nifi-aws*.nar
   
   COPY [ "*.nar", "lib/" ]
   ```
   
   Then run the custom image in the way you normally would.
   
   Note that I've not tested the above, so you may need to correct the instructions as you go - when developing NiFi, I just build from source and run natively, I've not got to the point of trying to copy the into a Docker container yet although that is how I run NiFi normally.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] ChrisSamo632 commented on pull request #4822: NIFI-2892 Implement AWS Kinesis Stream Get Processor

Posted by GitBox <gi...@apache.org>.
ChrisSamo632 commented on pull request #4822:
URL: https://github.com/apache/nifi/pull/4822#issuecomment-832641757


   > Hey @ChrisSamo632 - It seems there are some build issues with the unit tests.
   
   Yep, my bad - updated the processor and missed the unit test! Should be fixed now @pvillard31 (will keep an eye on the build status)


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org