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/22 09:19:55 UTC

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

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