NIFI-8531: This closes #7529. Upgrading Kinesis processors to AWS SDK 2.x

Signed-off-by: Joseph Witt <joewitt@apache.org>
This commit is contained in:
Joe Gresock 2023-07-26 23:43:03 -04:00 committed by Joseph Witt
parent 9228036e1d
commit ff8f1ff686
No known key found for this signature in database
GPG Key ID: 9093BF854F811A1A
36 changed files with 1320 additions and 898 deletions

View File

@ -43,6 +43,10 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>software.amazon.awssdk</groupId>
<artifactId>kinesis</artifactId>
</dependency>
<dependency>
<groupId>software.amazon.awssdk</groupId>
<artifactId>sns</artifactId>
@ -53,6 +57,25 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>software.amazon.awssdk</groupId>
<artifactId>firehose</artifactId>
</dependency>
<dependency>
<groupId>software.amazon.kinesis</groupId>
<artifactId>amazon-kinesis-client</artifactId>
<version>${aws-kinesis-client-library-version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
<exclusion>
<groupId>org.json</groupId>
<artifactId>json</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-core</artifactId>
@ -61,21 +84,6 @@
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-dynamodb</artifactId>
</dependency>
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-kinesis</artifactId>
</dependency>
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>amazon-kinesis-client</artifactId>
<version>${aws-kinesis-client-library-version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>jcl-over-slf4j</artifactId>

View File

@ -16,34 +16,39 @@
*/
package org.apache.nifi.processors.aws.kinesis;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processors.aws.v2.AbstractAwsProcessor;
import java.util.ArrayList;
import java.util.List;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import com.amazonaws.AmazonWebServiceClient;
/**
* This class provides processor the base class for kinesis client
* This class provides a base for all kinesis processors
*/
public abstract class AbstractBaseKinesisProcessor<ClientType extends AmazonWebServiceClient>
extends AbstractAWSCredentialsProviderProcessor<ClientType> {
public class KinesisProcessorUtils {
private KinesisProcessorUtils() {
}
/**
* Max buffer size 1 MB
*/
public static final int MAX_MESSAGE_SIZE = 1000 * 1024;
private void handleFlowFileTooBig(final ProcessSession session, final FlowFile flowFileCandidate, final String message) {
final FlowFile tooBig = session.putAttribute(flowFileCandidate, message,
"record too big " + flowFileCandidate.getSize() + " max allowed " + MAX_MESSAGE_SIZE);
session.transfer(tooBig, REL_FAILURE);
getLogger().error("Failed to publish to kinesis records {} because the size was greater than {} bytes",
tooBig, MAX_MESSAGE_SIZE);
}
protected List<FlowFile> filterMessagesByMaxSize(final ProcessSession session, final int batchSize, final long maxBufferSizeBytes, String message) {
/**
* Filters messages by max size, transferring any flowfiles larger than the max size to Failure.
* @param session The process session
* @param batchSize The batch size
* @param maxBufferSizeBytes The max buffer size in bytes
* @param errorMessageAttribute The attribute that will contain the error message in case of failure
* @param logger The component log
* @return A list of flowfiles that are less than the maximum errorMessageAttribute size
*/
public static List<FlowFile> filterMessagesByMaxSize(final ProcessSession session, final int batchSize, final long maxBufferSizeBytes,
final String errorMessageAttribute, final ComponentLog logger) {
final List<FlowFile> flowFiles = new ArrayList<>(batchSize);
long currentBufferSizeBytes = 0;
@ -51,7 +56,7 @@ public abstract class AbstractBaseKinesisProcessor<ClientType extends AmazonWebS
final FlowFile flowFileCandidate = session.get();
if (flowFileCandidate != null) {
if (flowFileCandidate.getSize() > MAX_MESSAGE_SIZE) {
handleFlowFileTooBig(session, flowFileCandidate, message);
handleFlowFileTooBig(session, flowFileCandidate, errorMessageAttribute, logger);
continue;
}
@ -62,4 +67,13 @@ public abstract class AbstractBaseKinesisProcessor<ClientType extends AmazonWebS
}
return flowFiles;
}
private static void handleFlowFileTooBig(final ProcessSession session, final FlowFile flowFileCandidate, final String message,
final ComponentLog logger) {
final FlowFile tooBig = session.putAttribute(flowFileCandidate, message,
"record too big " + flowFileCandidate.getSize() + " max allowed " + MAX_MESSAGE_SIZE);
session.transfer(tooBig, AbstractAwsProcessor.REL_FAILURE);
logger.error("Failed to publish to kinesis records {} because the size was greater than {} bytes",
tooBig, MAX_MESSAGE_SIZE);
}
}

View File

@ -20,17 +20,14 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.kinesis.AbstractBaseKinesisProcessor;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.services.kinesisfirehose.AmazonKinesisFirehoseClient;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.firehose.FirehoseClient;
import software.amazon.awssdk.services.firehose.FirehoseClientBuilder;
/**
* This class provides processor the base class for kinesis firehose
* This class is the base class for Kinesis Firehose processors
*/
public abstract class AbstractKinesisFirehoseProcessor extends AbstractBaseKinesisProcessor<AmazonKinesisFirehoseClient> {
public abstract class AbstractKinesisFirehoseProcessor extends AbstractAwsSyncProcessor<FirehoseClient, FirehoseClientBuilder> {
public static final PropertyDescriptor KINESIS_FIREHOSE_DELIVERY_STREAM_NAME = new PropertyDescriptor.Builder()
.name("Amazon Kinesis Firehose Delivery Stream Name")
@ -58,26 +55,8 @@ public abstract class AbstractKinesisFirehoseProcessor extends AbstractBaseKines
.sensitive(false)
.build();
/**
* Create client using aws credentials provider. This is the preferred way for creating clients
*/
@Override
protected AmazonKinesisFirehoseClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials provider");
return new AmazonKinesisFirehoseClient(credentialsProvider, config);
}
/**
* Create client using AWSCredentails
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Deprecated
@Override
protected AmazonKinesisFirehoseClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials");
return new AmazonKinesisFirehoseClient(credentials, config);
protected FirehoseClientBuilder createClientBuilder(final ProcessContext context) {
return FirehoseClient.builder();
}
}

View File

@ -0,0 +1,34 @@
/*
* 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 org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.aws.v2.AbstractAwsAsyncProcessor;
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder;
/**
* This class is the base class for kinesis stream processors that use the async KinesisClient
*/
public abstract class AbstractKinesisStreamAsyncProcessor extends AbstractAwsAsyncProcessor<KinesisAsyncClient, KinesisAsyncClientBuilder>
implements KinesisStreamProcessor {
@Override
protected KinesisAsyncClientBuilder createClientBuilder(final ProcessContext context) {
return KinesisAsyncClient.builder();
}
}

View File

@ -1,64 +0,0 @@
/*
* 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 org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.kinesis.AbstractBaseKinesisProcessor;
/**
* This class provides processor the base class for kinesis client
*/
public abstract class AbstractKinesisStreamProcessor extends AbstractBaseKinesisProcessor<AmazonKinesisClient> {
public static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
.name("kinesis-stream-name")
.displayName("Amazon Kinesis Stream Name")
.description("The name of Kinesis Stream")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
/**
* Create client using aws credentials provider. This is the preferred way for creating clients
*/
@Override
@SuppressWarnings("deprecated")
protected AmazonKinesisClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials provider");
return new AmazonKinesisClient(credentialsProvider, config);
}
/**
* Create client using AWSCredentails
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Override
@Deprecated
protected AmazonKinesisClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials");
return createClient(context, new AWSStaticCredentialsProvider(credentials), config);
}
}

View File

@ -0,0 +1,34 @@
/*
* 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 org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.kinesis.KinesisClient;
import software.amazon.awssdk.services.kinesis.KinesisClientBuilder;
/**
* This class is the base class for kinesis stream processors that use the synchronized KinesisClient
*/
public abstract class AbstractKinesisStreamSyncProcessor extends AbstractAwsSyncProcessor<KinesisClient, KinesisClientBuilder>
implements KinesisStreamProcessor {
@Override
protected KinesisClientBuilder createClientBuilder(final ProcessContext context) {
return KinesisClient.builder();
}
}

View File

@ -0,0 +1,31 @@
/*
* 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 org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.util.StandardValidators;
public interface KinesisStreamProcessor {
PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
.name("kinesis-stream-name")
.displayName("Amazon Kinesis Stream Name")
.description("The name of Kinesis Stream")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
}

View File

@ -21,12 +21,11 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.v2.AbstractAwsProcessor;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sns.SnsClient;
import software.amazon.awssdk.services.sns.SnsClientBuilder;
public abstract class AbstractSNSProcessor extends AbstractAwsProcessor<SnsClient, SnsClientBuilder> {
public abstract class AbstractSNSProcessor extends AbstractAwsSyncProcessor<SnsClient, SnsClientBuilder> {
protected static final AllowableValue ARN_TYPE_TOPIC
= new AllowableValue("Topic ARN", "Topic ARN", "The ARN is the name of a topic");

View File

@ -20,11 +20,11 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.v2.AbstractAwsProcessor;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sqs.SqsClient;
import software.amazon.awssdk.services.sqs.SqsClientBuilder;
public abstract class AbstractSQSProcessor extends AbstractAwsProcessor<SqsClient, SqsClientBuilder> {
public abstract class AbstractSQSProcessor extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")

View File

@ -0,0 +1,104 @@
/*
* 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.v2;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.proxy.ProxyConfiguration;
import software.amazon.awssdk.awscore.client.builder.AwsAsyncClientBuilder;
import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
import software.amazon.awssdk.core.SdkClient;
import software.amazon.awssdk.http.TlsKeyManagersProvider;
import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
import javax.net.ssl.TrustManager;
import java.time.Duration;
/**
* Base class for aws async client processors using the AWS v2 SDK.
*
* @param <T> client type
* @param <U> client builder type
*
* @see <a href="https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/auth/credentials/AwsCredentialsProvider.html">AwsCredentialsProvider</a>
*/
public abstract class AbstractAwsAsyncProcessor<
T extends SdkClient,
U extends AwsAsyncClientBuilder<U, T> & AwsClientBuilder<U, T>
>
extends AbstractAwsProcessor<T> {
/**
* Construct the AWS SDK client builder and perform any service-specific configuration of the builder.
* @param context The process context
* @return The SDK client builder
*/
protected abstract U createClientBuilder(final ProcessContext context);
/**
* Creates the AWS SDK client.
* @param context The process context
* @return The created client
*/
@Override
public T createClient(final ProcessContext context) {
final U clientBuilder = createClientBuilder(context);
this.configureClientBuilder(clientBuilder, context);
return clientBuilder.build();
}
@Override
protected <B extends AwsClientBuilder> void configureHttpClient(final B clientBuilder, final ProcessContext context) {
((AwsAsyncClientBuilder) clientBuilder).httpClient(createSdkAsyncHttpClient(context));
}
private SdkAsyncHttpClient createSdkAsyncHttpClient(final ProcessContext context) {
final NettyNioAsyncHttpClient.Builder builder = NettyNioAsyncHttpClient.builder();
final AwsHttpClientConfigurer configurer = new AwsHttpClientConfigurer() {
@Override
public void configureBasicSettings(final Duration communicationsTimeout, final int maxConcurrentTasks) {
builder.connectionTimeout(communicationsTimeout);
builder.readTimeout(communicationsTimeout);
builder.maxConcurrency(maxConcurrentTasks);
}
@Override
public void configureTls(final TrustManager[] trustManagers, final TlsKeyManagersProvider keyManagersProvider) {
builder.tlsTrustManagersProvider(() -> trustManagers);
builder.tlsKeyManagersProvider(keyManagersProvider);
}
@Override
public void configureProxy(final ProxyConfiguration proxyConfiguration) {
final software.amazon.awssdk.http.nio.netty.ProxyConfiguration.Builder proxyConfigBuilder = software.amazon.awssdk.http.nio.netty.ProxyConfiguration.builder()
.host(proxyConfiguration.getProxyServerHost())
.port(proxyConfiguration.getProxyServerPort());
if (proxyConfiguration.hasCredential()) {
proxyConfigBuilder.username(proxyConfiguration.getProxyUserName());
proxyConfigBuilder.password(proxyConfiguration.getProxyUserPassword());
}
builder.proxyConfiguration(proxyConfigBuilder.build());
}
};
this.configureSdkHttpClient(context, configurer);
return builder.build();
}
}

View File

@ -17,7 +17,7 @@
package org.apache.nifi.processors.aws.v2;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.lifecycle.OnShutdown;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
@ -27,10 +27,13 @@ import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
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.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.credentials.provider.PropertiesCredentialsProvider;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
@ -44,14 +47,12 @@ import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder;
import software.amazon.awssdk.core.SdkClient;
import software.amazon.awssdk.core.client.builder.SdkClientBuilder;
import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
import software.amazon.awssdk.core.retry.RetryPolicy;
import software.amazon.awssdk.http.FileStoreTlsKeyManagersProvider;
import software.amazon.awssdk.http.SdkHttpClient;
import software.amazon.awssdk.http.TlsKeyManagersProvider;
import software.amazon.awssdk.http.apache.ApacheHttpClient;
import software.amazon.awssdk.regions.Region;
import javax.net.ssl.TrustManager;
@ -77,8 +78,8 @@ import java.util.concurrent.TimeUnit;
*
* @see <a href="https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/auth/credentials/AwsCredentialsProvider.html">AwsCredentialsProvider</a>
*/
public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyncClientBuilder<U, T> & AwsClientBuilder<U, T>>
extends AbstractProcessor implements VerifiableProcessor, AwsClientProvider<T> {
public abstract class AbstractAwsProcessor<T extends SdkClient>
extends AbstractSessionFactoryProcessor implements VerifiableProcessor, AwsClientProvider<T> {
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
@ -186,12 +187,38 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
protected volatile T client;
protected volatile Region region;
private final AwsClientCache<T> awsClientCache = new AwsClientCache<>();
/**
* Configure the http client on the builder.
* @param <B> The builder type
* @param clientBuilder The client builder
* @param context The process context
*/
protected abstract <B extends AwsClientBuilder> void configureHttpClient(B clientBuilder, ProcessContext context);
/*
* Allow optional override of onTrigger with the ProcessSessionFactory where required for AWS processors (e.g. ConsumeKinesisStream)
*
* @see AbstractProcessor
*/
@Override
public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
final ProcessSession session = sessionFactory.createSession();
try {
onTrigger(context, session);
session.commitAsync();
} catch (final Throwable t) {
session.rollback(true);
throw t;
}
}
/*
* Default to requiring the "standard" onTrigger with a single ProcessSession
*/
public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
@Override
public Set<Relationship> getRelationships() {
return relationships;
@ -241,11 +268,9 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
return validationResults;
}
@OnShutdown
public void onShutDown() {
if (this.client != null) {
this.client.close();
}
@OnScheduled
public void onScheduled(final ProcessContext context) {
getClient(context);
}
@OnStopped
@ -257,8 +282,7 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
final List<ConfigVerificationResult> results = new ArrayList<>();
try {
createClient(context);
try (final T client = createClient(context)) {
results.add(new ConfigVerificationResult.Builder()
.outcome(Outcome.SUCCESSFUL)
.verificationStepName("Create Client")
@ -276,33 +300,6 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
return results;
}
/**
* Creates the AWS SDK client.
* @param context The process context
* @return The created client
*/
@Override
public T createClient(final ProcessContext context) {
final U clientBuilder = createClientBuilder(context);
this.configureClientBuilder(clientBuilder, context);
return clientBuilder.build();
}
protected void configureClientBuilder(final U clientBuilder, final ProcessContext context) {
clientBuilder.overrideConfiguration(builder -> builder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, DEFAULT_USER_AGENT));
clientBuilder.overrideConfiguration(builder -> builder.retryPolicy(RetryPolicy.none()));
clientBuilder.httpClient(createSdkHttpClient(context));
final Region region = getRegion(context);
if (region != null) {
clientBuilder.region(region);
}
configureEndpoint(context, clientBuilder);
final AwsCredentialsProvider credentialsProvider = getCredentialsProvider(context);
clientBuilder.credentialsProvider(credentialsProvider);
}
/**
* Creates an AWS service client from the context or returns an existing client from the cache
* @param context The process context
@ -318,12 +315,65 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
return getClient(context, awsClientDetails);
}
/**
* Construct the AWS SDK client builder and perform any service-specific configuration of the builder.
* @param context The process context
* @return The SDK client builder
*/
protected abstract U createClientBuilder(final ProcessContext context);
protected <C extends SdkClient, B extends AwsClientBuilder<B, C>>
void configureClientBuilder(final B clientBuilder, final ProcessContext context) {
configureClientBuilder(clientBuilder, context, ENDPOINT_OVERRIDE);
}
protected <C extends SdkClient, B extends AwsClientBuilder<B, C>>
void configureClientBuilder(final B clientBuilder, final ProcessContext context, final PropertyDescriptor endpointOverrideDescriptor) {
clientBuilder.overrideConfiguration(builder -> builder.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, DEFAULT_USER_AGENT));
clientBuilder.overrideConfiguration(builder -> builder.retryPolicy(RetryPolicy.none()));
this.configureHttpClient(clientBuilder, context);
final Region region = getRegion(context);
if (region != null) {
clientBuilder.region(region);
}
configureEndpoint(context, clientBuilder, endpointOverrideDescriptor);
final AwsCredentialsProvider credentialsProvider = getCredentialsProvider(context);
clientBuilder.credentialsProvider(credentialsProvider);
}
protected void configureSdkHttpClient(final ProcessContext context, final AwsHttpClientConfigurer httpClientConfigurer) {
final int communicationsTimeout = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
httpClientConfigurer.configureBasicSettings(Duration.ofMillis(communicationsTimeout), context.getMaxConcurrentTasks());
if (this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null) {
final TrustManager[] trustManagers = new TrustManager[] { sslContextService.createTrustManager() };
final TlsKeyManagersProvider keyManagersProvider = FileStoreTlsKeyManagersProvider
.create(Path.of(sslContextService.getKeyStoreFile()), sslContextService.getKeyStoreType(), sslContextService.getKeyStorePassword());
httpClientConfigurer.configureTls(trustManagers, keyManagersProvider);
}
}
final ProxyConfiguration proxyConfig = ProxyConfiguration.getConfiguration(context, () -> {
if (context.getProperty(PROXY_HOST).isSet()) {
final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
final String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
final Integer proxyPort = context.getProperty(PROXY_HOST_PORT).evaluateAttributeExpressions().asInteger();
final String proxyUsername = context.getProperty(PROXY_USERNAME).evaluateAttributeExpressions().getValue();
final String proxyPassword = context.getProperty(PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
componentProxyConfig.setProxyType(Proxy.Type.HTTP);
componentProxyConfig.setProxyServerHost(proxyHost);
componentProxyConfig.setProxyServerPort(proxyPort);
componentProxyConfig.setProxyUserName(proxyUsername);
componentProxyConfig.setProxyUserPassword(proxyPassword);
return componentProxyConfig;
} else if (context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet()) {
final ProxyConfigurationService configurationService = context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).asControllerService(ProxyConfigurationService.class);
return configurationService.getConfiguration();
}
return ProxyConfiguration.DIRECT_CONFIGURATION;
});
if (Proxy.Type.HTTP.equals(proxyConfig.getProxyType())) {
httpClientConfigurer.configureProxy(proxyConfig);
}
}
protected Region getRegion(final ProcessContext context) {
final Region region;
@ -341,11 +391,11 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
return region;
}
protected void configureEndpoint(final ProcessContext context, final U clientBuilder) {
protected void configureEndpoint(final ProcessContext context, final SdkClientBuilder clientBuilder, final PropertyDescriptor endpointOverrideDescriptor) {
// if the endpoint override has been configured, set the endpoint.
// (per Amazon docs this should only be configured at client creation)
if (getSupportedPropertyDescriptors().contains(ENDPOINT_OVERRIDE)) {
final String endpointOverride = StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue());
if (getSupportedPropertyDescriptors().contains(endpointOverrideDescriptor)) {
final String endpointOverride = StringUtils.trimToEmpty(context.getProperty(endpointOverrideDescriptor).evaluateAttributeExpressions().getValue());
if (!endpointOverride.isEmpty()) {
getLogger().info("Overriding endpoint with {}", endpointOverride);
@ -355,6 +405,10 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
}
}
protected void configureEndpoint(final ProcessContext context, final SdkClientBuilder clientBuilder) {
configureEndpoint(context, clientBuilder, ENDPOINT_OVERRIDE);
}
/**
* Get credentials provider using the {@link AwsCredentialsProvider}
* @param context the process context
@ -384,57 +438,4 @@ public abstract class AbstractAwsProcessor<T extends SdkClient, U extends AwsSyn
return AnonymousCredentialsProvider.create();
}
private SdkHttpClient createSdkHttpClient(final ProcessContext context) {
final ApacheHttpClient.Builder builder = ApacheHttpClient.builder();
final int communicationsTimeout = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
builder.connectionTimeout(Duration.ofMillis(communicationsTimeout));
builder.socketTimeout(Duration.ofMillis(communicationsTimeout));
builder.maxConnections(context.getMaxConcurrentTasks());
if (this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null) {
final TrustManager[] trustManagers = new TrustManager[] { sslContextService.createTrustManager() };
final TlsKeyManagersProvider keyManagersProvider = FileStoreTlsKeyManagersProvider
.create(Path.of(sslContextService.getKeyStoreFile()), sslContextService.getKeyStoreType(), sslContextService.getKeyStorePassword());
builder.tlsTrustManagersProvider(() -> trustManagers);
builder.tlsKeyManagersProvider(keyManagersProvider);
}
}
final ProxyConfiguration proxyConfig = ProxyConfiguration.getConfiguration(context, () -> {
if (context.getProperty(PROXY_HOST).isSet()) {
final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
final String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
final Integer proxyPort = context.getProperty(PROXY_HOST_PORT).evaluateAttributeExpressions().asInteger();
final String proxyUsername = context.getProperty(PROXY_USERNAME).evaluateAttributeExpressions().getValue();
final String proxyPassword = context.getProperty(PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
componentProxyConfig.setProxyType(Proxy.Type.HTTP);
componentProxyConfig.setProxyServerHost(proxyHost);
componentProxyConfig.setProxyServerPort(proxyPort);
componentProxyConfig.setProxyUserName(proxyUsername);
componentProxyConfig.setProxyUserPassword(proxyPassword);
return componentProxyConfig;
} else if (context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet()) {
final ProxyConfigurationService configurationService = context.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).asControllerService(ProxyConfigurationService.class);
return configurationService.getConfiguration();
}
return ProxyConfiguration.DIRECT_CONFIGURATION;
});
if (Proxy.Type.HTTP.equals(proxyConfig.getProxyType())) {
final software.amazon.awssdk.http.apache.ProxyConfiguration.Builder proxyConfigBuilder = software.amazon.awssdk.http.apache.ProxyConfiguration.builder()
.endpoint(URI.create(String.format("%s:%s", proxyConfig.getProxyServerHost(), proxyConfig.getProxyServerPort())));
if (proxyConfig.hasCredential()) {
proxyConfigBuilder.username(proxyConfig.getProxyUserName());
proxyConfigBuilder.password(proxyConfig.getProxyUserPassword());
}
builder.proxyConfiguration(proxyConfigBuilder.build());
}
return builder.build();
}
}

View File

@ -0,0 +1,105 @@
/*
* 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.v2;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.proxy.ProxyConfiguration;
import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder;
import software.amazon.awssdk.core.SdkClient;
import software.amazon.awssdk.http.SdkHttpClient;
import software.amazon.awssdk.http.TlsKeyManagersProvider;
import software.amazon.awssdk.http.apache.ApacheHttpClient;
import javax.net.ssl.TrustManager;
import java.net.URI;
import java.time.Duration;
/**
* Base class for aws sync client processors using the AWS v2 SDK.
*
* @param <T> client type
* @param <U> client builder type
*
* @see <a href="https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/auth/credentials/AwsCredentialsProvider.html">AwsCredentialsProvider</a>
*/
public abstract class AbstractAwsSyncProcessor<
T extends SdkClient,
U extends AwsSyncClientBuilder<U, T> & AwsClientBuilder<U, T>
>
extends AbstractAwsProcessor<T> {
/**
* Construct the AWS SDK client builder and perform any service-specific configuration of the builder.
* @param context The process context
* @return The SDK client builder
*/
protected abstract U createClientBuilder(final ProcessContext context);
/**
* Creates the AWS SDK client.
* @param context The process context
* @return The created client
*/
@Override
public T createClient(final ProcessContext context) {
final U clientBuilder = createClientBuilder(context);
this.configureClientBuilder(clientBuilder, context);
return clientBuilder.build();
}
@Override
protected <B extends AwsClientBuilder> void configureHttpClient(final B clientBuilder, final ProcessContext context) {
((AwsSyncClientBuilder) clientBuilder).httpClient(createSdkHttpClient(context));
}
private SdkHttpClient createSdkHttpClient(final ProcessContext context) {
final ApacheHttpClient.Builder builder = ApacheHttpClient.builder();
final AwsHttpClientConfigurer configurer = new AwsHttpClientConfigurer() {
@Override
public void configureBasicSettings(final Duration communicationsTimeout, final int maxConcurrentTasks) {
builder.connectionTimeout(communicationsTimeout);
builder.socketTimeout(communicationsTimeout);
builder.maxConnections(context.getMaxConcurrentTasks());
}
@Override
public void configureTls(final TrustManager[] trustManagers, final TlsKeyManagersProvider keyManagersProvider) {
builder.tlsTrustManagersProvider(() -> trustManagers);
builder.tlsKeyManagersProvider(keyManagersProvider);
}
@Override
public void configureProxy(final ProxyConfiguration proxyConfiguration) {
final software.amazon.awssdk.http.apache.ProxyConfiguration.Builder proxyConfigBuilder = software.amazon.awssdk.http.apache.ProxyConfiguration.builder()
.endpoint(URI.create(String.format("%s:%s", proxyConfiguration.getProxyServerHost(), proxyConfiguration.getProxyServerPort())));
if (proxyConfiguration.hasCredential()) {
proxyConfigBuilder.username(proxyConfiguration.getProxyUserName());
proxyConfigBuilder.password(proxyConfiguration.getProxyUserPassword());
}
builder.proxyConfiguration(proxyConfigBuilder.build());
}
};
this.configureSdkHttpClient(context, configurer);
return builder.build();
}
}

View File

@ -29,7 +29,12 @@ public class AwsClientCache<T extends SdkClient> {
return clientCache.get(clientDetails, ignored -> provider.createClient(context));
}
public void closeClients() {
clientCache.asMap().values().stream().forEach(SdkClient::close);
}
public void clearCache() {
closeClients();
clientCache.invalidateAll();
clientCache.cleanUp();
}

View File

@ -0,0 +1,32 @@
/*
* 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.v2;
import org.apache.nifi.proxy.ProxyConfiguration;
import software.amazon.awssdk.http.TlsKeyManagersProvider;
import javax.net.ssl.TrustManager;
import java.time.Duration;
public interface AwsHttpClientConfigurer {
void configureBasicSettings(Duration communicationsTimeout, int maxConcurrentTasks);
void configureTls(TrustManager[] trustManagers, TlsKeyManagersProvider keyManagersProvider);
void configureProxy(ProxyConfiguration proxyConfiguration);
}

View File

@ -16,15 +16,6 @@
*/
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.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;
@ -38,11 +29,20 @@ import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.DataUnit;
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;
import org.apache.nifi.processors.aws.kinesis.KinesisProcessorUtils;
import software.amazon.awssdk.core.SdkBytes;
import software.amazon.awssdk.services.firehose.FirehoseClient;
import software.amazon.awssdk.services.firehose.model.PutRecordBatchRequest;
import software.amazon.awssdk.services.firehose.model.PutRecordBatchResponse;
import software.amazon.awssdk.services.firehose.model.PutRecordBatchResponseEntry;
import software.amazon.awssdk.services.firehose.model.Record;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@SupportsBatching
@InputRequirement(Requirement.INPUT_REQUIRED)
@ -90,15 +90,15 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
final long maxBufferSizeBytes = context.getProperty(MAX_MESSAGE_BUFFER_SIZE_MB).asDataSize(DataUnit.B).longValue();
List<FlowFile> flowFiles = filterMessagesByMaxSize(session, batchSize, maxBufferSizeBytes, AWS_KINESIS_FIREHOSE_ERROR_MESSAGE);
HashMap<String, List<FlowFile>> hashFlowFiles = new HashMap<>();
HashMap<String, List<Record>> recordHash = new HashMap<String, List<Record>>();
final List<FlowFile> flowFiles = KinesisProcessorUtils.filterMessagesByMaxSize(session, batchSize, maxBufferSizeBytes, AWS_KINESIS_FIREHOSE_ERROR_MESSAGE, getLogger());
final Map<String, List<FlowFile>> hashFlowFiles = new HashMap<>();
final Map<String, List<Record>> recordHash = new HashMap<>();
final AmazonKinesisFirehoseClient client = getClient(context);
final FirehoseClient client = getClient(context);
try {
List<FlowFile> failedFlowFiles = new ArrayList<>();
List<FlowFile> successfulFlowFiles = new ArrayList<>();
final List<FlowFile> failedFlowFiles = new ArrayList<>();
final List<FlowFile> successfulFlowFiles = new ArrayList<>();
// Prepare batch of records
for (int i = 0; i < flowFiles.size(); i++) {
@ -106,8 +106,7 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
final String firehoseStreamName = context.getProperty(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME).evaluateAttributeExpressions(flowFile).getValue();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
session.exportTo(flowFile, baos);
session.read(flowFile, in -> recordHash.get(firehoseStreamName).add(Record.builder().data(SdkBytes.fromInputStream(in)).build()));
if (recordHash.containsKey(firehoseStreamName) == false) {
recordHash.put(firehoseStreamName, new ArrayList<>());
@ -118,33 +117,33 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
}
hashFlowFiles.get(firehoseStreamName).add(flowFile);
recordHash.get(firehoseStreamName).add(new Record().withData(ByteBuffer.wrap(baos.toByteArray())));
}
for (Map.Entry<String, List<Record>> entryRecord : recordHash.entrySet()) {
String streamName = entryRecord.getKey();
List<Record> records = entryRecord.getValue();
for (final Map.Entry<String, List<Record>> entryRecord : recordHash.entrySet()) {
final String streamName = entryRecord.getKey();
final List<Record> records = entryRecord.getValue();
if (records.size() > 0) {
// Send the batch
PutRecordBatchRequest putRecordBatchRequest = new PutRecordBatchRequest();
putRecordBatchRequest.setDeliveryStreamName(streamName);
putRecordBatchRequest.setRecords(records);
PutRecordBatchResult results = client.putRecordBatch(putRecordBatchRequest);
final PutRecordBatchRequest putRecordBatchRequest = PutRecordBatchRequest.builder()
.deliveryStreamName(streamName)
.records(records)
.build();
final PutRecordBatchResponse response = client.putRecordBatch(putRecordBatchRequest);
// Separate out the successful and failed flow files
List<PutRecordBatchResponseEntry> responseEntries = results.getRequestResponses();
final List<PutRecordBatchResponseEntry> responseEntries = response.requestResponses();
for (int i = 0; i < responseEntries.size(); i++ ) {
PutRecordBatchResponseEntry entry = responseEntries.get(i);
final PutRecordBatchResponseEntry entry = responseEntries.get(i);
FlowFile flowFile = hashFlowFiles.get(streamName).get(i);
Map<String,String> attributes = new HashMap<>();
attributes.put(AWS_KINESIS_FIREHOSE_RECORD_ID, entry.getRecordId());
flowFile = session.putAttribute(flowFile, AWS_KINESIS_FIREHOSE_RECORD_ID, entry.getRecordId());
if (StringUtils.isBlank(entry.getErrorCode()) == false) {
attributes.put(AWS_KINESIS_FIREHOSE_ERROR_CODE, entry.getErrorCode());
attributes.put(AWS_KINESIS_FIREHOSE_ERROR_MESSAGE, entry.getErrorMessage());
final Map<String,String> attributes = new HashMap<>();
attributes.put(AWS_KINESIS_FIREHOSE_RECORD_ID, entry.recordId());
flowFile = session.putAttribute(flowFile, AWS_KINESIS_FIREHOSE_RECORD_ID, entry.recordId());
if (StringUtils.isNotBlank(entry.errorCode())) {
attributes.put(AWS_KINESIS_FIREHOSE_ERROR_CODE, entry.errorCode());
attributes.put(AWS_KINESIS_FIREHOSE_ERROR_MESSAGE, entry.errorMessage());
flowFile = session.putAllAttributes(flowFile, attributes);
failedFlowFiles.add(flowFile);
} else {

View File

@ -16,12 +16,7 @@
*/
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 com.google.common.annotations.VisibleForTesting;
import org.apache.commons.beanutils.BeanUtilsBean;
import org.apache.commons.beanutils.ConvertUtilsBean2;
import org.apache.commons.beanutils.FluentPropertyBeanIntrospector;
@ -52,13 +47,34 @@ 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.processors.aws.v2.AbstractAwsProcessor;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.RecordSetWriterFactory;
import org.apache.nifi.serialization.record.RecordFieldType;
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient;
import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClientBuilder;
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClientBuilder;
import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
import software.amazon.kinesis.checkpoint.CheckpointConfig;
import software.amazon.kinesis.common.ConfigsBuilder;
import software.amazon.kinesis.common.InitialPositionInStream;
import software.amazon.kinesis.common.InitialPositionInStreamExtended;
import software.amazon.kinesis.coordinator.CoordinatorConfig;
import software.amazon.kinesis.coordinator.Scheduler;
import software.amazon.kinesis.coordinator.WorkerStateChangeListener;
import software.amazon.kinesis.leases.LeaseManagementConfig;
import software.amazon.kinesis.lifecycle.LifecycleConfig;
import software.amazon.kinesis.metrics.MetricsConfig;
import software.amazon.kinesis.metrics.NullMetricsFactory;
import software.amazon.kinesis.processor.ProcessorConfig;
import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
import software.amazon.kinesis.retrieval.RetrievalConfig;
import software.amazon.kinesis.retrieval.polling.PollingConfig;
import java.lang.reflect.InvocationTargetException;
import java.net.InetAddress;
@ -113,9 +129,9 @@ import java.util.stream.Collectors;
})
@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, " +
description="Override default KCL Configuration ConfigsBuilder properties with required values. Supports setting of values directly on " +
"the ConfigsBuilder, such as 'namespace', as well as properties on nested builders. For example, to set configsBuilder.retrievalConfig().maxListShardsRetryAttempts(value), " +
"name the property as 'retrievalConfig.maxListShardsRetryAttempts'. 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")
})
@ -126,7 +142,15 @@ import java.util.stream.Collectors;
@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 {
public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
private static final String CHECKPOINT_CONFIG = "checkpointConfig";
private static final String COORDINATOR_CONFIG = "coordinatorConfig";
private static final String LEASE_MANAGEMENT_CONFIG = "leaseManagementConfig";
private static final String LIFECYCLE_CONFIG = "lifecycleConfig";
private static final String METRICS_CONFIG = "metricsConfig";
private static final String PROCESSOR_CONFIG = "processorConfig";
private static final String RETRIEVAL_CONFIG = "retrievalConfig";
static final AllowableValue TRIM_HORIZON = new AllowableValue(
InitialPositionInStream.TRIM_HORIZON.toString(),
InitialPositionInStream.TRIM_HORIZON.toString(),
@ -270,7 +294,7 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
.build();
public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
.fromPropertyDescriptor(AbstractAwsProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE)
.required(true)
.build();
@ -292,27 +316,23 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
)
);
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 Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = new HashMap<>() {{
put("leaseManagementConfig.initialPositionInStream", INITIAL_STREAM_POSITION);
put("leaseManagementConfig.failoverTimeMillis", FAILOVER_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 String SCHEDULER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Scheduler.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"));
PROPERTY_UTILS_BEAN.addBeanIntrospector(new FluentPropertyBeanIntrospector(""));
final ConvertUtilsBean2 convertUtilsBean2 = new ConvertUtilsBean2() {
@SuppressWarnings("unchecked") // generic Enum conversion from String property values
@ -332,7 +352,7 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
private volatile boolean isRecordReaderSet;
private volatile boolean isRecordWriterSet;
private volatile Worker worker;
private volatile Scheduler scheduler;
final AtomicReference<WorkerStateChangeListener.WorkerState> workerState = new AtomicReference<>(null);
private final AtomicBoolean stopped = new AtomicBoolean(false);
@ -368,12 +388,17 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
return builder.build();
}
@OnStopped
public void onStopped() {
super.onStopped();
}
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*$")) {
if (!subject.matches("^(?!with)[a-zA-Z]\\w*(\\.[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")
.explanation("Property name must not have a prefix of \"with\", must start with a letter and contain only letters, numbers, periods, or underscores")
.valid(false).build();
}
@ -383,22 +408,48 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
.valid(false).build();
}
final KinesisClientLibConfiguration kclTemp = new KinesisClientLibConfiguration("validate", "validate", null, "validate");
final Region region = Region.of(context.getProperty(REGION).getValue());
// This is a temporary builder that is not used outside of validation
final ConfigsBuilder configsBuilderTemp = new ConfigsBuilder(
getStreamName(context),
getApplicationName(context),
KinesisAsyncClient.builder().region(region).build(),
DynamoDbAsyncClient.builder().region(region).build(),
CloudWatchAsyncClient.builder().region(region).build(),
UUID.randomUUID().toString(),
() -> null
);
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();
if (subject.contains(".")) {
final String[] beanParts = subject.split("\\.");
if (beanParts.length != 2) {
throw new IllegalArgumentException("Kinesis Client Configuration Builder properties only support one level of nesting");
}
final String configurationMethod = beanParts[0];
final String setterMethod = StringUtils.uncapitalize(beanParts[1]);
final Object configurationObject = configsBuilderTemp.getClass().getMethod(configurationMethod).invoke(configsBuilderTemp);
if (!PROPERTY_UTILS_BEAN.isWriteable(configurationObject, setterMethod)) {
return validationResult
.explanation(String.format("Kinesis Client Configuration Builder property with name %s does not exist or is not writable", StringUtils.capitalize(subject)))
.valid(false).build();
}
BEAN_UTILS_BEAN.setProperty(configurationObject, setterMethod, input);
} else {
final String propName = StringUtils.uncapitalize(subject);
if (!PROPERTY_UTILS_BEAN.isWriteable(configsBuilderTemp, propName)) {
return validationResult
.explanation(String.format("Kinesis Client Configuration Builder property with name %s does not exist or is not writable", StringUtils.capitalize(subject)))
.valid(false).build();
}
BEAN_UTILS_BEAN.setProperty(configsBuilderTemp, propName, input);
}
BEAN_UTILS_BEAN.setProperty(kclTemp, propName, input);
} catch (IllegalAccessException e) {
} catch (final IllegalAccessException | NoSuchMethodException e) {
return validationResult
.explanation(String.format("Kinesis Client Library Configuration property with name %s is not accessible", StringUtils.capitalize(subject)))
.explanation(String.format("Kinesis Client Configuration Builder property with name %s is not accessible", StringUtils.capitalize(subject)))
.valid(false).build();
} catch (InvocationTargetException e) {
} catch (final InvocationTargetException e) {
return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getTargetException().getLocalizedMessage());
} catch (IllegalArgumentException e) {
} catch (final IllegalArgumentException e) {
return buildDynamicPropertyBeanValidationResult(validationResult, subject, input, e.getLocalizedMessage());
}
@ -409,7 +460,7 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
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",
String.format("Kinesis Client Configuration Builder property with name %s cannot be used with value \"%s\" : %s",
StringUtils.capitalize(subject), input, message)
)
.valid(false).build();
@ -481,25 +532,25 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
@Override
public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
if (worker == null) {
if (scheduler == 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();
if (scheduler == null) {
final String schedulerId = generateSchedulerId();
getLogger().info("Starting Kinesis Scheduler {}", schedulerId);
// create scheduler (WorkerState will be CREATED)
scheduler = prepareScheduler(context, sessionFactory, schedulerId);
// initialise and start Scheduler (will set WorkerState to INITIALIZING and attempt to start)
new Thread(scheduler, SCHEDULER_THREAD_NAME_TEMPLATE + schedulerId).start();
}
}
} else {
// after a Worker is registered successfully, nothing has to be done at onTrigger
// new sessions are created when new messages are consumed by the Worker
// after a Scheduler is registered successfully, nothing has to be done at onTrigger
// new sessions are created when new messages are consumed by the Scheduler
// and if the WorkerState is unexpectedly SHUT_DOWN, then we don't want to immediately re-enter onTrigger
context.yield();
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");
throw new ProcessException("Scheduler has shutdown unexpectedly, possibly due to a configuration issue; check logs for details");
}
}
}
@ -511,15 +562,15 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
@OnStopped
public void stopConsuming(final ProcessContext context) {
if (worker != null) {
if (scheduler != null) {
synchronized (WORKER_LOCK) {
if (worker != null) {
if (scheduler != null) {
// indicate whether the processor has been Stopped; the Worker can be marked as SHUT_DOWN but still be waiting
// for ShardConsumers/RecordProcessors to complete, etc.
stopped.set(true);
final boolean success = shutdownWorker(context);
worker = null;
scheduler = null;
workerState.set(null);
if (!success) {
@ -530,65 +581,27 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
}
}
private synchronized Worker prepareWorker(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String workerId) {
final IRecordProcessorFactory factory = prepareRecordProcessorFactory(context, sessionFactory);
@VisibleForTesting
synchronized Scheduler prepareScheduler(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String schedulerId) {
final KinesisAsyncClient kinesisClient = getClient(context);
final ConfigsBuilder configsBuilder = prepareConfigsBuilder(context, schedulerId, 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(context).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(context).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")
final KinesisClientLibConfiguration kinesisClientLibConfiguration = new KinesisClientLibConfiguration(
getApplicationName(context),
getStreamName(context),
getCredentialsProvider(context),
workerId
)
.withCommonClientConfig(getClient(context).getClientConfiguration())
.withRegionName(getRegion(context).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);
final MetricsConfig metricsConfig = configsBuilder.metricsConfig();
if (!isReportCloudWatchMetrics(context)) {
metricsConfig.metricsFactory(new NullMetricsFactory());
}
getDynamoDBOverride(context).ifPresent(kinesisClientLibConfiguration::withDynamoDBEndpoint);
getKinesisEndpoint(context).ifPresent(kinesisClientLibConfiguration::withKinesisEndpoint);
final String streamName = getStreamName(context);
final LeaseManagementConfig leaseManagementConfig = configsBuilder.leaseManagementConfig()
.failoverTimeMillis(getFailoverTimeMillis(context))
.streamName(streamName);
final CoordinatorConfig coordinatorConfig = configsBuilder.coordinatorConfig().workerStateChangeListener(workerState::set);
final InitialPositionInStream initialPositionInStream = getInitialPositionInStream(context);
final InitialPositionInStreamExtended initialPositionInStreamValue = (InitialPositionInStream.AT_TIMESTAMP == initialPositionInStream)
? InitialPositionInStreamExtended.newInitialPositionAtTimestamp(getStartStreamTimestamp(context))
: InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream);
leaseManagementConfig.initialPositionInStream(initialPositionInStreamValue);
final List<PropertyDescriptor> dynamicProperties = context.getProperties()
.keySet()
@ -596,40 +609,115 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
.filter(PropertyDescriptor::isDynamic)
.collect(Collectors.toList());
final RetrievalConfig retrievalConfig = configsBuilder.retrievalConfig()
.retrievalSpecificConfig(new PollingConfig(streamName, kinesisClient));
final Map<String, Object> configMap = new HashMap<>();
configMap.put(CHECKPOINT_CONFIG, configsBuilder.checkpointConfig());
configMap.put(COORDINATOR_CONFIG, coordinatorConfig);
configMap.put(LEASE_MANAGEMENT_CONFIG, leaseManagementConfig);
configMap.put(LIFECYCLE_CONFIG, configsBuilder.lifecycleConfig());
configMap.put(METRICS_CONFIG, metricsConfig);
configMap.put(PROCESSOR_CONFIG, configsBuilder.processorConfig());
configMap.put(RETRIEVAL_CONFIG, retrievalConfig);
dynamicProperties.forEach(descriptor -> {
final String name = descriptor.getName();
final String value = context.getProperty(descriptor).getValue();
try {
BEAN_UTILS_BEAN.setProperty(kinesisClientLibConfiguration, StringUtils.uncapitalize(name), value);
} catch (IllegalAccessException | InvocationTargetException e) {
throw new ProcessException(String.format("Unable to set Kinesis Client Library Configuration property %s with value %s", StringUtils.capitalize(name), value), e);
if (name.contains(".")) {
final String[] beanParts = name.split("\\.");
if (beanParts.length != 2) {
throw new IllegalArgumentException("Kinesis Client Configuration Builder properties only support one level of nesting");
}
final String configurationMethod = beanParts[0];
final String setterMethod = beanParts[1];
final Object configurationObject = configMap.get(configurationMethod);
if (configurationObject == null) {
throw new IllegalArgumentException("Kinesis Client Configuration Builder does not have a configuration method named " + configurationMethod);
}
BEAN_UTILS_BEAN.setProperty(configurationObject, StringUtils.uncapitalize(setterMethod), value);
} else {
BEAN_UTILS_BEAN.setProperty(configsBuilder, StringUtils.uncapitalize(name), value);
}
} catch (final IllegalAccessException | InvocationTargetException e) {
throw new ProcessException(String.format("Unable to set Kinesis Client Configuration Builder property %s with value %s", StringUtils.capitalize(name), value), e);
}
});
return kinesisClientLibConfiguration;
getLogger().info("Kinesis Scheduler prepared for application {} to process stream {} as scheduler ID {}...",
getApplicationName(context), streamName, schedulerId);
return new Scheduler(
(CheckpointConfig) configMap.get(CHECKPOINT_CONFIG),
(CoordinatorConfig) configMap.get(COORDINATOR_CONFIG),
(LeaseManagementConfig) configMap.get(LEASE_MANAGEMENT_CONFIG),
(LifecycleConfig) configMap.get(LIFECYCLE_CONFIG),
(MetricsConfig) configMap.get(METRICS_CONFIG),
(ProcessorConfig) configMap.get(PROCESSOR_CONFIG),
(RetrievalConfig) configMap.get(RETRIEVAL_CONFIG)
);
}
Worker.Builder prepareWorkerBuilder(final ProcessContext context, final KinesisClientLibConfiguration kinesisClientLibConfiguration,
final IRecordProcessorFactory factory) {
final Worker.Builder workerBuilder = new Worker.Builder()
.config(kinesisClientLibConfiguration)
.kinesisClient(getClient(context))
.workerStateChangeListener(workerState::set)
.recordProcessorFactory(factory);
private ShardRecordProcessorFactory prepareRecordProcessorFactory(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
return () -> {
if (isRecordReaderSet && isRecordWriterSet) {
return new KinesisRecordProcessorRecord(
sessionFactory, getLogger(), getStreamName(context), getEndpointPrefix(context),
getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context),
getReaderFactory(context), getWriterFactory(context)
);
} else {
return new KinesisRecordProcessorRaw(
sessionFactory, getLogger(), getStreamName(context), getEndpointPrefix(context),
getKinesisEndpoint(context).orElse(null), getCheckpointIntervalMillis(context),
getRetryWaitMillis(context), getNumRetries(context), getDateTimeFormatter(context)
);
}
};
}
if (!isReportCloudWatchMetrics(context)) {
workerBuilder.metricsFactory(new NullMetricsFactory());
}
private String getEndpointPrefix(final ProcessContext context) {
return "kinesis." + getRegion(context).id().toLowerCase();
}
return workerBuilder;
/*
* 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
*/
@VisibleForTesting
ConfigsBuilder prepareConfigsBuilder(final ProcessContext context, final String workerId, final ProcessSessionFactory sessionFactory) {
return new ConfigsBuilder(
getStreamName(context),
getApplicationName(context),
getClient(context),
getDynamoClient(context),
getCloudwatchClient(context),
workerId,
prepareRecordProcessorFactory(context, sessionFactory)
);
}
private CloudWatchAsyncClient getCloudwatchClient(final ProcessContext context) {
final CloudWatchAsyncClientBuilder builder = CloudWatchAsyncClient.builder();
configureClientBuilder(builder, context, null);
return builder.build();
}
private DynamoDbAsyncClient getDynamoClient(final ProcessContext context) {
final DynamoDbAsyncClientBuilder dynamoClientBuilder = DynamoDbAsyncClient.builder();
configureClientBuilder(dynamoClientBuilder, context, DYNAMODB_ENDPOINT_OVERRIDE);
return dynamoClientBuilder.build();
}
private boolean shutdownWorker(final ProcessContext context) {
boolean success = true;
try {
if (!worker.hasGracefulShutdownStarted()) {
if (!scheduler.hasGracefulShutdownStarted()) {
getLogger().info("Requesting Kinesis Worker shutdown");
final Future<Boolean> shutdown = worker.startGracefulShutdown();
final Future<Boolean> shutdown = scheduler.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");
@ -638,14 +726,14 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
getLogger().info("Kinesis Worker shutdown");
}
}
} catch (InterruptedException | TimeoutException | ExecutionException e) {
} catch (final InterruptedException | TimeoutException | ExecutionException e) {
getLogger().warn("Problem while shutting down Kinesis Worker: {}", e.getLocalizedMessage(), e);
success = false;
}
return success;
}
private String generateWorkerId() {
private String generateSchedulerId() {
try {
return InetAddress.getLocalHost().getCanonicalHostName() + ":" + UUID.randomUUID();
} catch (UnknownHostException e) {
@ -691,12 +779,6 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamProcessor {
: 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);
}

View File

@ -16,11 +16,6 @@
*/
package org.apache.nifi.processors.aws.kinesis.stream;
import com.amazonaws.services.kinesis.AmazonKinesisClient;
import com.amazonaws.services.kinesis.model.PutRecordsRequest;
import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry;
import com.amazonaws.services.kinesis.model.PutRecordsResult;
import com.amazonaws.services.kinesis.model.PutRecordsResultEntry;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@ -37,9 +32,15 @@ import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.kinesis.KinesisProcessorUtils;
import software.amazon.awssdk.core.SdkBytes;
import software.amazon.awssdk.services.kinesis.KinesisClient;
import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest;
import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
import software.amazon.awssdk.services.kinesis.model.PutRecordsResponse;
import software.amazon.awssdk.services.kinesis.model.PutRecordsResultEntry;
import java.io.ByteArrayOutputStream;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -59,7 +60,7 @@ import java.util.Random;
@WritesAttribute(attribute = "aws.kinesis.sequence.number", description = "Sequence number for the message when posting to AWS Kinesis"),
@WritesAttribute(attribute = "aws.kinesis.shard.id", description = "Shard id of the message posted to AWS Kinesis")})
@SeeAlso(ConsumeKinesisStream.class)
public class PutKinesisStream extends AbstractKinesisStreamProcessor {
public class PutKinesisStream extends AbstractKinesisStreamSyncProcessor {
/**
* Kinesis put record response error message
*/
@ -104,7 +105,7 @@ public class PutKinesisStream extends AbstractKinesisStreamProcessor {
.build();
public static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AbstractKinesisStreamProcessor.KINESIS_STREAM_NAME)
.fromPropertyDescriptor(AbstractKinesisStreamSyncProcessor.KINESIS_STREAM_NAME)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
@ -126,72 +127,58 @@ public class PutKinesisStream extends AbstractKinesisStreamProcessor {
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
final long maxBufferSizeBytes = context.getProperty(MAX_MESSAGE_BUFFER_SIZE_MB).asDataSize(DataUnit.B).longValue();
List<FlowFile> flowFiles = filterMessagesByMaxSize(session, batchSize, maxBufferSizeBytes, AWS_KINESIS_ERROR_MESSAGE);
final List<FlowFile> flowFiles = KinesisProcessorUtils.filterMessagesByMaxSize(session, batchSize, maxBufferSizeBytes, AWS_KINESIS_ERROR_MESSAGE, getLogger());
HashMap<String, List<FlowFile>> hashFlowFiles = new HashMap<>();
HashMap<String, List<PutRecordsRequestEntry>> recordHash = new HashMap<String, List<PutRecordsRequestEntry>>();
final AmazonKinesisClient client = getClient(context);
final HashMap<String, List<FlowFile>> hashFlowFiles = new HashMap<>();
final HashMap<String, List<PutRecordsRequestEntry>> recordHash = new HashMap<>();
final KinesisClient client = getClient(context);
try {
List<FlowFile> failedFlowFiles = new ArrayList<>();
List<FlowFile> successfulFlowFiles = new ArrayList<>();
final List<FlowFile> failedFlowFiles = new ArrayList<>();
final List<FlowFile> successfulFlowFiles = new ArrayList<>();
// Prepare batch of records
for (int i = 0; i < flowFiles.size(); i++) {
FlowFile flowFile = flowFiles.get(i);
String streamName = context.getProperty(KINESIS_STREAM_NAME).evaluateAttributeExpressions(flowFile).getValue();;
for (final FlowFile flowFile : flowFiles) {
final String streamName = context.getProperty(KINESIS_STREAM_NAME).evaluateAttributeExpressions(flowFile).getValue();
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
session.exportTo(flowFile, baos);
PutRecordsRequestEntry record = new PutRecordsRequestEntry().withData(ByteBuffer.wrap(baos.toByteArray()));
final PutRecordsRequestEntry.Builder recordBuilder = PutRecordsRequestEntry.builder().data(SdkBytes.fromByteArray(baos.toByteArray()));
String partitionKey = context.getProperty(PutKinesisStream.KINESIS_PARTITION_KEY)
.evaluateAttributeExpressions(flowFiles.get(i)).getValue();
final String partitionKey = context.getProperty(PutKinesisStream.KINESIS_PARTITION_KEY)
.evaluateAttributeExpressions(flowFile).getValue();
if (StringUtils.isBlank(partitionKey) == false) {
record.setPartitionKey(partitionKey);
} else {
record.setPartitionKey(Integer.toString(randomPartitionKeyGenerator.nextInt()));
}
recordBuilder.partitionKey(StringUtils.isBlank(partitionKey) ? Integer.toString(randomPartitionKeyGenerator.nextInt()) : partitionKey);
if (recordHash.containsKey(streamName) == false) {
recordHash.put(streamName, new ArrayList<>());
}
if (hashFlowFiles.containsKey(streamName) == false) {
hashFlowFiles.put(streamName, new ArrayList<>());
}
hashFlowFiles.get(streamName).add(flowFile);
recordHash.get(streamName).add(record);
hashFlowFiles.computeIfAbsent(streamName, key -> new ArrayList<>()).add(flowFile);
recordHash.computeIfAbsent(streamName, key -> new ArrayList<>()).add(recordBuilder.build());
}
for (Map.Entry<String, List<PutRecordsRequestEntry>> entryRecord : recordHash.entrySet()) {
String streamName = entryRecord.getKey();
List<PutRecordsRequestEntry> records = entryRecord.getValue();
for (final Map.Entry<String, List<PutRecordsRequestEntry>> entryRecord : recordHash.entrySet()) {
final String streamName = entryRecord.getKey();
final List<PutRecordsRequestEntry> records = entryRecord.getValue();
if (records.size() > 0) {
if (!records.isEmpty()) {
final PutRecordsRequest putRecordRequest = PutRecordsRequest.builder()
.streamName(streamName)
.records(records)
.build();
final PutRecordsResponse response = client.putRecords(putRecordRequest);
PutRecordsRequest putRecordRequest = new PutRecordsRequest();
putRecordRequest.setStreamName(streamName);
putRecordRequest.setRecords(records);
PutRecordsResult results = client.putRecords(putRecordRequest);
List<PutRecordsResultEntry> responseEntries = results.getRecords();
for (int i = 0; i < responseEntries.size(); i++ ) {
PutRecordsResultEntry entry = responseEntries.get(i);
final List<PutRecordsResultEntry> responseEntries = response.records();
for (int i = 0; i < responseEntries.size(); i++) {
final PutRecordsResultEntry entry = responseEntries.get(i);
FlowFile flowFile = hashFlowFiles.get(streamName).get(i);
Map<String,String> attributes = new HashMap<>();
attributes.put(AWS_KINESIS_SHARD_ID, entry.getShardId());
attributes.put(AWS_KINESIS_SEQUENCE_NUMBER, entry.getSequenceNumber());
attributes.put(AWS_KINESIS_SHARD_ID, entry.shardId());
attributes.put(AWS_KINESIS_SEQUENCE_NUMBER, entry.sequenceNumber());
if (StringUtils.isBlank(entry.getErrorCode()) == false) {
attributes.put(AWS_KINESIS_ERROR_CODE, entry.getErrorCode());
attributes.put(AWS_KINESIS_ERROR_MESSAGE, entry.getErrorMessage());
if (StringUtils.isNotBlank(entry.errorCode())) {
attributes.put(AWS_KINESIS_ERROR_CODE, entry.errorCode());
attributes.put(AWS_KINESIS_ERROR_MESSAGE, entry.errorMessage());
flowFile = session.putAllAttributes(flowFile, attributes);
failedFlowFiles.add(flowFile);
} else {
@ -204,17 +191,17 @@ public class PutKinesisStream extends AbstractKinesisStreamProcessor {
records.clear();
}
if ( failedFlowFiles.size() > 0 ) {
if (!failedFlowFiles.isEmpty()) {
session.transfer(failedFlowFiles, REL_FAILURE);
getLogger().error("Failed to publish to kinesis records {}", new Object[]{failedFlowFiles});
getLogger().error("Failed to publish to kinesis records {}", failedFlowFiles);
}
if ( successfulFlowFiles.size() > 0 ) {
if (!successfulFlowFiles.isEmpty()) {
session.transfer(successfulFlowFiles, REL_SUCCESS);
getLogger().debug("Successfully published to kinesis records {}", new Object[]{successfulFlowFiles});
getLogger().debug("Successfully published to kinesis records {}", successfulFlowFiles);
}
} catch (final Exception exception) {
getLogger().error("Failed to publish due to exception {} flowfiles {} ", new Object[]{exception, flowFiles});
getLogger().error("Failed to publish due to exception {} flowfiles {} ", exception, flowFiles);
session.transfer(flowFiles, REL_FAILURE);
context.yield();
}

View File

@ -16,16 +16,6 @@
*/
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;
@ -34,18 +24,30 @@ 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 software.amazon.awssdk.services.kinesis.model.Record;
import software.amazon.kinesis.exceptions.InvalidStateException;
import software.amazon.kinesis.exceptions.ShutdownException;
import software.amazon.kinesis.exceptions.ThrottlingException;
import software.amazon.kinesis.lifecycle.events.InitializationInput;
import software.amazon.kinesis.lifecycle.events.LeaseLostInput;
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
import software.amazon.kinesis.lifecycle.events.ShardEndedInput;
import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput;
import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
import software.amazon.kinesis.processor.ShardRecordProcessor;
import software.amazon.kinesis.retrieval.KinesisClientRecord;
import java.time.Instant;
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 abstract class AbstractKinesisRecordProcessor implements ShardRecordProcessor {
public static final String AWS_KINESIS_SHARD_ID = "aws.kinesis.shard.id";
public static final String AWS_KINESIS_SEQUENCE_NUMBER = "aws.kinesis.sequence.number";
@ -90,15 +92,15 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
@Override
public void initialize(final InitializationInput initializationInput) {
if (initializationInput.getPendingCheckpointSequenceNumber() != null) {
if (initializationInput.pendingCheckpointSequenceNumber() != null) {
log.warn("Initializing record processor for stream: {} / shard {}; from sequence number: {}; indicates previously uncheckpointed sequence number: {}",
streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber(), initializationInput.getPendingCheckpointSequenceNumber());
streamName, initializationInput.shardId(), initializationInput.extendedSequenceNumber(), initializationInput.pendingCheckpointSequenceNumber());
} else {
log.debug("Initializing record processor for stream: {} / shard: {}; from sequence number: {}",
streamName, initializationInput.getShardId(), initializationInput.getExtendedSequenceNumber());
streamName, initializationInput.shardId(), initializationInput.extendedSequenceNumber());
}
this.kinesisShardId = initializationInput.getShardId();
this.kinesisShardId = initializationInput.shardId();
// ensure we don't immediately checkpoint
this.nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
@ -108,15 +110,15 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
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());
processRecordsInput.records().size(), kinesisShardId,
processRecordsInput.cacheEntryTime() != null ? dateTimeFormatter.format(processRecordsInput.cacheEntryTime().atZone(ZoneId.systemDefault())) : null,
processRecordsInput.cacheExitTime() != null ? dateTimeFormatter.format(processRecordsInput.cacheExitTime().atZone(ZoneId.systemDefault())) : null,
processRecordsInput.millisBehindLatest());
}
ProcessSession session = null;
try {
final List<Record> records = processRecordsInput.getRecords();
final List<KinesisClientRecord> records = processRecordsInput.records();
if (!records.isEmpty()) {
final List<FlowFile> flowFiles = new ArrayList<>(records.size());
final StopWatch stopWatch = new StopWatch(true);
@ -130,7 +132,7 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
processingRecords = false;
// if creating an Kinesis checkpoint fails, then the same record(s) can be retrieved again
checkpointOnceEveryCheckpointInterval(processRecordsInput.getCheckpointer());
checkpointOnceEveryCheckpointInterval(processRecordsInput.checkpointer());
});
}
} catch (final Exception e) {
@ -146,11 +148,11 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
processingRecords = true;
}
private int processRecordsWithRetries(final List<Record> records, final List<FlowFile> flowFiles,
private int processRecordsWithRetries(final List<KinesisClientRecord> 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);
final KinesisClientRecord 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);
@ -166,7 +168,7 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
return recordsTransformed;
}
private boolean attemptProcessRecord(final List<FlowFile> flowFiles, final Record kinesisRecord, final boolean lastRecord,
private boolean attemptProcessRecord(final List<FlowFile> flowFiles, final KinesisClientRecord kinesisRecord, final boolean lastRecord,
final ProcessSession session, final StopWatch stopWatch) {
boolean processedSuccessfully = false;
try {
@ -197,7 +199,7 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
*
* @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,
abstract void processRecord(final List<FlowFile> flowFiles, final KinesisClientRecord kinesisRecord, final boolean lastRecord,
final ProcessSession session, final StopWatch stopWatch);
void reportProvenance(final ProcessSession session, final FlowFile flowFile, final String partitionKey,
@ -209,14 +211,14 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
session.getProvenanceReporter().receive(flowFile, transitUri, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
}
Map<String, String> getDefaultAttributes(final String sequenceNumber, final String partitionKey, final Date approximateArrivalTimestamp) {
Map<String, String> getDefaultAttributes(final String sequenceNumber, final String partitionKey, final Instant 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())));
dateTimeFormatter.format(approximateArrivalTimestamp.atZone(ZoneId.systemDefault())));
}
return attributes;
}
@ -230,7 +232,7 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
}
}
private void checkpointOnceEveryCheckpointInterval(final IRecordProcessorCheckpointer checkpointer) {
private void checkpointOnceEveryCheckpointInterval(final RecordProcessorCheckpointer checkpointer) {
if (System.currentTimeMillis() > nextCheckpointTimeInMillis) {
checkpointWithRetries(checkpointer);
nextCheckpointTimeInMillis = System.currentTimeMillis() + checkpointIntervalMillis;
@ -238,28 +240,37 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
}
@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 (InterruptedException ie) {
log.debug("Interrupted sleep while waiting for record processing to complete before shutdown (TERMINATE)", ie);
}
}
if (processingRecords) {
log.warn("Record Processor for shard {} still running, but maximum wait time elapsed, checkpoint will be attempted", kinesisShardId);
}
}
checkpointWithRetries(shutdownInput.getCheckpointer());
public void leaseLost(final LeaseLostInput leaseLostInput) {
log.debug("Lease lost");
}
private void checkpointWithRetries(final IRecordProcessorCheckpointer checkpointer) {
@Override
public void shardEnded(final ShardEndedInput shardEndedInput) {
log.debug("Shutting down Record Processor for shard: {} with reason: Shard Ended", kinesisShardId);
checkpointWithRetries(shardEndedInput.checkpointer());
}
@Override
public void shutdownRequested(final ShutdownRequestedInput shutdownRequestedInput) {
log.debug("Shutting down Record Processor for shard: {} with reason: Shutdown Requested", kinesisShardId);
// be sure to finish processing any records before shutdown
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 (InterruptedException ie) {
log.debug("Interrupted sleep while waiting for record processing to complete before shutdown (TERMINATE)", ie);
}
}
if (processingRecords) {
log.warn("Record Processor for shard {} still running, but maximum wait time elapsed, checkpoint will be attempted", kinesisShardId);
}
checkpointWithRetries(shutdownRequestedInput.checkpointer());
}
private void checkpointWithRetries(final RecordProcessorCheckpointer checkpointer) {
log.debug("Checkpointing shard " + kinesisShardId);
try {
for (int i = 0; i < numRetries; i++) {
@ -267,7 +278,7 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
break;
}
}
} catch (ShutdownException se) {
} catch (final ShutdownException se) {
// Ignore checkpoint if the processor instance has been shutdown (fail over).
log.info("Caught shutdown exception, skipping checkpoint.", se);
} catch (InvalidStateException e) {
@ -276,12 +287,12 @@ public abstract class AbstractKinesisRecordProcessor implements IRecordProcessor
}
}
private boolean attemptCheckpoint(final IRecordProcessorCheckpointer checkpointer, final int attempt) throws ShutdownException, InvalidStateException {
private boolean attemptCheckpoint(final RecordProcessorCheckpointer checkpointer, final int attempt) throws ShutdownException, InvalidStateException {
boolean success = false;
try {
checkpointer.checkpoint();
success = true;
} catch (ThrottlingException e) {
} catch (final ThrottlingException e) {
// Backoff and re-attempt checkpoint upon transient failures
if (attempt >= (numRetries - 1)) {
log.error("Checkpoint failed after {} attempts.", attempt + 1, e);

View File

@ -16,15 +16,16 @@
*/
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 software.amazon.kinesis.retrieval.KinesisClientRecord;
import java.nio.ByteBuffer;
import java.time.Instant;
import java.time.format.DateTimeFormatter;
import java.util.Date;
import java.util.List;
import java.util.Map;
@ -38,12 +39,16 @@ public class KinesisRecordProcessorRaw extends AbstractKinesisRecordProcessor {
}
@Override
void processRecord(final List<FlowFile> flowFiles, final Record kinesisRecord, final boolean lastRecord,
void processRecord(final List<FlowFile> flowFiles, final KinesisClientRecord kinesisRecord, final boolean lastRecord,
final ProcessSession session, final StopWatch stopWatch) {
final String partitionKey = kinesisRecord.getPartitionKey();
final String sequenceNumber = kinesisRecord.getSequenceNumber();
final Date approximateArrivalTimestamp = kinesisRecord.getApproximateArrivalTimestamp();
final byte[] data = kinesisRecord.getData() != null ? kinesisRecord.getData().array() : new byte[0];
final String partitionKey = kinesisRecord.partitionKey();
final String sequenceNumber = kinesisRecord.sequenceNumber();
final Instant approximateArrivalTimestamp = kinesisRecord.approximateArrivalTimestamp();
final ByteBuffer dataBuffer = kinesisRecord.data();
byte[] data = dataBuffer != null ? new byte[dataBuffer.remaining()] : new byte[0];
if (dataBuffer != null) {
dataBuffer.get(data);
}
FlowFile flowFile = session.create();
session.write(flowFile, out -> out.write(data));

View File

@ -16,7 +16,6 @@
*/
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.flowfile.attributes.CoreAttributes;
import org.apache.nifi.logging.ComponentLog;
@ -33,14 +32,16 @@ import org.apache.nifi.serialization.WriteResult;
import org.apache.nifi.serialization.record.PushBackRecordSet;
import org.apache.nifi.serialization.record.RecordSchema;
import org.apache.nifi.util.StopWatch;
import software.amazon.kinesis.retrieval.KinesisClientRecord;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.time.Instant;
import java.time.format.DateTimeFormatter;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.Map;
@ -73,11 +74,15 @@ public class KinesisRecordProcessorRecord extends AbstractKinesisRecordProcessor
}
@Override
void processRecord(final List<FlowFile> flowFiles, final Record kinesisRecord, final boolean lastRecord,
void processRecord(final List<FlowFile> flowFiles, final KinesisClientRecord kinesisRecord, final boolean lastRecord,
final ProcessSession session, final StopWatch stopWatch) {
boolean firstOutputRecord = true;
int recordCount = 0;
final byte[] data = kinesisRecord.getData() != null ? kinesisRecord.getData().array() : new byte[0];
final ByteBuffer dataBuffer = kinesisRecord.data();
byte[] data = dataBuffer != null ? new byte[dataBuffer.remaining()] : new byte[0];
if (dataBuffer != null) {
dataBuffer.get(data);
}
FlowFile flowFile = null;
try (final InputStream in = new ByteArrayInputStream(data);
@ -112,7 +117,7 @@ public class KinesisRecordProcessorRecord extends AbstractKinesisRecordProcessor
if (getLogger().isDebugEnabled()) {
getLogger().debug("Sequence No: {}, Partition Key: {}, Data: {}",
kinesisRecord.getSequenceNumber(), kinesisRecord.getPartitionKey(), BASE_64_ENCODER.encodeToString(data));
kinesisRecord.sequenceNumber(), kinesisRecord.partitionKey(), BASE_64_ENCODER.encodeToString(data));
}
}
@ -128,7 +133,7 @@ public class KinesisRecordProcessorRecord extends AbstractKinesisRecordProcessor
}
private void completeFlowFile(final List<FlowFile> flowFiles, final ProcessSession session, final int recordCount,
final WriteResult writeResult, final Record lastRecord, final StopWatch stopWatch)
final WriteResult writeResult, final KinesisClientRecord lastRecord, final StopWatch stopWatch)
throws IOException {
try {
@ -161,7 +166,7 @@ public class KinesisRecordProcessorRecord extends AbstractKinesisRecordProcessor
private void outputRawRecordOnException(final boolean firstOutputRecord, final FlowFile flowFile,
final List<FlowFile> flowFiles, final ProcessSession session,
final byte[] data, final Record kinesisRecord, final Exception e) {
final byte[] data, final KinesisClientRecord kinesisRecord, final Exception e) {
if (firstOutputRecord && flowFile != null) {
session.remove(flowFile);
flowFiles.remove(0);
@ -183,10 +188,10 @@ public class KinesisRecordProcessorRecord extends AbstractKinesisRecordProcessor
transferTo(ConsumeKinesisStream.REL_PARSE_FAILURE, session, 0, 0, Collections.singletonList(failed));
}
private Map<String, String> getDefaultAttributes(final Record kinesisRecord) {
final String partitionKey = kinesisRecord.getPartitionKey();
final String sequenceNumber = kinesisRecord.getSequenceNumber();
final Date approximateArrivalTimestamp = kinesisRecord.getApproximateArrivalTimestamp();
private Map<String, String> getDefaultAttributes(final KinesisClientRecord kinesisRecord) {
final String partitionKey = kinesisRecord.partitionKey();
final String sequenceNumber = kinesisRecord.sequenceNumber();
final Instant approximateArrivalTimestamp = kinesisRecord.approximateArrivalTimestamp();
return getDefaultAttributes(sequenceNumber, partitionKey, approximateArrivalTimestamp);
}
}

View File

@ -17,21 +17,25 @@
package org.apache.nifi.processors.aws.kinesis.stream;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
import com.amazonaws.services.dynamodbv2.model.ListTablesResult;
import com.amazonaws.services.kinesis.AmazonKinesis;
import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
import com.amazonaws.services.kinesis.model.DeleteStreamRequest;
import com.amazonaws.services.kinesis.model.ListStreamsResult;
import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import software.amazon.awssdk.core.SdkBytes;
import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
import software.amazon.awssdk.services.dynamodb.model.DeleteTableRequest;
import software.amazon.awssdk.services.dynamodb.model.ListTablesResponse;
import software.amazon.awssdk.services.kinesis.KinesisClient;
import software.amazon.awssdk.services.kinesis.model.DeleteStreamRequest;
import software.amazon.awssdk.services.kinesis.model.ListStreamsResponse;
import software.amazon.awssdk.services.kinesis.model.PutRecordRequest;
import software.amazon.kinesis.common.InitialPositionInStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import static com.amazonaws.SDKGlobalConfiguration.AWS_CBOR_DISABLE_SYSTEM_PROPERTY;
@ -44,14 +48,19 @@ public abstract class ITConsumeKinesisStream {
protected TestRunner runner;
AmazonKinesis kinesis;
AmazonDynamoDB dynamoDB;
KinesisClient kinesis;
DynamoDbClient dynamoDB;
@Test
public void readHorizon() throws InterruptedException, IOException {
String partitionKey = "1";
kinesis.putRecord(KINESIS_STREAM_NAME, ByteBuffer.wrap("horizon".getBytes()), partitionKey);
final PutRecordRequest request = PutRecordRequest.builder()
.streamName(KINESIS_STREAM_NAME)
.data(SdkBytes.fromString("horizon", Charset.defaultCharset()))
.partitionKey(partitionKey)
.build();
kinesis.putRecord(request);
startKCL(runner, InitialPositionInStream.TRIM_HORIZON);
@ -69,11 +78,21 @@ public abstract class ITConsumeKinesisStream {
public void readLatest() throws InterruptedException, IOException {
String partitionKey = "1";
kinesis.putRecord(KINESIS_STREAM_NAME, ByteBuffer.wrap("horizon".getBytes()), partitionKey);
final PutRecordRequest request = PutRecordRequest.builder()
.streamName(KINESIS_STREAM_NAME)
.data(SdkBytes.fromString("horizon", Charset.defaultCharset()))
.partitionKey(partitionKey)
.build();
kinesis.putRecord(request);
startKCL(runner, InitialPositionInStream.LATEST);
kinesis.putRecord(KINESIS_STREAM_NAME, ByteBuffer.wrap("latest".getBytes()), partitionKey);
final PutRecordRequest request2 = PutRecordRequest.builder()
.streamName(KINESIS_STREAM_NAME)
.data(SdkBytes.fromString("latest", Charset.defaultCharset()))
.partitionKey(partitionKey)
.build();
kinesis.putRecord(request2);
waitForKCLToProcessTheLatestMessage();
@ -102,7 +121,7 @@ public abstract class ITConsumeKinesisStream {
}
@AfterEach
public void tearDown() throws InterruptedException {
public void tearDown() throws InterruptedException, ExecutionException {
cleanupKinesis();
cleanupDynamoDB();
@ -115,10 +134,10 @@ public abstract class ITConsumeKinesisStream {
private void cleanupDynamoDB() {
if (dynamoDB != null) {
ListTablesResult tableResults = dynamoDB.listTables();
List<String> tableName = tableResults.getTableNames();
ListTablesResponse tableResults = dynamoDB.listTables();
List<String> tableName = tableResults.tableNames();
if (tableName.contains(APPLICATION_NAME)) {
dynamoDB.deleteTable(APPLICATION_NAME);
dynamoDB.deleteTable(DeleteTableRequest.builder().tableName(APPLICATION_NAME).build());
}
}
dynamoDB = null;
@ -126,10 +145,10 @@ public abstract class ITConsumeKinesisStream {
private void cleanupKinesis() {
if (kinesis != null) {
ListStreamsResult streamsResult = kinesis.listStreams();
List<String> streamNames = streamsResult.getStreamNames();
final ListStreamsResponse streamsResult = kinesis.listStreams();
List<String> streamNames = streamsResult.streamNames();
if (streamNames.contains(KINESIS_STREAM_NAME)) {
kinesis.deleteStream(new DeleteStreamRequest().withStreamName(KINESIS_STREAM_NAME));
kinesis.deleteStream(DeleteStreamRequest.builder().streamName(KINESIS_STREAM_NAME).build());
}
}
kinesis = null;

View File

@ -16,41 +16,53 @@
*/
package org.apache.nifi.processors.aws.kinesis.stream;
import com.amazonaws.auth.PropertiesFileCredentialsProvider;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
import com.amazonaws.services.kinesis.AmazonKinesisClient;
import org.apache.nifi.processors.aws.credentials.provider.PropertiesCredentialsProvider;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
import software.amazon.awssdk.http.apache.ApacheHttpClient;
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
import software.amazon.awssdk.services.kinesis.KinesisClient;
import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest;
import static com.amazonaws.SDKGlobalConfiguration.AWS_CBOR_DISABLE_SYSTEM_PROPERTY;
import java.io.File;
public class ITConsumeKinesisStreamConnectAWS extends ITConsumeKinesisStream {
private final static String CREDENTIALS_FILE =
System.getProperty("user.home") + "/aws-credentials.properties";
private final static File CREDENTIALS_FILE =
new File(System.getProperty("user.home") + "/aws-credentials.properties");
@BeforeEach
public void setUp() throws InterruptedException {
System.setProperty(AWS_CBOR_DISABLE_SYSTEM_PROPERTY, "true");
public void setUp() throws InterruptedException, InitializationException {
System.setProperty("aws.cborEnabled", "false");
kinesis = AmazonKinesisClient.builder()
.withCredentials(new PropertiesFileCredentialsProvider(CREDENTIALS_FILE))
.withRegion(REGION)
kinesis = KinesisClient.builder()
.credentialsProvider(new PropertiesCredentialsProvider(CREDENTIALS_FILE))
.httpClient(ApacheHttpClient.create())
.region(Region.of(REGION))
.build();
kinesis.createStream(KINESIS_STREAM_NAME, 1);
kinesis.createStream(CreateStreamRequest.builder().streamName(KINESIS_STREAM_NAME).shardCount(1).build());
dynamoDB = AmazonDynamoDBClient.builder()
.withCredentials(new PropertiesFileCredentialsProvider(CREDENTIALS_FILE))
.withRegion(REGION)
dynamoDB = DynamoDbClient.builder()
.credentialsProvider(new PropertiesCredentialsProvider(CREDENTIALS_FILE))
.region(Region.of(REGION))
.httpClient(ApacheHttpClient.create())
.build();
waitForKinesisToInitialize();
runner = TestRunners.newTestRunner(ConsumeKinesisStream.class);
final AWSCredentialsProviderControllerService credentialsService = new AWSCredentialsProviderControllerService();
runner.addControllerService("credentials-service", credentialsService);
runner.setProperty(credentialsService, CredentialPropertyDescriptors.CREDENTIALS_FILE, CREDENTIALS_FILE.getAbsolutePath());
runner.enableControllerService(credentialsService);
runner.setProperty(ConsumeKinesisStream.APPLICATION_NAME, APPLICATION_NAME);
runner.setProperty(ConsumeKinesisStream.KINESIS_STREAM_NAME, KINESIS_STREAM_NAME);
runner.setProperty(ConsumeKinesisStream.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(ConsumeKinesisStream.AWS_CREDENTIALS_PROVIDER_SERVICE, "credentials-service");
runner.setProperty(ConsumeKinesisStream.REGION, REGION);
runner.setProperty(ConsumeKinesisStream.REPORT_CLOUDWATCH_METRICS, "false");
runner.assertValid();

View File

@ -16,19 +16,28 @@
*/
package org.apache.nifi.processors.aws.kinesis.stream;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
import com.amazonaws.services.kinesis.AmazonKinesisClient;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
import software.amazon.awssdk.auth.credentials.AwsCredentials;
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
import software.amazon.awssdk.http.apache.ApacheHttpClient;
import software.amazon.awssdk.regions.Region;
import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
import software.amazon.awssdk.services.kinesis.KinesisClient;
import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest;
import static com.amazonaws.SDKGlobalConfiguration.AWS_CBOR_DISABLE_SYSTEM_PROPERTY;
import java.io.File;
import java.net.URI;
public class ITConsumeKinesisStreamEndpointOverride extends ITConsumeKinesisStream {
private final static File CREDENTIALS_FILE =
new File(System.getProperty("user.home") + "/aws-credentials.properties");
private static final String ACCESS_KEY = "test";
private static final String SECRET_KEY = "test";
private static final String KINESIS_STREAM_NAME = "test-stream";
@ -36,38 +45,50 @@ public class ITConsumeKinesisStreamEndpointOverride extends ITConsumeKinesisStre
private static final String LOCAL_STACK_KINESIS_ENDPOINT_OVERRIDE = "http://localhost:4566";
private static final String LOCAL_STACK_DYNAMODB_ENDPOINT_OVERRIDE = "http://localhost:4566";
private final AWSCredentialsProvider awsCredentialsProvider =
new AWSStaticCredentialsProvider(new BasicAWSCredentials(ACCESS_KEY, SECRET_KEY));
private final AwsCredentialsProvider awsCredentialsProvider =
StaticCredentialsProvider.create(new AwsCredentials() {
@Override
public String accessKeyId() {
return ACCESS_KEY;
}
private final AwsClientBuilder.EndpointConfiguration kinesisEndpointConfig =
new AwsClientBuilder.EndpointConfiguration(LOCAL_STACK_KINESIS_ENDPOINT_OVERRIDE, REGION);
private final AwsClientBuilder.EndpointConfiguration dynamoDBEndpointConfig =
new AwsClientBuilder.EndpointConfiguration(LOCAL_STACK_DYNAMODB_ENDPOINT_OVERRIDE, REGION);
@Override
public String secretAccessKey() {
return SECRET_KEY;
}
});
@BeforeEach
public void setUp() throws InterruptedException {
System.setProperty(AWS_CBOR_DISABLE_SYSTEM_PROPERTY, "true");
public void setUp() throws InterruptedException, InitializationException {
System.setProperty("aws.cborEnabled", "false");
kinesis = AmazonKinesisClient.builder()
.withEndpointConfiguration(kinesisEndpointConfig)
.withCredentials(awsCredentialsProvider)
kinesis = KinesisClient.builder()
.credentialsProvider(awsCredentialsProvider)
.endpointOverride(URI.create(LOCAL_STACK_KINESIS_ENDPOINT_OVERRIDE))
.httpClient(ApacheHttpClient.create())
.region(Region.of(REGION))
.build();
kinesis.createStream(KINESIS_STREAM_NAME, 1);
kinesis.createStream(CreateStreamRequest.builder().streamName(KINESIS_STREAM_NAME).shardCount(1).build());
dynamoDB = AmazonDynamoDBClient.builder()
.withEndpointConfiguration(dynamoDBEndpointConfig)
.withCredentials(awsCredentialsProvider)
dynamoDB = DynamoDbClient.builder()
.credentialsProvider(awsCredentialsProvider)
.endpointOverride(URI.create(LOCAL_STACK_DYNAMODB_ENDPOINT_OVERRIDE))
.region(Region.of(REGION))
.httpClient(ApacheHttpClient.create())
.build();
waitForKinesisToInitialize();
runner = TestRunners.newTestRunner(ConsumeKinesisStream.class);
final AWSCredentialsProviderControllerService credentialsService = new AWSCredentialsProviderControllerService();
runner.addControllerService("credentials-service", credentialsService);
runner.setProperty(credentialsService, CredentialPropertyDescriptors.CREDENTIALS_FILE, CREDENTIALS_FILE.getAbsolutePath());
runner.enableControllerService(credentialsService);
runner.setProperty(ConsumeKinesisStream.AWS_CREDENTIALS_PROVIDER_SERVICE, "credentials-service");
runner.setProperty(ConsumeKinesisStream.APPLICATION_NAME, APPLICATION_NAME);
runner.setProperty(ConsumeKinesisStream.KINESIS_STREAM_NAME, KINESIS_STREAM_NAME);
runner.setProperty(ConsumeKinesisStream.ACCESS_KEY, ACCESS_KEY);
runner.setProperty(ConsumeKinesisStream.SECRET_KEY, SECRET_KEY);
runner.setProperty(ConsumeKinesisStream.REGION, REGION);
runner.setProperty(ConsumeKinesisStream.REPORT_CLOUDWATCH_METRICS, "false");
runner.setProperty(ConsumeKinesisStream.ENDPOINT_OVERRIDE, LOCAL_STACK_KINESIS_ENDPOINT_OVERRIDE + "/kinesis");

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.processors.aws.kinesis.stream;
import org.apache.nifi.processors.aws.kinesis.KinesisProcessorUtils;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
@ -126,7 +127,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "2");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -144,7 +145,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisStream.KINESIS_PARTITION_KEY, "${partitionKey}");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -163,7 +164,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "5");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "2 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -185,7 +186,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "10");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -208,7 +209,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "10");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -230,7 +231,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "2");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -253,7 +254,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "10");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE * 2)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE * 2)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -281,7 +282,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "10");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE * 2)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE * 2)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -309,7 +310,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "10");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE * 2)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE * 2)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
@ -359,7 +360,7 @@ public class ITPutKinesisStream {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "5");
runner.setProperty(PutKinesisStream.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}

View File

@ -16,17 +16,12 @@
*/
package org.apache.nifi.processors.aws.kinesis.stream;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.regions.Regions;
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 org.apache.nifi.controller.ControllerService;
import org.apache.nifi.json.JsonRecordSetWriter;
import org.apache.nifi.json.JsonTreeReader;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
@ -38,6 +33,11 @@ import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import software.amazon.awssdk.regions.Region;
import software.amazon.kinesis.common.ConfigsBuilder;
import software.amazon.kinesis.common.InitialPositionInStream;
import software.amazon.kinesis.coordinator.Scheduler;
import software.amazon.kinesis.coordinator.WorkerStateChangeListener;
import java.net.InetAddress;
import java.net.UnknownHostException;
@ -46,7 +46,6 @@ import static org.hamcrest.CoreMatchers.anyOf;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.nullValue;
import static org.hamcrest.CoreMatchers.startsWith;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertThrows;
@ -257,18 +256,18 @@ public class TestConsumeKinesisStream {
runner.setProperty("no)allowed", "no-)");
// can't override static properties
runner.setProperty("regionName", Regions.AF_SOUTH_1.getName());
runner.setProperty("timestampAtInitialPositionInStream", "2021-01-01 00:00:00");
runner.setProperty("initialPositionInStream", "AT_TIMESTAMP");
runner.setProperty("dynamoDBEndpoint", "http://localhost:4566/dynamodb");
runner.setProperty("kinesisEndpoint", "http://localhost:4566/kinesis");
runner.setProperty("leaseManagementConfig.failoverTimeMillis", "1000");
runner.setProperty("leaseManagementConfig.initialPositionInStream", "AT_TIMESTAMP");
// invalid parameter conversions
runner.setProperty("dynamoDBClientConfig", "too-complex");
runner.setProperty("shutdownGraceMillis", "not-long");
runner.setProperty("checkpointConfig.checkpointFactory", "too-complex");
runner.setProperty("coordinatorConfig.schedulerInitializationBackoffTimeMillis", "not-long");
// valid dynamic parameters
runner.setProperty("namespace", "value");
final AssertionError ae = assertThrows(AssertionError.class, runner::assertValid);
assertThat(ae.getMessage(), startsWith("Processor has 17 validation failures:\n"));
assertThat(ae.getMessage(), startsWith("Processor has 13 validation failures:\n"));
// blank properties
assertThat(ae.getMessage(), containsString("'Property Name' validated against '' is invalid because Invalid attribute key: <Empty String>\n"));
@ -277,74 +276,60 @@ public class TestConsumeKinesisStream {
// invalid property names
assertThat(ae.getMessage(), containsString(
"'withPrefixNotAllowed' validated against 'a-value' is invalid because Property name must not have a prefix of \"with\", " +
"must start with a letter and contain only letters, numbers or underscores\n"
"must start with a letter and contain only letters, numbers, periods, or underscores\n"
));
assertThat(ae.getMessage(), containsString(
"'unknownProperty' validated against 'a-third-value' is invalid because Kinesis Client Library Configuration property with name " +
"'unknownProperty' validated against 'a-third-value' is invalid because Kinesis Client Configuration Builder property with name " +
"UnknownProperty does not exist or is not writable\n"
));
assertThat(ae.getMessage(), containsString(
"'toString' validated against 'cannot-call' is invalid because Kinesis Client Library Configuration property with name " +
"'toString' validated against 'cannot-call' is invalid because Kinesis Client Configuration Builder property with name " +
"ToString does not exist or is not writable\n"
));
// invalid property names (cannot use nested/indexed/mapped properties via BeanUtils)
assertThat(ae.getMessage(), containsString(
"'no.allowed' validated against 'no-.' is invalid because Property name must not have a prefix of \"with\", " +
"must start with a letter and contain only letters, numbers or underscores\n"
));
assertThat(ae.getMessage(), containsString(
"'no[allowed' validated against 'no-[' is invalid because Property name must not have a prefix of \"with\", " +
"must start with a letter and contain only letters, numbers or underscores\n"
"must start with a letter and contain only letters, numbers, periods, or underscores\n"
));
assertThat(ae.getMessage(), containsString(
"'no]allowed' validated against 'no-]' is invalid because Property name must not have a prefix of \"with\", " +
"must start with a letter and contain only letters, numbers or underscores\n"
"must start with a letter and contain only letters, numbers, periods, or underscores\n"
));
assertThat(ae.getMessage(), containsString(
"'no(allowed' validated against 'no-(' is invalid because Property name must not have a prefix of \"with\", " +
"must start with a letter and contain only letters, numbers or underscores\n"
"must start with a letter and contain only letters, numbers, periods, or underscores\n"
));
assertThat(ae.getMessage(), containsString(
"'no)allowed' validated against 'no-)' is invalid because Property name must not have a prefix of \"with\", " +
"must start with a letter and contain only letters, numbers or underscores\n"
"must start with a letter and contain only letters, numbers, periods, or underscores\n"
));
// can't override static properties
assertThat(ae.getMessage(), containsString("'regionName' validated against 'af-south-1' is invalid because Use \"Region\" instead of a dynamic property\n"));
assertThat(ae.getMessage(), containsString(
"'timestampAtInitialPositionInStream' validated against '2021-01-01 00:00:00' is invalid because Use \"Stream Position Timestamp\" instead of a dynamic property\n"
"'leaseManagementConfig.failoverTimeMillis' validated against '1000' is invalid because Use \"Failover Timeout\" instead of a dynamic property\n"
));
assertThat(ae.getMessage(), containsString(
"'initialPositionInStream' validated against 'AT_TIMESTAMP' is invalid because Use \"Initial Stream Position\" instead of a dynamic property\n"
));
assertThat(ae.getMessage(), containsString(
"'dynamoDBEndpoint' validated against 'http://localhost:4566/dynamodb' is invalid because Use \"DynamoDB Override\" instead of a dynamic property\n"
));
assertThat(ae.getMessage(), containsString(
"'kinesisEndpoint' validated against 'http://localhost:4566/kinesis' is invalid because Use \"Endpoint Override URL\" instead of a dynamic property\n"
"'leaseManagementConfig.initialPositionInStream' validated against 'AT_TIMESTAMP' is invalid because Use \"Initial Stream Position\" instead of a dynamic property\n"
));
// invalid parameter conversions
assertThat(ae.getMessage(), containsString(
"'dynamoDBClientConfig' validated against 'too-complex' is invalid because Kinesis Client Library Configuration property " +
"with name DynamoDBClientConfig cannot be used with value \"too-complex\" : " +
"Cannot invoke com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration.withDynamoDBClientConfig on bean class " +
"'class com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration' - argument type mismatch - " +
"had objects of type \"java.lang.String\" but expected signature \"com.amazonaws.ClientConfiguration\"\n"
"'checkpointConfig.checkpointFactory' validated against 'too-complex' is invalid because Kinesis Client Configuration Builder property " +
"with name CheckpointConfig.checkpointFactory cannot be used with value \"too-complex\" : " +
"Cannot invoke software.amazon.kinesis.checkpoint.CheckpointConfig.checkpointFactory on bean class " +
"'class software.amazon.kinesis.checkpoint.CheckpointConfig' - argument type mismatch - had objects of type \"java.lang.String\" " +
"but expected signature \"software.amazon.kinesis.checkpoint.CheckpointFactory\"\n"
));
assertThat(ae.getMessage(), containsString("'shutdownGraceMillis' validated against 'not-long' is invalid because " +
"Kinesis Client Library Configuration property with name ShutdownGraceMillis " +
"cannot be used with value \"not-long\" : Value of ShutdownGraceMillis should be positive, but current value is 0\n"));
}
@Test
public void testValidDynamicKCLProperties() {
runner.setProperty("billingMode", "PROVISIONED"); // enum
runner.setProperty("idleMillisBetweenCalls", "1000"); // long
runner.setProperty("cleanupLeasesUponShardCompletion", "true"); // boolean
runner.setProperty("initialLeaseTableReadCapacity", "1"); // int
runner.setProperty("DataFetchingStrategy", "DEFAULT"); // String with uppercase leading character in property name
runner.setProperty("leaseManagementConfig.billingMode", "PROVISIONED"); // enum
runner.setProperty("leaseManagementConfig.leasesRecoveryAuditorExecutionFrequencyMillis", "1000"); // long
runner.setProperty("leaseManagementConfig.cleanupLeasesUponShardCompletion", "true"); // boolean
runner.setProperty("leaseManagementConfig.initialLeaseTableReadCapacity", "1"); // int
runner.setProperty("leaseManagementConfig.MaxCacheMissesBeforeReload", "2"); // String with uppercase leading character in property name
runner.assertValid();
}
@ -374,7 +359,7 @@ public class TestConsumeKinesisStream {
mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.AWS_CREDENTIALS_PROVIDER_SERVICE, "aws-credentials");
// speed up init process for the unit test (and show use of dynamic properties to configure KCL)
mockConsumeKinesisStreamRunner.setProperty("parentShardPollIntervalMillis", "1");
mockConsumeKinesisStreamRunner.setProperty("coordinatorConfig.parentShardPollIntervalMillis", "1");
mockConsumeKinesisStreamRunner.assertValid();
@ -382,13 +367,16 @@ public class TestConsumeKinesisStream {
mockConsumeKinesisStreamRunner.run(1, false);
final MockConsumeKinesisStream processor = ((MockConsumeKinesisStream) mockConsumeKinesisStreamRunner.getProcessor());
// WorkerState should get to INITIALIZING pretty quickly, but there's a change it will still be at CREATED by the time we get here
Thread.sleep(50);
// WorkerState should get to INITIALIZING pretty quickly, but there's a chance it will still be at CREATED by the time we get here
assertThat(processor.workerState.get(), anyOf(equalTo(WorkerStateChangeListener.WorkerState.INITIALIZING), equalTo(WorkerStateChangeListener.WorkerState.CREATED)));
final String hostname = InetAddress.getLocalHost().getCanonicalHostName();
assertKinesisClientLibConfiguration(processor.kinesisClientLibConfiguration, withCredentials, hostname);
assertThat(processor.workerBuilder.build().getApplicationName(), equalTo("test-application"));
assertSchedulerConfigs(processor.scheduler, hostname);
assertConfigsBuilder(processor.configsBuilder);
assertThat(processor.scheduler.applicationName(), equalTo("test-application"));
if (!waitForFailure) {
// re-trigger the processor to ensure the Worker isn't re-initialised when already running
@ -412,57 +400,36 @@ public class TestConsumeKinesisStream {
}
}
private void assertKinesisClientLibConfiguration(final KinesisClientLibConfiguration kinesisClientLibConfiguration,
final boolean withCredentials, final String hostname) {
assertThat(kinesisClientLibConfiguration.getWorkerIdentifier(), startsWith(hostname));
assertThat(kinesisClientLibConfiguration.getApplicationName(), equalTo("test-application"));
assertThat(kinesisClientLibConfiguration.getStreamName(), equalTo("test-stream"));
private void assertConfigsBuilder(final ConfigsBuilder configsBuilder) {
assertThat(configsBuilder.kinesisClient().serviceClientConfiguration().region().id(), equalTo(Region.EU_WEST_2.id()));
assertTrue(configsBuilder.dynamoDBClient().serviceClientConfiguration().endpointOverride().isEmpty());
assertTrue(configsBuilder.kinesisClient().serviceClientConfiguration().endpointOverride().isEmpty());
}
if (withCredentials) {
assertThat(kinesisClientLibConfiguration.getKinesisCredentialsProvider().getCredentials().getAWSAccessKeyId(), equalTo("test-access"));
assertThat(kinesisClientLibConfiguration.getKinesisCredentialsProvider().getCredentials().getAWSSecretKey(), equalTo("test-secret"));
assertThat(kinesisClientLibConfiguration.getDynamoDBCredentialsProvider().getCredentials().getAWSAccessKeyId(), equalTo("test-access"));
assertThat(kinesisClientLibConfiguration.getDynamoDBCredentialsProvider().getCredentials().getAWSSecretKey(), equalTo("test-secret"));
assertThat(kinesisClientLibConfiguration.getCloudWatchCredentialsProvider().getCredentials().getAWSAccessKeyId(), equalTo("test-access"));
assertThat(kinesisClientLibConfiguration.getCloudWatchCredentialsProvider().getCredentials().getAWSSecretKey(), equalTo("test-secret"));
} else {
assertThat(kinesisClientLibConfiguration.getKinesisCredentialsProvider().getCredentials().getAWSAccessKeyId(), nullValue());
assertThat(kinesisClientLibConfiguration.getKinesisCredentialsProvider().getCredentials().getAWSSecretKey(), nullValue());
assertThat(kinesisClientLibConfiguration.getDynamoDBCredentialsProvider().getCredentials().getAWSAccessKeyId(), nullValue());
assertThat(kinesisClientLibConfiguration.getDynamoDBCredentialsProvider().getCredentials().getAWSSecretKey(), nullValue());
assertThat(kinesisClientLibConfiguration.getCloudWatchCredentialsProvider().getCredentials().getAWSAccessKeyId(), nullValue());
assertThat(kinesisClientLibConfiguration.getCloudWatchCredentialsProvider().getCredentials().getAWSSecretKey(), nullValue());
}
assertThat(kinesisClientLibConfiguration.getRegionName(), equalTo(Regions.EU_WEST_2.getName()));
assertThat(kinesisClientLibConfiguration.getInitialPositionInStream(), equalTo(InitialPositionInStream.LATEST));
assertThat(kinesisClientLibConfiguration.getDynamoDBEndpoint(), nullValue());
assertThat(kinesisClientLibConfiguration.getKinesisEndpoint(), nullValue());
assertThat(kinesisClientLibConfiguration.getKinesisClientConfiguration(), instanceOf(ClientConfiguration.class));
assertThat(kinesisClientLibConfiguration.getDynamoDBClientConfiguration(), instanceOf(ClientConfiguration.class));
assertThat(kinesisClientLibConfiguration.getCloudWatchClientConfiguration(), instanceOf(ClientConfiguration.class));
assertThat(kinesisClientLibConfiguration.getParentShardPollIntervalMillis(), equalTo(1L));
private void assertSchedulerConfigs(final Scheduler scheduler, final String hostname) {
assertThat(scheduler.leaseManagementConfig().workerIdentifier(), startsWith(hostname));
assertThat(scheduler.coordinatorConfig().applicationName(), equalTo("test-application"));
assertThat(scheduler.leaseManagementConfig().streamName(), equalTo("test-stream"));
assertThat(scheduler.leaseManagementConfig().initialPositionInStream().getInitialPositionInStream(), equalTo(InitialPositionInStream.LATEST));
assertThat(scheduler.coordinatorConfig().parentShardPollIntervalMillis(), equalTo(1L));
}
// public so TestRunners is able to see and instantiate the class for the tests
public static class MockConsumeKinesisStream extends ConsumeKinesisStream {
// capture the WorkerBuilder and KinesisClientLibConfiguration for unit test assertions
KinesisClientLibConfiguration kinesisClientLibConfiguration;
Worker.Builder workerBuilder;
// capture the Scheduler and ConfigsBuilder for unit test assertions
ConfigsBuilder configsBuilder;
Scheduler scheduler;
@Override
Worker.Builder prepareWorkerBuilder(final ProcessContext context, final KinesisClientLibConfiguration kinesisClientLibConfiguration,
final IRecordProcessorFactory factory) {
workerBuilder = super.prepareWorkerBuilder(context, kinesisClientLibConfiguration, factory);
return workerBuilder;
synchronized Scheduler prepareScheduler(final ProcessContext context, final ProcessSessionFactory sessionFactory, final String schedulerId) {
scheduler = super.prepareScheduler(context, sessionFactory, schedulerId);
return scheduler;
}
@Override
KinesisClientLibConfiguration prepareKinesisClientLibConfiguration(final ProcessContext context, final String workerId) {
kinesisClientLibConfiguration = super.prepareKinesisClientLibConfiguration(context, workerId);
return kinesisClientLibConfiguration;
ConfigsBuilder prepareConfigsBuilder(final ProcessContext context, final String workerId, final ProcessSessionFactory sessionFactory) {
configsBuilder = super.prepareConfigsBuilder(context, workerId, sessionFactory);
return configsBuilder;
}
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.processors.aws.kinesis.stream;
import org.apache.nifi.processors.aws.kinesis.KinesisProcessorUtils;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
@ -66,7 +67,7 @@ public class TestPutKinesisStream {
public void testWithSizeGreaterThan1MB() {
runner.setProperty(PutKinesisStream.BATCH_SIZE, "1");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisStream.MAX_MESSAGE_SIZE + 1)];
byte [] bytes = new byte[(KinesisProcessorUtils.MAX_MESSAGE_SIZE + 1)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}

View File

@ -16,19 +16,6 @@
*/
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.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 java.time.format.DateTimeFormatter;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessSessionFactory;
@ -41,6 +28,19 @@ import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import software.amazon.kinesis.common.InitialPositionInStream;
import software.amazon.kinesis.exceptions.InvalidStateException;
import software.amazon.kinesis.exceptions.ShutdownException;
import software.amazon.kinesis.exceptions.ThrottlingException;
import software.amazon.kinesis.lifecycle.events.InitializationInput;
import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput;
import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
import software.amazon.kinesis.retrieval.KinesisClientRecord;
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
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;
@ -64,9 +64,9 @@ public class TestAbstractKinesisRecordProcessor {
private final MockProcessSession session = new MockProcessSession(new SharedSessionState(runner.getProcessor(), new AtomicLong(0)), runner.getProcessor());
private AbstractKinesisRecordProcessor fixture;
private final IRecordProcessorCheckpointer checkpointer = mock(IRecordProcessorCheckpointer.class);
private final RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class);
private final Record kinesisRecord = mock(Record.class);
private final KinesisClientRecord kinesisRecord = mock(KinesisClientRecord.class);
@BeforeEach
public void setUp() {
@ -76,7 +76,7 @@ public class TestAbstractKinesisRecordProcessor {
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) {
void processRecord(List<FlowFile> flowFiles, KinesisClientRecord kinesisRecord, boolean lastRecord, ProcessSession session, StopWatch stopWatch) {
// intentionally blank
}
};
@ -91,9 +91,10 @@ public class TestAbstractKinesisRecordProcessor {
@Test
public void testInitialisation() {
final ExtendedSequenceNumber esn = new ExtendedSequenceNumber(InitialPositionInStream.AT_TIMESTAMP.toString(), 123L);
final InitializationInput initializationInput = new InitializationInput()
.withExtendedSequenceNumber(esn)
.withShardId("shard-id");
final InitializationInput initializationInput = InitializationInput.builder()
.extendedSequenceNumber(esn)
.shardId("shard-id")
.build();
fixture.initialize(initializationInput);
@ -105,10 +106,11 @@ public class TestAbstractKinesisRecordProcessor {
public void testInitialisationWithPendingCheckpoint() {
final ExtendedSequenceNumber esn = new ExtendedSequenceNumber(InitialPositionInStream.AT_TIMESTAMP.toString(), 123L);
final ExtendedSequenceNumber prev = new ExtendedSequenceNumber(InitialPositionInStream.LATEST.toString(), 456L);
final InitializationInput initializationInput = new InitializationInput()
.withExtendedSequenceNumber(esn)
.withPendingCheckpointSequenceNumber(prev)
.withShardId("shard-id");
final InitializationInput initializationInput = InitializationInput.builder()
.extendedSequenceNumber(esn)
.pendingCheckpointSequenceNumber(prev)
.shardId("shard-id")
.build();
fixture.initialize(initializationInput);
@ -118,51 +120,51 @@ public class TestAbstractKinesisRecordProcessor {
@Test
public void testShutdown() throws InvalidStateException, ShutdownException {
final ShutdownInput shutdownInput = new ShutdownInput()
.withShutdownReason(ShutdownReason.REQUESTED)
.withCheckpointer(checkpointer);
final ShutdownRequestedInput shutdownInput = ShutdownRequestedInput.builder()
.checkpointer(checkpointer)
.build();
fixture.setKinesisShardId("test-shard");
fixture.shutdown(shutdownInput);
fixture.shutdownRequested(shutdownInput);
verify(checkpointer, times(1)).checkpoint();
}
@Test
public void testShutdownWithThrottlingFailures() throws InvalidStateException, ShutdownException {
final ShutdownInput shutdownInput = new ShutdownInput()
.withShutdownReason(ShutdownReason.REQUESTED)
.withCheckpointer(checkpointer);
final ShutdownRequestedInput shutdownInput = ShutdownRequestedInput.builder()
.checkpointer(checkpointer)
.build();
doThrow(new ThrottlingException("throttled")).when(checkpointer).checkpoint();
fixture.shutdown(shutdownInput);
fixture.shutdownRequested(shutdownInput);
verify(checkpointer, times(2)).checkpoint();
}
@Test
public void testShutdownWithShutdownFailure() throws InvalidStateException, ShutdownException {
final ShutdownInput shutdownInput = new ShutdownInput()
.withShutdownReason(ShutdownReason.REQUESTED)
.withCheckpointer(checkpointer);
final ShutdownRequestedInput shutdownInput = ShutdownRequestedInput.builder()
.checkpointer(checkpointer)
.build();
doThrow(new ShutdownException("shutdown")).when(checkpointer).checkpoint();
fixture.shutdown(shutdownInput);
fixture.shutdownRequested(shutdownInput);
verify(checkpointer, times(1)).checkpoint();
}
@Test
public void testShutdownWithInvalidStateFailure() throws InvalidStateException, ShutdownException {
final ShutdownInput shutdownInput = new ShutdownInput()
.withShutdownReason(ShutdownReason.ZOMBIE)
.withCheckpointer(checkpointer);
final ShutdownRequestedInput shutdownInput = ShutdownRequestedInput.builder()
.checkpointer(checkpointer)
.build();
doThrow(new InvalidStateException("invalid state")).when(checkpointer).checkpoint();
fixture.shutdown(shutdownInput);
fixture.shutdownRequested(shutdownInput);
verify(checkpointer, times(1)).checkpoint();
@ -171,13 +173,13 @@ public class TestAbstractKinesisRecordProcessor {
@Test
public void testShutdownTerminateRecordsNotProcessing() throws InvalidStateException, ShutdownException {
final ShutdownInput shutdownInput = new ShutdownInput()
.withShutdownReason(ShutdownReason.TERMINATE)
.withCheckpointer(checkpointer);
final ShutdownRequestedInput shutdownInput = ShutdownRequestedInput.builder()
.checkpointer(checkpointer)
.build();
fixture.setKinesisShardId("test-shard");
fixture.setProcessingRecords(false);
fixture.shutdown(shutdownInput);
fixture.shutdownRequested(shutdownInput);
verify(checkpointer, times(1)).checkpoint();
@ -186,13 +188,13 @@ public class TestAbstractKinesisRecordProcessor {
@Test
public void testShutdownTerminateRecordsProcessing() throws InvalidStateException, ShutdownException {
final ShutdownInput shutdownInput = new ShutdownInput()
.withShutdownReason(ShutdownReason.TERMINATE)
.withCheckpointer(checkpointer);
final ShutdownRequestedInput shutdownInput = ShutdownRequestedInput.builder()
.checkpointer(checkpointer)
.build();
fixture.setKinesisShardId("test-shard");
fixture.setProcessingRecords(true);
fixture.shutdown(shutdownInput);
fixture.shutdownRequested(shutdownInput);
verify(checkpointer, times(1)).checkpoint();

View File

@ -16,11 +16,23 @@
*/
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.interfaces.IRecordProcessorCheckpointer;
import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
import com.amazonaws.services.kinesis.model.Record;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.MockProcessSession;
import org.apache.nifi.util.SharedSessionState;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
import software.amazon.kinesis.exceptions.InvalidStateException;
import software.amazon.kinesis.exceptions.ShutdownException;
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
import software.amazon.kinesis.retrieval.KinesisClientRecord;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.time.Instant;
@ -33,16 +45,6 @@ import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processors.aws.kinesis.stream.ConsumeKinesisStream;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.MockProcessSession;
import org.apache.nifi.util.SharedSessionState;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
@ -67,8 +69,11 @@ public class TestKinesisRecordProcessorRaw {
private AbstractKinesisRecordProcessor fixture;
private final IRecordProcessorCheckpointer checkpointer = mock(IRecordProcessorCheckpointer.class);
private final Record kinesisRecord = mock(Record.class);
@Mock
private final RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class);
@Mock
private final KinesisClientRecord kinesisRecord = mock(KinesisClientRecord.class);
@BeforeEach
public void setUp() {
@ -85,12 +90,13 @@ public class TestKinesisRecordProcessorRaw {
@Test
public void testProcessRecordsEmpty() {
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput()
.withRecords(Collections.emptyList())
.withCheckpointer(checkpointer)
.withCacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.withCacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.withMillisBehindLatest(100L);
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
.records(Collections.emptyList())
.checkpointer(checkpointer)
.cacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.cacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.millisBehindLatest(100L)
.build();
// would checkpoint (but should skip because there are no records processed)
fixture.setNextCheckpointTimeInMillis(System.currentTimeMillis() - 10_000L);
@ -117,25 +123,29 @@ public class TestKinesisRecordProcessorRaw {
final Date firstDate = Date.from(Instant.now().minus(1, ChronoUnit.MINUTES));
final Date secondDate = new Date();
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput()
.withRecords(Arrays.asList(
new Record().withApproximateArrivalTimestamp(firstDate)
.withPartitionKey("partition-1")
.withSequenceNumber("1")
.withData(ByteBuffer.wrap("record-1".getBytes(StandardCharsets.UTF_8))),
new Record().withApproximateArrivalTimestamp(secondDate)
.withPartitionKey("partition-2")
.withSequenceNumber("2")
.withData(ByteBuffer.wrap("record-2".getBytes(StandardCharsets.UTF_8))),
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-no-date")
.withSequenceNumber("no-date")
.withData(ByteBuffer.wrap("record-no-date".getBytes(StandardCharsets.UTF_8)))
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
.records(Arrays.asList(
KinesisClientRecord.builder().approximateArrivalTimestamp(firstDate.toInstant())
.partitionKey("partition-1")
.sequenceNumber("1")
.data(ByteBuffer.wrap("record-1".getBytes(StandardCharsets.UTF_8)))
.build(),
KinesisClientRecord.builder().approximateArrivalTimestamp(secondDate.toInstant())
.partitionKey("partition-2")
.sequenceNumber("2")
.data(ByteBuffer.wrap("record-2".getBytes(StandardCharsets.UTF_8)))
.build(),
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-no-date")
.sequenceNumber("no-date")
.data(ByteBuffer.wrap("record-no-date".getBytes(StandardCharsets.UTF_8)))
.build()
))
.withCheckpointer(checkpointer)
.withCacheEntryTime(null)
.withCacheExitTime(null)
.withMillisBehindLatest(null);
.checkpointer(checkpointer)
.cacheEntryTime(null)
.cacheExitTime(null)
.millisBehindLatest(null)
.build();
final String transitUriPrefix = endpointOverridden ? "https://another-endpoint.com:8443" : "http://endpoint-prefix.amazonaws.com";
if (endpointOverridden) {
@ -151,8 +161,8 @@ public class TestKinesisRecordProcessorRaw {
fixture.processRecords(processRecordsInput);
verify(processSessionFactory, times(1)).createSession();
session.assertTransferCount(ConsumeKinesisStream.REL_SUCCESS, processRecordsInput.getRecords().size());
assertThat(sharedState.getProvenanceEvents().size(), is(processRecordsInput.getRecords().size()));
session.assertTransferCount(ConsumeKinesisStream.REL_SUCCESS, processRecordsInput.records().size());
assertThat(sharedState.getProvenanceEvents().size(), is(processRecordsInput.records().size()));
assertThat(sharedState.getProvenanceEvents().get(0).getTransitUri(), is(String.format("%s/test-shard/partition-1#1", transitUriPrefix)));
assertThat(sharedState.getProvenanceEvents().get(1).getTransitUri(), is(String.format("%s/test-shard/partition-2#2", transitUriPrefix)));
assertThat(sharedState.getProvenanceEvents().get(2).getTransitUri(), is(String.format("%s/test-shard/partition-no-date#no-date", transitUriPrefix)));
@ -168,24 +178,27 @@ public class TestKinesisRecordProcessorRaw {
@Test
public void testProcessPoisonPillRecordWithCheckpoint() throws ShutdownException, InvalidStateException {
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput()
.withRecords(Arrays.asList(
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-1")
.withSequenceNumber("1")
.withData(ByteBuffer.wrap("record-1".getBytes(StandardCharsets.UTF_8))),
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
.records(Arrays.asList(
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-1")
.sequenceNumber("1")
.data(ByteBuffer.wrap("record-1".getBytes(StandardCharsets.UTF_8)))
.build(),
kinesisRecord,
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-3")
.withSequenceNumber("3")
.withData(ByteBuffer.wrap("record-3".getBytes(StandardCharsets.UTF_8)))
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-3")
.sequenceNumber("3")
.data(ByteBuffer.wrap("record-3".getBytes(StandardCharsets.UTF_8)))
.build()
))
.withCheckpointer(checkpointer)
.withCacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.withCacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.withMillisBehindLatest(100L);
.checkpointer(checkpointer)
.cacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.cacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.millisBehindLatest(100L)
.build();
when(kinesisRecord.getData()).thenThrow(new IllegalStateException("illegal state"));
when(kinesisRecord.data()).thenThrow(new IllegalStateException("illegal state"));
when(kinesisRecord.toString()).thenReturn("poison-pill");
fixture.setKinesisShardId("test-shard");
@ -201,10 +214,10 @@ public class TestKinesisRecordProcessorRaw {
assertFlowFile(flowFiles.get(1), null, "partition-3", "3", "record-3");
// check the "poison pill" record was retried a 2nd time
assertNull(verify(kinesisRecord, times(2)).getPartitionKey());
assertNull(verify(kinesisRecord, times(2)).getSequenceNumber());
assertNull(verify(kinesisRecord, times(2)).getApproximateArrivalTimestamp());
assertNull(verify(kinesisRecord, times(2)).getData());
assertNull(verify(kinesisRecord, times(2)).partitionKey());
assertNull(verify(kinesisRecord, times(2)).sequenceNumber());
assertNull(verify(kinesisRecord, times(2)).approximateArrivalTimestamp());
assertNull(verify(kinesisRecord, times(2)).data());
verify(checkpointer, times(1)).checkpoint();
assertFalse(runner.getLogger().getErrorMessages().isEmpty());

View File

@ -16,23 +16,6 @@
*/
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.interfaces.IRecordProcessorCheckpointer;
import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
import com.amazonaws.services.kinesis.model.Record;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.time.Instant;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.time.temporal.ChronoUnit;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.json.JsonRecordSetWriter;
import org.apache.nifi.json.JsonTreeReader;
@ -51,6 +34,25 @@ import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mock;
import software.amazon.kinesis.exceptions.InvalidStateException;
import software.amazon.kinesis.exceptions.ShutdownException;
import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput;
import software.amazon.kinesis.processor.RecordProcessorCheckpointer;
import software.amazon.kinesis.retrieval.KinesisClientRecord;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.time.Instant;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.time.format.DateTimeFormatter;
import java.time.temporal.ChronoUnit;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
@ -78,8 +80,11 @@ public class TestKinesisRecordProcessorRecord {
private final RecordReaderFactory reader = new JsonTreeReader();
private final RecordSetWriterFactory writer = new JsonRecordSetWriter();
private final IRecordProcessorCheckpointer checkpointer = mock(IRecordProcessorCheckpointer.class);
private final Record kinesisRecord = mock(Record.class);
@Mock
private final RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class);
@Mock
private final KinesisClientRecord kinesisRecord = mock(KinesisClientRecord.class);
@BeforeEach
public void setUp() throws InitializationException {
@ -108,12 +113,13 @@ public class TestKinesisRecordProcessorRecord {
@Test
public void testProcessRecordsEmpty() {
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput()
.withRecords(Collections.emptyList())
.withCheckpointer(checkpointer)
.withCacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.withCacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.withMillisBehindLatest(100L);
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
.records(Collections.emptyList())
.checkpointer(checkpointer)
.cacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.cacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.millisBehindLatest(100L)
.build();
// would checkpoint (but should skip because there are no records processed)
fixture.setNextCheckpointTimeInMillis(System.currentTimeMillis() - 10_000L);
@ -140,25 +146,29 @@ public class TestKinesisRecordProcessorRecord {
final Date firstDate = Date.from(Instant.now().minus(1, ChronoUnit.MINUTES));
final Date secondDate = new Date();
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput()
.withRecords(Arrays.asList(
new Record().withApproximateArrivalTimestamp(firstDate)
.withPartitionKey("partition-1")
.withSequenceNumber("1")
.withData(ByteBuffer.wrap("{\"record\":\"1\"}\n{\"record\":\"1b\"}".getBytes(StandardCharsets.UTF_8))),
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-no-date")
.withSequenceNumber("no-date")
.withData(ByteBuffer.wrap("{\"record\":\"no-date\"}".getBytes(StandardCharsets.UTF_8))),
new Record().withApproximateArrivalTimestamp(secondDate)
.withPartitionKey("partition-2")
.withSequenceNumber("2")
.withData(ByteBuffer.wrap("{\"record\":\"2\"}".getBytes(StandardCharsets.UTF_8)))
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
.records(Arrays.asList(
KinesisClientRecord.builder().approximateArrivalTimestamp(firstDate.toInstant())
.partitionKey("partition-1")
.sequenceNumber("1")
.data(ByteBuffer.wrap("{\"record\":\"1\"}\n{\"record\":\"1b\"}".getBytes(StandardCharsets.UTF_8)))
.build(),
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-no-date")
.sequenceNumber("no-date")
.data(ByteBuffer.wrap("{\"record\":\"no-date\"}".getBytes(StandardCharsets.UTF_8)))
.build(),
KinesisClientRecord.builder().approximateArrivalTimestamp(secondDate.toInstant())
.partitionKey("partition-2")
.sequenceNumber("2")
.data(ByteBuffer.wrap("{\"record\":\"2\"}".getBytes(StandardCharsets.UTF_8)))
.build()
))
.withCheckpointer(checkpointer)
.withCacheEntryTime(null)
.withCacheExitTime(null)
.withMillisBehindLatest(null);
.checkpointer(checkpointer)
.cacheEntryTime(null)
.cacheExitTime(null)
.millisBehindLatest(null)
.build();
final String transitUriPrefix = endpointOverridden ? "https://another-endpoint.com:8443" : "http://endpoint-prefix.amazonaws.com";
if (endpointOverridden) {
@ -193,24 +203,27 @@ public class TestKinesisRecordProcessorRecord {
@Test
public void testProcessPoisonPillRecordButNoRawOutputWithCheckpoint() throws ShutdownException, InvalidStateException {
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput()
.withRecords(Arrays.asList(
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-1")
.withSequenceNumber("1")
.withData(ByteBuffer.wrap("{\"record\":\"1\"}".getBytes(StandardCharsets.UTF_8))),
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
.records(Arrays.asList(
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-1")
.sequenceNumber("1")
.data(ByteBuffer.wrap("{\"record\":\"1\"}".getBytes(StandardCharsets.UTF_8)))
.build(),
kinesisRecord,
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-3")
.withSequenceNumber("3")
.withData(ByteBuffer.wrap("{\"record\":\"3\"}".getBytes(StandardCharsets.UTF_8)))
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-3")
.sequenceNumber("3")
.data(ByteBuffer.wrap("{\"record\":\"3\"}".getBytes(StandardCharsets.UTF_8)))
.build()
))
.withCheckpointer(checkpointer)
.withCacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.withCacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.withMillisBehindLatest(100L);
.checkpointer(checkpointer)
.cacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.cacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.millisBehindLatest(100L)
.build();
when(kinesisRecord.getData()).thenThrow(new IllegalStateException("illegal state"));
when(kinesisRecord.data()).thenThrow(new IllegalStateException("illegal state"));
when(kinesisRecord.toString()).thenReturn("poison-pill");
fixture.setKinesisShardId("test-shard");
@ -230,7 +243,7 @@ public class TestKinesisRecordProcessorRecord {
session.assertTransferCount(ConsumeKinesisStream.REL_PARSE_FAILURE, 0);
// check the "poison pill" record was retried a 2nd time
assertNull(verify(kinesisRecord, times(2)).getData());
assertNull(verify(kinesisRecord, times(2)).data());
verify(checkpointer, times(1)).checkpoint();
assertFalse(runner.getLogger().getErrorMessages().isEmpty());
@ -241,27 +254,30 @@ public class TestKinesisRecordProcessorRecord {
@Test
public void testProcessUnparsableRecordWithRawOutputWithCheckpoint() throws ShutdownException, InvalidStateException {
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput()
.withRecords(Arrays.asList(
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-1")
.withSequenceNumber("1")
.withData(ByteBuffer.wrap("{\"record\":\"1\"}".getBytes(StandardCharsets.UTF_8))),
final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder()
.records(Arrays.asList(
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-1")
.sequenceNumber("1")
.data(ByteBuffer.wrap("{\"record\":\"1\"}".getBytes(StandardCharsets.UTF_8)))
.build(),
kinesisRecord,
new Record().withApproximateArrivalTimestamp(null)
.withPartitionKey("partition-3")
.withSequenceNumber("3")
.withData(ByteBuffer.wrap("{\"record\":\"3\"}".getBytes(StandardCharsets.UTF_8)))
KinesisClientRecord.builder().approximateArrivalTimestamp(null)
.partitionKey("partition-3")
.sequenceNumber("3")
.data(ByteBuffer.wrap("{\"record\":\"3\"}".getBytes(StandardCharsets.UTF_8)))
.build()
))
.withCheckpointer(checkpointer)
.withCacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.withCacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.withMillisBehindLatest(100L);
.checkpointer(checkpointer)
.cacheEntryTime(Instant.now().minus(1, ChronoUnit.MINUTES))
.cacheExitTime(Instant.now().minus(1, ChronoUnit.SECONDS))
.millisBehindLatest(100L)
.build();
when(kinesisRecord.getData()).thenReturn(ByteBuffer.wrap("invalid-json".getBytes(StandardCharsets.UTF_8)));
when(kinesisRecord.getPartitionKey()).thenReturn("unparsable-partition");
when(kinesisRecord.getSequenceNumber()).thenReturn("unparsable-sequence");
when(kinesisRecord.getApproximateArrivalTimestamp()).thenReturn(null);
when(kinesisRecord.data()).thenReturn(ByteBuffer.wrap("invalid-json".getBytes(StandardCharsets.UTF_8)));
when(kinesisRecord.partitionKey()).thenReturn("unparsable-partition");
when(kinesisRecord.sequenceNumber()).thenReturn("unparsable-sequence");
when(kinesisRecord.approximateArrivalTimestamp()).thenReturn(null);
fixture.setKinesisShardId("test-shard");
@ -283,10 +299,10 @@ public class TestKinesisRecordProcessorRecord {
failureFlowFiles.get(0).assertAttributeExists("record.error.message");
// check the invalid json record was *not* retried a 2nd time
assertNull(verify(kinesisRecord, times(1)).getPartitionKey());
assertNull(verify(kinesisRecord, times(1)).getSequenceNumber());
assertNull(verify(kinesisRecord, times(1)).getApproximateArrivalTimestamp());
assertNull(verify(kinesisRecord, times(2)).getData());
assertNull(verify(kinesisRecord, times(1)).partitionKey());
assertNull(verify(kinesisRecord, times(1)).sequenceNumber());
assertNull(verify(kinesisRecord, times(1)).approximateArrivalTimestamp());
assertNull(verify(kinesisRecord, times(1)).data());
verify(checkpointer, times(1)).checkpoint();
assertEquals(1, runner.getLogger().getErrorMessages().size());

View File

@ -26,8 +26,7 @@
<packaging>pom</packaging>
<properties>
<!-- keep KCL 1.x until NIFI-8531 (blocked by NIFI-8287) -->
<aws-kinesis-client-library-version>1.15.0</aws-kinesis-client-library-version>
<aws-kinesis-client-library-version>2.5.2</aws-kinesis-client-library-version>
</properties>
<modules>

View File

@ -187,7 +187,7 @@
<logger name="com.hierynomus.smbj" level="WARN" />
<!-- Suppress non-error messages from AWS KCL which was emitting large amounts of INFO logs by default -->
<logger name="com.amazonaws.services.kinesis" level="WARN" />
<logger name="software.amazon.awssdk.kinesis" level="WARN" />
<!-- Suppress non-error messages from Apache Atlas which was emitting large amounts of INFO logs by default -->
<logger name="org.apache.atlas" level="WARN"/>

View File

@ -63,7 +63,7 @@
<logger name="com.hierynomus.smbj" level="WARN" />
<!-- Suppress non-error messages from AWS KCL which was emitting large amounts of INFO logs by default -->
<logger name="com.amazonaws.services.kinesis" level="WARN" />
<logger name="software.amazon.awssdk.kinesis" level="WARN" />
<logger name="org.apache.nifi.stateless" level="INFO" additivity="false">
<appender-ref ref="CONSOLE" />

View File

@ -63,7 +63,7 @@
<logger name="com.hierynomus.smbj" level="WARN" />
<!-- Suppress non-error messages from AWS KCL which was emitting large amounts of INFO logs by default -->
<logger name="com.amazonaws.services.kinesis" level="WARN" />
<logger name="software.amazon.awssdk.kinesis" level="WARN" />
<root level="INFO">
<appender-ref ref="CONSOLE" />

View File

@ -143,7 +143,7 @@
<logger name="com.hierynomus.smbj" level="WARN" />
<!-- Suppress non-error messages from AWS KCL which was emitting large amounts of INFO logs by default -->
<logger name="com.amazonaws.services.kinesis" level="WARN" />
<logger name="software.amazon.awssdk.kinesis" level="WARN" />
<!-- Suppress non-error messages from Apache Atlas which was emitting large amounts of INFO logs by default -->
<logger name="org.apache.atlas" level="WARN"/>

View File

@ -143,7 +143,7 @@
<logger name="com.hierynomus.smbj" level="WARN" />
<!-- Suppress non-error messages from AWS KCL which was emitting large amounts of INFO logs by default -->
<logger name="com.amazonaws.services.kinesis" level="WARN" />
<logger name="software.amazon.awssdk.kinesis" level="WARN" />
<!-- Suppress non-error messages from Apache Atlas which was emitting large amounts of INFO logs by default -->
<logger name="org.apache.atlas" level="WARN"/>

View File

@ -144,7 +144,7 @@
<logger name="com.hierynomus.smbj" level="WARN" />
<!-- Suppress non-error messages from AWS KCL which was emitting large amounts of INFO logs by default -->
<logger name="com.amazonaws.services.kinesis" level="WARN" />
<logger name="software.amazon.awssdk.kinesis" level="WARN" />
<!-- Suppress non-error messages from Apache Atlas which was emitting large amounts of INFO logs by default -->
<logger name="org.apache.atlas" level="WARN"/>