Github user apiri commented on a diff in the pull request:
https://github.com/apache/nifi/pull/213#discussion_r53849365
--- Diff:
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/kinesis/firehose/PutKinesisFirehose.java
---
@@ -0,0 +1,184 @@
+/*
+ * 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.firehose;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+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.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import com.amazonaws.services.kinesisfirehose.AmazonKinesisFirehoseClient;
+import com.amazonaws.services.kinesisfirehose.model.PutRecordBatchRequest;
+import
com.amazonaws.services.kinesisfirehose.model.PutRecordBatchResponseEntry;
+import com.amazonaws.services.kinesisfirehose.model.PutRecordBatchResult;
+import com.amazonaws.services.kinesisfirehose.model.Record;
+
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"amazon", "aws", "firehose", "kinesis", "put", "stream"})
+@CapabilityDescription("Sends the contents to a specified Amazon Kinesis
Firehose. "
+ + "In order to send data to firehose, the firehose delivery stream
name has to be specified.")
+@WritesAttributes({
+ @WritesAttribute(attribute = "aws.kinesis.firehose.error.message",
description = "Error message on posting message to AWS Kinesis Firehose"),
+ @WritesAttribute(attribute = "aws.kinesis.firehose.error.code",
description = "Error code for the message when posting to AWS Kinesis
Firehose"),
+ @WritesAttribute(attribute = "aws.kinesis.firehose.record.id",
description = "Record id of the message posted to Kinesis Firehose")})
+public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
+
+ /**
+ * Kinesis put record response error message
+ */
+ public static final String AWS_KINESIS_FIREHOSE_ERROR_MESSAGE =
"aws.kinesis.firehose.error.message";
+
+ /**
+ * Kinesis put record response error code
+ */
+ public static final String AWS_KINESIS_FIREHOSE_ERROR_CODE =
"aws.kinesis.firehose.error.code";
+
+ /**
+ * Kinesis put record response record id
+ */
+ public static final String AWS_KINESIS_FIREHOSE_RECORD_ID =
"aws.kinesis.firehose.record.id";
+
+ public static final List<PropertyDescriptor> properties =
Collections.unmodifiableList(
+ Arrays.asList(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME,
MAX_BUFFER_INTERVAL,
+ MAX_BUFFER_SIZE, BATCH_SIZE, REGION, ACCESS_KEY,
SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
+ PROXY_HOST,PROXY_HOST_PORT));
+
+ @Override
+ protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+ return properties;
+ }
+
+ @Override
+ protected Collection<ValidationResult> customValidate(final
ValidationContext validationContext) {
+ final List<ValidationResult> problems = new
ArrayList<>(super.customValidate(validationContext));
+ final boolean batchSizeSet =
validationContext.getProperty(BATCH_SIZE).isSet();
+
+ if ( batchSizeSet) {
+ int batchSize =
validationContext.getProperty(BATCH_SIZE).asInteger();
+ if ( batchSize < 1 || batchSize > 500 ) {
+ problems.add(new ValidationResult.Builder().input("Batch
Size").valid(false).explanation("Batch size must be between 1 and 500 but was "
+ batchSize).build());
+ }
+ }
+
+ final boolean maxBufferIntervalIsSet =
validationContext.getProperty(MAX_BUFFER_INTERVAL).isSet();
+ if ( maxBufferIntervalIsSet) {
+ int maxBufferInterval =
validationContext.getProperty(MAX_BUFFER_INTERVAL).asInteger();
+ if ( maxBufferInterval < 60 || maxBufferInterval > 900 ) {
+ problems.add(new ValidationResult.Builder().input("Max
Buffer Interval").valid(false)
+ .explanation("Max Buffer Interval must be between 60
and 900 seconds but was " + maxBufferInterval).build());
+ }
+ }
+
+ final boolean maxBufferSizeIsSet =
validationContext.getProperty(MAX_BUFFER_SIZE).isSet();
+ if ( maxBufferSizeIsSet) {
+ int maxBufferSize =
validationContext.getProperty(MAX_BUFFER_SIZE).asInteger();
+ if ( maxBufferSize < 1 || maxBufferSize > 128 ) {
+ problems.add(new ValidationResult.Builder().input("Max
Buffer Size").valid(false).explanation("Max Buffer Size must be between 1 and
128 (mb) but was " + maxBufferSize).build());
+ }
+ }
+ return problems;
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context, final
ProcessSession session) {
+ final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
+
+ List<FlowFile> flowFiles = session.get(batchSize);
+ if (flowFiles == null || flowFiles.size() == 0) {
+ return;
+ }
+
+ final String firehoseStreamName =
context.getProperty(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME).getValue();
+ final AmazonKinesisFirehoseClient client = getClient();
+
+ try {
+ List<Record> records = new ArrayList<>();
+
+ // Prepare batch of records
+ for (int i = 0; i < flowFiles.size(); i++) {
+ final ByteArrayOutputStream baos = new
ByteArrayOutputStream();
--- End diff --
This section is a bit concerning as we are taking the contents of the
FlowFiles and putting these items into memory which is going to impact heap.
According to the Kinesis docs
(http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/kinesisfirehose/AmazonKinesisFirehoseClient.html)
it appears the maximum value this can support is 1,000 kB raw (before encoding
occurs). Given the default batch size, even if constrained, would be
~250MB/instance of exertion on the heap.
A couple of points:
- Not overly familiar with the Kinesis API but is there a way to kind of
stream these items directly from the session so we don't have to load them into
memory.
- It would likely be worthwhile to also inspect each flowfile's size to
ensure it does not violate the 1000kB supported. Doing some quick stepping
through, it does not appear this is even detected client side.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---