NIFI-12144: Eliminated deprecation warnings by moving to newer Amazon SDK APIs. Eliminated several abstract classes that provided no value. Code cleanup. Refactored many of the integration tests to make use of Test Containers. Removed some tests that were incorrect

Signed-off-by: Joe Gresock <jgresock@gmail.com>
This closes #7854.
This commit is contained in:
Mark Payne 2023-10-04 17:00:38 -04:00 committed by Joe Gresock
parent ce2a156b9c
commit 0fd4ec50ad
No known key found for this signature in database
GPG Key ID: 37F5B9B6E258C8B7
108 changed files with 2486 additions and 4139 deletions

View File

@ -131,6 +131,11 @@
<artifactId>caffeine</artifactId>
<version>2.9.2</version>
</dependency>
<!-- Version 2 of the AmazonS3EncryptionClient requires bouncy castle -->
<dependency>
<groupId>org.bouncycastle</groupId>
<artifactId>bcprov-jdk18on</artifactId>
</dependency>
</dependencies>
<build>
<plugins>

View File

@ -18,19 +18,55 @@ package org.apache.nifi.processors.aws;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.Protocol;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.auth.AnonymousAWSCredentials;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.auth.PropertiesCredentials;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.http.conn.ssl.SdkTLSSocketFactory;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import org.apache.http.conn.ssl.DefaultHostnameVerifier;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
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.factory.CredentialPropertyDescriptors;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
import org.apache.nifi.proxy.ProxyConfiguration;
import org.apache.nifi.proxy.ProxyConfigurationService;
import org.apache.nifi.proxy.ProxySpec;
import org.apache.nifi.ssl.SSLContextService;
import javax.net.ssl.SSLContext;
import java.io.File;
import java.io.IOException;
import java.net.Proxy;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
/**
* Base class for AWS processors that uses AWSCredentialsProvider interface for creating AWS clients.
@ -39,14 +75,82 @@ import java.util.Map;
*
* @see <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
*/
public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends AmazonWebServiceClient>
extends AbstractAWSProcessor<ClientType> implements VerifiableProcessor {
public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends AmazonWebServiceClient> extends AbstractProcessor
implements VerifiableProcessor {
// Property Descriptors
public static final PropertyDescriptor CREDENTIALS_FILE = CredentialPropertyDescriptors.CREDENTIALS_FILE;
public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY_ID;
public static final PropertyDescriptor SECRET_KEY = CredentialPropertyDescriptors.SECRET_KEY;
public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
.name("Proxy Host")
.description("Proxy host name or IP")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PROXY_HOST_PORT = new PropertyDescriptor.Builder()
.name("Proxy Host Port")
.description("Proxy host port")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.PORT_VALIDATOR)
.build();
public static final PropertyDescriptor PROXY_USERNAME = new PropertyDescriptor.Builder()
.name("proxy-user-name")
.displayName("Proxy Username")
.description("Proxy username")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PROXY_PASSWORD = new PropertyDescriptor.Builder()
.name("proxy-user-password")
.displayName("Proxy Password")
.description("Proxy password")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
.build();
public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
.name("Region")
.description("The AWS Region to connect to.")
.required(true)
.allowableValues(getAvailableRegions())
.defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
.build();
public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
.name("Communications Timeout")
.description("The amount of time to wait in order to establish a connection to AWS or receive data from AWS before timing out.")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.defaultValue("30 secs")
.build();
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
.name("SSL Context Service")
.description("Specifies an optional SSL Context Service that, if provided, will be used to create connections")
.required(false)
.identifiesControllerService(SSLContextService.class)
.build();
public static final PropertyDescriptor ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
.name("Endpoint Override URL")
.description("Endpoint URL to use instead of the AWS default including scheme, host, port, and path. " +
"The AWS libraries select an endpoint URL based on the AWS region, but this property overrides " +
"the selected endpoint URL, allowing use with other S3-compatible endpoints.")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.URL_VALIDATOR)
.build();
/**
* AWS credentials provider service
*
* @see <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
*/
public static final PropertyDescriptor AWS_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder()
.name("AWS Credentials Provider service")
.displayName("AWS Credentials Provider Service")
@ -55,27 +159,214 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
.identifiesControllerService(AWSCredentialsProviderService.class)
.build();
// Relationships
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("FlowFiles are routed to this Relationship after they have been successfully processed.")
.build();
public static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.description("If the Processor is unable to process a given FlowFile, it will be routed to this Relationship.")
.build();
public static final Set<Relationship> relationships = Set.of(REL_SUCCESS, REL_FAILURE);
// Constants
private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
// Member variables
private final Cache<String, ClientType> clientCache = Caffeine.newBuilder()
.maximumSize(10)
.build();
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@OnScheduled
public void onScheduled(final ProcessContext context) {
getClient(context);
}
@OnStopped
public void onStopped() {
this.clientCache.invalidateAll();
this.clientCache.cleanUp();
}
public static AllowableValue createAllowableValue(final Regions region) {
return new AllowableValue(region.getName(), region.getDescription(), "AWS Region Code : " + region.getName());
}
public static AllowableValue[] getAvailableRegions() {
final List<AllowableValue> values = new ArrayList<>();
for (final Regions region : Regions.values()) {
values.add(createAllowableValue(region));
}
return values.toArray(new AllowableValue[0]);
}
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final List<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
final boolean accessKeySet = validationContext.getProperty(ACCESS_KEY).isSet();
final boolean secretKeySet = validationContext.getProperty(SECRET_KEY).isSet();
if ((accessKeySet && !secretKeySet) || (secretKeySet && !accessKeySet)) {
validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("If setting Secret Key or Access Key, must set both").build());
}
final boolean credentialsFileSet = validationContext.getProperty(CREDENTIALS_FILE).isSet();
if ((secretKeySet || accessKeySet) && credentialsFileSet) {
validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("Cannot set both Credentials File and Secret Key/Access Key").build());
}
final boolean proxyHostSet = validationContext.getProperty(PROXY_HOST).isSet();
final boolean proxyPortSet = validationContext.getProperty(PROXY_HOST_PORT).isSet();
final boolean proxyConfigServiceSet = validationContext.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet();
if ((proxyHostSet && !proxyPortSet) || (!proxyHostSet && proxyPortSet)) {
validationResults.add(new ValidationResult.Builder().subject("Proxy Host and Port").valid(false).explanation("If Proxy Host or Proxy Port is set, both must be set").build());
}
final boolean proxyUserSet = validationContext.getProperty(PROXY_USERNAME).isSet();
final boolean proxyPwdSet = validationContext.getProperty(PROXY_PASSWORD).isSet();
if ((proxyUserSet && !proxyPwdSet) || (!proxyUserSet && proxyPwdSet)) {
validationResults.add(new ValidationResult.Builder().subject("Proxy User and Password").valid(false).explanation("If Proxy Username or Proxy Password is set, both must be set").build());
}
if (proxyUserSet && !proxyHostSet) {
validationResults.add(new ValidationResult.Builder().subject("Proxy").valid(false).explanation("If Proxy Username or Proxy Password").build());
}
ProxyConfiguration.validateProxySpec(validationContext, validationResults, PROXY_SPECS);
if (proxyHostSet && proxyConfigServiceSet) {
validationResults.add(new ValidationResult.Builder().subject("Proxy Configuration Service").valid(false)
.explanation("Either Proxy Username and Proxy Password must be set or Proxy Configuration Service but not both").build());
}
return validationResults;
}
protected AWSCredentials getCredentials(final PropertyContext context) {
final String accessKey = context.getProperty(ACCESS_KEY).evaluateAttributeExpressions().getValue();
final String secretKey = context.getProperty(SECRET_KEY).evaluateAttributeExpressions().getValue();
final String credentialsFile = context.getProperty(CREDENTIALS_FILE).getValue();
if (credentialsFile != null) {
try {
return new PropertiesCredentials(new File(credentialsFile));
} catch (final IOException ioe) {
throw new ProcessException("Could not read Credentials File", ioe);
}
}
if (accessKey != null && secretKey != null) {
return new BasicAWSCredentials(accessKey, secretKey);
}
return new AnonymousAWSCredentials();
}
protected ClientConfiguration createConfiguration(final ProcessContext context) {
return createConfiguration(context, context.getMaxConcurrentTasks());
}
protected ClientConfiguration createConfiguration(final PropertyContext context, final int maxConcurrentTasks) {
final ClientConfiguration config = new ClientConfiguration();
config.setMaxConnections(maxConcurrentTasks);
config.setMaxErrorRetry(0);
config.setUserAgentPrefix("NiFi");
config.setProtocol(Protocol.HTTPS);
final int commsTimeout = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
config.setConnectionTimeout(commsTimeout);
config.setSocketTimeout(commsTimeout);
if (this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null) {
final SSLContext sslContext = sslContextService.createContext();
// NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
}
}
final ProxyConfiguration proxyConfig = ProxyConfiguration.getConfiguration(context, () -> {
if (context.getProperty(PROXY_HOST).isSet()) {
final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
Integer proxyPort = context.getProperty(PROXY_HOST_PORT).evaluateAttributeExpressions().asInteger();
String proxyUsername = context.getProperty(PROXY_USERNAME).evaluateAttributeExpressions().getValue();
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())) {
config.setProxyHost(proxyConfig.getProxyServerHost());
config.setProxyPort(proxyConfig.getProxyServerPort());
if (proxyConfig.hasCredential()) {
config.setProxyUsername(proxyConfig.getProxyUserName());
config.setProxyPassword(proxyConfig.getProxyUserPassword());
}
}
return config;
}
protected ClientType createClient(final ProcessContext context) {
return createClient(context, getRegion(context));
}
/**
* Attempts to create the client using the controller service first before falling back to the standard configuration.
* @param context The process context
* @return The created client
*/
@Override
public ClientType createClient(final ProcessContext context, AwsClientDetails awsClientDetails) {
final ControllerService service = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).asControllerService();
if (service != null) {
public ClientType createClient(final ProcessContext context, final Region region) {
getLogger().debug("Using AWS credentials provider service for creating client");
final AWSCredentialsProvider credentialsProvider = getCredentialsProvider(context);
final ClientConfiguration configuration = createConfiguration(context);
final ClientType createdClient = createClient(context, credentialsProvider, configuration);
setRegionAndInitializeEndpoint(awsClientDetails.getRegion(), context, createdClient);
final ClientType createdClient = createClient(context, credentialsProvider, region, configuration, getEndpointConfiguration(context, region));
return createdClient;
} else {
getLogger().debug("Using AWS credentials for creating client");
return super.createClient(context, awsClientDetails);
}
protected AwsClientBuilder.EndpointConfiguration getEndpointConfiguration(final ProcessContext context, final Region region) {
final PropertyValue overrideValue = context.getProperty(ENDPOINT_OVERRIDE);
if (overrideValue == null || !overrideValue.isSet()) {
return null;
}
final String endpointOverride = overrideValue.getValue();
return new AwsClientBuilder.EndpointConfiguration(endpointOverride, region.getName());
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
@ -83,6 +374,7 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
try {
createClient(context);
results.add(new ConfigVerificationResult.Builder()
.outcome(Outcome.SUCCESSFUL)
.verificationStepName("Create Client and Configure Region")
@ -107,13 +399,42 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
* @see <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
*/
protected AWSCredentialsProvider getCredentialsProvider(final ProcessContext context) {
final AWSCredentialsProviderService awsCredentialsProviderService =
context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).asControllerService(AWSCredentialsProviderService.class);
if (awsCredentialsProviderService == null) {
final AWSCredentials credentials = getCredentials(context);
return new AWSStaticCredentialsProvider(credentials);
}
return awsCredentialsProviderService.getCredentialsProvider();
}
protected Region getRegion(final ProcessContext context) {
// if the processor supports REGION, get the configured region.
if (getSupportedPropertyDescriptors().contains(REGION)) {
final String regionValue = context.getProperty(REGION).getValue();
if (regionValue != null) {
return Region.getRegion(Regions.fromName(regionValue));
}
}
return null;
}
/**
* Creates an AWS service client from the context or returns an existing client from the cache
*/
protected ClientType getClient(final ProcessContext context, final Region region) {
final String regionName = region == null ? "" : region.getName();
return clientCache.get(regionName, ignored -> createClient(context, region));
}
protected ClientType getClient(final ProcessContext context) {
return getClient(context, getRegion(context));
}
/**
* Abstract method to create AWS client using credentials provider. This is the preferred method
* for creating AWS clients
@ -122,5 +443,7 @@ public abstract class AbstractAWSCredentialsProviderProcessor<ClientType extends
* @param config AWS client configuration
* @return ClientType the client
*/
protected abstract ClientType createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config);
protected abstract ClientType createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, Region region, ClientConfiguration config,
AwsClientBuilder.EndpointConfiguration endpointConfiguration);
}

View File

@ -1,449 +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;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.Protocol;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AnonymousAWSCredentials;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.auth.PropertiesCredentials;
import com.amazonaws.http.conn.ssl.SdkTLSSocketFactory;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import org.apache.commons.lang3.StringUtils;
import org.apache.http.conn.ssl.DefaultHostnameVerifier;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.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.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
import org.apache.nifi.proxy.ProxyConfiguration;
import org.apache.nifi.proxy.ProxyConfigurationService;
import org.apache.nifi.proxy.ProxySpec;
import org.apache.nifi.ssl.SSLContextService;
import javax.net.ssl.SSLContext;
import java.io.File;
import java.io.IOException;
import java.net.Proxy;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* Abstract base class for AWS processors. This class uses AWS credentials for creating AWS clients
*
* @deprecated use {@link AbstractAWSCredentialsProviderProcessor} instead which uses credentials providers or creating AWS clients
* @see AbstractAWSCredentialsProviderProcessor
*
*/
@Deprecated
public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceClient> extends AbstractSessionFactoryProcessor implements AwsClientProvider<ClientType> {
public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
.description("FlowFiles are routed to success relationship").build();
public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
.description("FlowFiles are routed to failure relationship").build();
public static final Set<Relationship> relationships = Collections.unmodifiableSet(
new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE)));
public static final PropertyDescriptor CREDENTIALS_FILE = CredentialPropertyDescriptors.CREDENTIALS_FILE;
public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY;
public static final PropertyDescriptor SECRET_KEY = CredentialPropertyDescriptors.SECRET_KEY;
public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
.name("Proxy Host")
.description("Proxy host name or IP")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PROXY_HOST_PORT = new PropertyDescriptor.Builder()
.name("Proxy Host Port")
.description("Proxy host port")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.PORT_VALIDATOR)
.build();
public static final PropertyDescriptor PROXY_USERNAME = new PropertyDescriptor.Builder()
.name("proxy-user-name")
.displayName("Proxy Username")
.description("Proxy username")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PROXY_PASSWORD = new PropertyDescriptor.Builder()
.name("proxy-user-password")
.displayName("Proxy Password")
.description("Proxy password")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
.build();
public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
.name("Region")
.description("The AWS Region to connect to.")
.required(true)
.allowableValues(getAvailableRegions())
.defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
.build();
public static final PropertyDescriptor TIMEOUT = new PropertyDescriptor.Builder()
.name("Communications Timeout")
.description("The amount of time to wait in order to establish a connection to AWS or receive data from AWS before timing out.")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.defaultValue("30 secs")
.build();
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
.name("SSL Context Service")
.description("Specifies an optional SSL Context Service that, if provided, will be used to create connections")
.required(false)
.identifiesControllerService(SSLContextService.class)
.build();
public static final PropertyDescriptor ENDPOINT_OVERRIDE = new PropertyDescriptor.Builder()
.name("Endpoint Override URL")
.description("Endpoint URL to use instead of the AWS default including scheme, host, port, and path. " +
"The AWS libraries select an endpoint URL based on the AWS region, but this property overrides " +
"the selected endpoint URL, allowing use with other S3-compatible endpoints.")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(false)
.addValidator(StandardValidators.URL_VALIDATOR)
.build();
protected static final String VPCE_ENDPOINT_SUFFIX = ".vpce.amazonaws.com";
protected static final Pattern VPCE_ENDPOINT_PATTERN = Pattern.compile("^(?:.+[vpce-][a-z0-9-]+\\.)?([a-z0-9-]+)$");
// If protocol is changed to be a property, ensure other uses are also changed
protected static final Protocol DEFAULT_PROTOCOL = Protocol.HTTPS;
protected static final String DEFAULT_USER_AGENT = "NiFi";
private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH};
public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
private final AwsClientCache<ClientType> awsClientCache = new AwsClientCache<>();
public static AllowableValue createAllowableValue(final Regions region) {
return new AllowableValue(region.getName(), region.getDescription(), "AWS Region Code : " + region.getName());
}
public static AllowableValue[] getAvailableRegions() {
final List<AllowableValue> values = new ArrayList<>();
for (final Regions region : Regions.values()) {
values.add(createAllowableValue(region));
}
return values.toArray(new AllowableValue[0]);
}
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final List<ValidationResult> validationResults = new ArrayList<>(super.customValidate(validationContext));
final boolean accessKeySet = validationContext.getProperty(ACCESS_KEY).isSet();
final boolean secretKeySet = validationContext.getProperty(SECRET_KEY).isSet();
if ((accessKeySet && !secretKeySet) || (secretKeySet && !accessKeySet)) {
validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("If setting Secret Key or Access Key, must set both").build());
}
final boolean credentialsFileSet = validationContext.getProperty(CREDENTIALS_FILE).isSet();
if ((secretKeySet || accessKeySet) && credentialsFileSet) {
validationResults.add(new ValidationResult.Builder().input("Access Key").valid(false).explanation("Cannot set both Credentials File and Secret Key/Access Key").build());
}
final boolean proxyHostSet = validationContext.getProperty(PROXY_HOST).isSet();
final boolean proxyPortSet = validationContext.getProperty(PROXY_HOST_PORT).isSet();
final boolean proxyConfigServiceSet = validationContext.getProperty(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE).isSet();
if ((proxyHostSet && !proxyPortSet) || (!proxyHostSet && proxyPortSet)) {
validationResults.add(new ValidationResult.Builder().subject("Proxy Host and Port").valid(false).explanation("If Proxy Host or Proxy Port is set, both must be set").build());
}
final boolean proxyUserSet = validationContext.getProperty(PROXY_USERNAME).isSet();
final boolean proxyPwdSet = validationContext.getProperty(PROXY_PASSWORD).isSet();
if ((proxyUserSet && !proxyPwdSet) || (!proxyUserSet && proxyPwdSet)) {
validationResults.add(new ValidationResult.Builder().subject("Proxy User and Password").valid(false).explanation("If Proxy Username or Proxy Password is set, both must be set").build());
}
if (proxyUserSet && !proxyHostSet) {
validationResults.add(new ValidationResult.Builder().subject("Proxy").valid(false).explanation("If Proxy Username or Proxy Password").build());
}
ProxyConfiguration.validateProxySpec(validationContext, validationResults, PROXY_SPECS);
if (proxyHostSet && proxyConfigServiceSet) {
validationResults.add(new ValidationResult.Builder().subject("Proxy Configuration Service").valid(false)
.explanation("Either Proxy Username and Proxy Password must be set or Proxy Configuration Service but not both").build());
}
return validationResults;
}
protected ClientConfiguration createConfiguration(final ProcessContext context) {
return createConfiguration(context, context.getMaxConcurrentTasks());
}
protected ClientConfiguration createConfiguration(final PropertyContext context, final int maxConcurrentTasks) {
final ClientConfiguration config = new ClientConfiguration();
config.setMaxConnections(maxConcurrentTasks);
config.setMaxErrorRetry(0);
config.setUserAgent(DEFAULT_USER_AGENT);
// If this is changed to be a property, ensure other uses are also changed
config.setProtocol(DEFAULT_PROTOCOL);
final int commsTimeout = context.getProperty(TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
config.setConnectionTimeout(commsTimeout);
config.setSocketTimeout(commsTimeout);
if(this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null) {
final SSLContext sslContext = sslContextService.createContext();
// NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
}
}
final ProxyConfiguration proxyConfig = ProxyConfiguration.getConfiguration(context, () -> {
if (context.getProperty(PROXY_HOST).isSet()) {
final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
Integer proxyPort = context.getProperty(PROXY_HOST_PORT).evaluateAttributeExpressions().asInteger();
String proxyUsername = context.getProperty(PROXY_USERNAME).evaluateAttributeExpressions().getValue();
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())) {
config.setProxyHost(proxyConfig.getProxyServerHost());
config.setProxyPort(proxyConfig.getProxyServerPort());
if (proxyConfig.hasCredential()) {
config.setProxyUsername(proxyConfig.getProxyUserName());
config.setProxyPassword(proxyConfig.getProxyUserPassword());
}
}
return config;
}
@OnScheduled
public void onScheduled(final ProcessContext context) {
getClient(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;
public ClientType createClient(final ProcessContext context, AwsClientDetails awsClientDetails) {
final AWSCredentials credentials = getCredentials(context);
final ClientConfiguration configuration = createConfiguration(context);
final ClientType createdClient = createClient(context, credentials, configuration);
setRegionAndInitializeEndpoint(awsClientDetails.getRegion(), context, createdClient);
return createdClient;
}
protected ClientType createClient(ProcessContext context) {
return createClient(context, new AwsClientDetails(getRegion(context)));
}
protected void setRegionAndInitializeEndpoint(final Region region, final ProcessContext context, final AmazonWebServiceClient client) {
if (region!= null && client != null) {
client.setRegion(region);
}
// if the endpoint override has been configured, set the endpoint.
// (per Amazon docs this should only be configured at client creation)
if (client != null && getSupportedPropertyDescriptors().contains(ENDPOINT_OVERRIDE)) {
final String urlstr = StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue());
if (!urlstr.isEmpty()) {
getLogger().info("Overriding endpoint with {}", urlstr);
if (urlstr.endsWith(VPCE_ENDPOINT_SUFFIX)) {
// handling vpce endpoints
// falling back to the configured region if the parse fails
// e.g. in case of https://vpce-***-***.sqs.{region}.vpce.amazonaws.com
String regionValue = parseRegionForVPCE(urlstr, region.getName());
client.setEndpoint(urlstr, client.getServiceName(), regionValue);
} else if (isCustomSignerConfigured(context)) {
// handling endpoints with a user defined custom signer
client.setEndpoint(urlstr, client.getServiceName(), region.getName());
} else {
// handling other (non-vpce, no custom signer) custom endpoints where the AWS library can parse the region out
// e.g. https://sqs.{region}.***.***.***.gov
client.setEndpoint(urlstr);
}
}
}
}
protected Region getRegion(final ProcessContext context) {
// if the processor supports REGION, get the configured region.
if (getSupportedPropertyDescriptors().contains(REGION)) {
final String regionValue = context.getProperty(REGION).getValue();
if (regionValue != null) {
return Region.getRegion(Regions.fromName(regionValue));
}
}
return null;
}
protected boolean isCustomSignerConfigured(final ProcessContext context) {
return false;
}
/*
Note to developer(s):
When setting an endpoint for an AWS Client i.e. client.setEndpoint(endpointUrl),
AWS Java SDK fails to parse the region correctly when the provided endpoint
is an AWS PrivateLink so this method does the job of parsing the region name and
returning it.
Refer NIFI-5456, NIFI-5893 & NIFI-8662
*/
private String parseRegionForVPCE(String url, String configuredRegion) {
int index = url.length() - VPCE_ENDPOINT_SUFFIX.length();
Matcher matcher = VPCE_ENDPOINT_PATTERN.matcher(url.substring(0, index));
if (matcher.matches()) {
return matcher.group(1);
} else {
getLogger().info("Unable to get a match with the VPCE endpoint pattern; using the configured region: " + configuredRegion);
return configuredRegion;
}
}
protected AWSCredentials getCredentials(final PropertyContext context) {
final String accessKey = context.getProperty(ACCESS_KEY).evaluateAttributeExpressions().getValue();
final String secretKey = context.getProperty(SECRET_KEY).evaluateAttributeExpressions().getValue();
final String credentialsFile = context.getProperty(CREDENTIALS_FILE).getValue();
if (credentialsFile != null) {
try {
return new PropertiesCredentials(new File(credentialsFile));
} catch (final IOException ioe) {
throw new ProcessException("Could not read Credentials File", ioe);
}
}
if (accessKey != null && secretKey != null) {
return new BasicAWSCredentials(accessKey, secretKey);
}
return new AnonymousAWSCredentials();
}
@OnStopped
public void onStopped() {
this.awsClientCache.clearCache();
}
/**
* Creates an AWS service client from the context or returns an existing client from the cache
* @param context The process context
* @param awsClientDetails details of the AWS client
* @return The created client
*/
protected ClientType getClient(final ProcessContext context, AwsClientDetails awsClientDetails) {
return this.awsClientCache.getOrCreateClient(context, awsClientDetails, this);
}
protected ClientType getClient(final ProcessContext context) {
final AwsClientDetails awsClientDetails = new AwsClientDetails(getRegion(context));
return getClient(context, awsClientDetails);
}
/**
* Create client from the arguments
* @param context process context
* @param credentials static aws credentials
* @param config aws client configuration
* @return ClientType aws client
*
* @deprecated use {@link AbstractAWSCredentialsProviderProcessor#createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)}
*/
@Deprecated
protected abstract ClientType createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config);
}

View File

@ -1,41 +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;
import com.amazonaws.AmazonWebServiceClient;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import org.apache.nifi.processor.ProcessContext;
public class AwsClientCache<ClientType extends AmazonWebServiceClient> {
private static final int MAXIMUM_CACHE_SIZE = 10;
private final Cache<AwsClientDetails, ClientType> clientCache = Caffeine.newBuilder()
.maximumSize(MAXIMUM_CACHE_SIZE)
.build();
public ClientType getOrCreateClient(final ProcessContext context, final AwsClientDetails clientDetails, final AwsClientProvider<ClientType> provider) {
return clientCache.get(clientDetails, ignored -> provider.createClient(context, clientDetails));
}
public void clearCache() {
clientCache.invalidateAll();
clientCache.cleanUp();
}
}

View File

@ -1,58 +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;
import com.amazonaws.regions.Region;
import java.util.Objects;
/**
* This class contains the AWS client details used to distinguish between the various AWS clients stored in the cache.
* The class acts as a cache key for @link AwsClientCache.
* AwsClientDetails contains the region only, since actually the region value may come from the FlowFile attributes.
*/
public class AwsClientDetails {
private Region region;
public AwsClientDetails(Region region) {
this.region = region;
}
public Region getRegion() {
return region;
}
public void setRegion(final Region region) {
this.region = region;
}
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final AwsClientDetails that = (AwsClientDetails) o;
return Objects.equals(region, that.region);
}
@Override
public int hashCode() {
return Objects.hash(region);
}
}

View File

@ -1,32 +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;
import com.amazonaws.AmazonWebServiceClient;
import org.apache.nifi.processor.ProcessContext;
public interface AwsClientProvider<ClientType extends AmazonWebServiceClient> {
/**
* Creates an AWS client using process context and AWS client details.
*
* @param context process context
* @param awsClientDetails AWS client details
* @return AWS client
*/
ClientType createClient(final ProcessContext context, final AwsClientDetails awsClientDetails);
}

View File

@ -1,52 +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;
import com.amazonaws.auth.Signer;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.resource.ResourceCardinality;
import org.apache.nifi.components.resource.ResourceType;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.util.StandardValidators;
public final class AwsPropertyDescriptors {
private AwsPropertyDescriptors() {
// constant class' constructor
}
public static final PropertyDescriptor CUSTOM_SIGNER_CLASS_NAME = new PropertyDescriptor.Builder()
.name("custom-signer-class-name")
.displayName("Custom Signer Class Name")
.description(String.format("Fully qualified class name of the custom signer class. The signer must implement %s interface.", Signer.class.getName()))
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.build();
public static final PropertyDescriptor CUSTOM_SIGNER_MODULE_LOCATION = new PropertyDescriptor.Builder()
.name("custom-signer-module-location")
.displayName("Custom Signer Module Location")
.description("Comma-separated list of paths to files and/or directories which contain the custom signer's JAR file and its dependencies (if any).")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE, ResourceType.DIRECTORY)
.dynamicallyModifiesClasspath(true)
.build();
}

View File

@ -16,13 +16,13 @@
*/
package org.apache.nifi.processors.aws.credentials.provider.factory;
import com.amazonaws.auth.Signer;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.resource.ResourceCardinality;
import org.apache.nifi.components.resource.ResourceType;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AwsPropertyDescriptors;
import org.apache.nifi.ssl.SSLContextService;
import software.amazon.awssdk.regions.Region;
@ -71,7 +71,7 @@ public class CredentialPropertyDescriptors {
.description("Path to a file containing AWS access key and secret key in properties file format.")
.build();
public static final PropertyDescriptor ACCESS_KEY = new PropertyDescriptor.Builder()
public static final PropertyDescriptor ACCESS_KEY_ID = new PropertyDescriptor.Builder()
.name("Access Key")
.displayName("Access Key ID")
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
@ -245,13 +245,25 @@ public class CredentialPropertyDescriptors {
.build();
public static final PropertyDescriptor ASSUME_ROLE_STS_CUSTOM_SIGNER_CLASS_NAME = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_CLASS_NAME)
.name("custom-signer-class-name")
.displayName("Custom Signer Class Name")
.description(String.format("Fully qualified class name of the custom signer class. The signer must implement %s interface.", Signer.class.getName()))
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.dependsOn(ASSUME_ROLE_STS_SIGNER_OVERRIDE, CUSTOM_SIGNER)
.build();
public static final PropertyDescriptor ASSUME_ROLE_STS_CUSTOM_SIGNER_MODULE_LOCATION = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_MODULE_LOCATION)
.name("custom-signer-module-location")
.displayName("Custom Signer Module Location")
.description("Comma-separated list of paths to files and/or directories which contain the custom signer's JAR file and its dependencies (if any).")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE, ResourceType.DIRECTORY)
.dependsOn(ASSUME_ROLE_STS_SIGNER_OVERRIDE, CUSTOM_SIGNER)
.dynamicallyModifiesClasspath(true)
.build();
public static AllowableValue createAllowableValue(final Region region) {

View File

@ -16,17 +16,15 @@
*/
package org.apache.nifi.processors.aws.dynamodb;
import java.math.BigDecimal;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.model.AttributeValue;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.AllowableValue;
@ -39,14 +37,16 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.model.AttributeValue;
import java.math.BigDecimal;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Base class for NiFi dynamo db related processors
@ -177,22 +177,17 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
* Create client using credentials provider. This is the preferred way for creating clients
*/
@Override
protected AmazonDynamoDBClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
protected AmazonDynamoDBClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
getLogger().debug("Creating client with credentials provider");
return new AmazonDynamoDBClient(credentialsProvider, config);
return (AmazonDynamoDBClient) AmazonDynamoDBClient.builder()
.withClientConfiguration(config)
.withCredentials(credentialsProvider)
.withEndpointConfiguration(endpointConfiguration)
.withRegion(region.getName())
.build();
}
/**
* Create client using AWSCredentials
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Deprecated
@Override
protected AmazonDynamoDBClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().debug("Creating client with aws credentials");
return new AmazonDynamoDBClient(credentials, config);
}
protected Object getValue(final ProcessContext context, final PropertyDescriptor type, final PropertyDescriptor value, final Map<String, String> attributes) {
if ( context.getProperty(type).getValue().equals(ALLOWABLE_VALUE_STRING.getValue())) {
@ -348,6 +343,7 @@ public abstract class AbstractDynamoDBProcessor extends AbstractAWSCredentialsPr
}
}
@OnStopped
public void onStopped() {
this.dynamoDB = null;

View File

@ -1,66 +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.dynamodb;
import java.util.List;
import java.util.Map;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessSession;
import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
import com.amazonaws.services.dynamodbv2.model.AttributeValue;
import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult;
import com.amazonaws.services.dynamodbv2.model.WriteRequest;
public abstract class AbstractWriteDynamoDBProcessor extends AbstractDynamoDBProcessor {
/**
* Helper method to handle unprocessed items items
* @param session process session
* @param keysToFlowFileMap map of flow db primary key to flow file
* @param table dynamodb table
* @param hashKeyName the hash key name
* @param hashKeyValueType the hash key value
* @param rangeKeyName the range key name
* @param rangeKeyValueType range key value
* @param outcome the write outcome
*/
protected void handleUnprocessedItems(final ProcessSession session, Map<ItemKeys, FlowFile> keysToFlowFileMap, final String table, final String hashKeyName, final String hashKeyValueType,
final String rangeKeyName, final String rangeKeyValueType, BatchWriteItemOutcome outcome) {
BatchWriteItemResult result = outcome.getBatchWriteItemResult();
// Handle unprocessed items
List<WriteRequest> unprocessedItems = result.getUnprocessedItems().get(table);
if ( unprocessedItems != null && unprocessedItems.size() > 0 ) {
for ( WriteRequest request : unprocessedItems) {
Map<String,AttributeValue> item = getRequestItem(request);
Object hashKeyValue = getValue(item, hashKeyName, hashKeyValueType);
Object rangeKeyValue = getValue(item, rangeKeyName, rangeKeyValueType);
sendUnprocessedToUnprocessedRelationship(session, keysToFlowFileMap, hashKeyValue, rangeKeyValue);
}
}
}
/**
* Get the request item key and attribute value
* @param writeRequest write request
* @return Map of keys and values
*/
protected abstract Map<String, AttributeValue> getRequestItem(WriteRequest writeRequest);
}

View File

@ -16,10 +16,7 @@
*/
package org.apache.nifi.processors.aws.dynamodb;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import java.util.Objects;
/**
* Utility class to keep a map of keys and flow files
@ -29,25 +26,35 @@ class ItemKeys {
protected Object hashKey = "";
protected Object rangeKey = "";
public ItemKeys(Object hashKey, Object rangeKey) {
if ( hashKey != null )
public ItemKeys(final Object hashKey, final Object rangeKey) {
if (hashKey != null) {
this.hashKey = hashKey;
if ( rangeKey != null )
}
if (rangeKey != null) {
this.rangeKey = rangeKey;
}
}
@Override
public String toString() {
return ToStringBuilder.reflectionToString(this,ToStringStyle.SHORT_PREFIX_STYLE);
return "ItemKeys[hashKey=" + hashKey + ", rangeKey=" + rangeKey + "]";
}
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final ItemKeys itemKeys = (ItemKeys) o;
return Objects.equals(hashKey, itemKeys.hashKey) && Objects.equals(rangeKey, itemKeys.rangeKey);
}
@Override
public int hashCode() {
return HashCodeBuilder.reflectionHashCode(this, false);
}
@Override
public boolean equals(Object other) {
return EqualsBuilder.reflectionEquals(this, other, false);
return Objects.hash(hashKey, rangeKey);
}
}

View File

@ -49,22 +49,24 @@ public class KinesisProcessorUtils {
*/
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);
final List<FlowFile> flowFiles = new ArrayList<>();
long currentBufferSizeBytes = 0;
for (int i = 0; (i < batchSize) && (currentBufferSizeBytes <= maxBufferSizeBytes); i++) {
final FlowFile flowFileCandidate = session.get();
if (flowFileCandidate != null) {
if (flowFileCandidate == null) {
break;
}
if (flowFileCandidate.getSize() > MAX_MESSAGE_SIZE) {
handleFlowFileTooBig(session, flowFileCandidate, errorMessageAttribute, logger);
continue;
}
currentBufferSizeBytes += flowFileCandidate.getSize();
flowFiles.add(flowFileCandidate);
}
}
return flowFiles;
}

View File

@ -1,62 +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.firehose;
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.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.firehose.FirehoseClient;
import software.amazon.awssdk.services.firehose.FirehoseClientBuilder;
/**
* This class is the base class for Kinesis Firehose processors
*/
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")
.description("The name of kinesis firehose delivery stream")
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")
.description("Batch size for messages (1-500).")
.defaultValue("250")
.required(false)
.addValidator(StandardValidators.createLongValidator(1, 500, true))
.sensitive(false)
.build();
public static final PropertyDescriptor MAX_MESSAGE_BUFFER_SIZE_MB = new PropertyDescriptor.Builder()
.name("Max message buffer size")
.description("Max message buffer")
.defaultValue("1 MB")
.required(false)
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
.sensitive(false)
.build();
@Override
protected FirehoseClientBuilder createClientBuilder(final ProcessContext context) {
return FirehoseClient.builder();
}
}

View File

@ -1,34 +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 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,34 +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 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

@ -1,31 +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 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

@ -1,73 +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.lambda;
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.AbstractAWSCredentialsProviderProcessor;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.services.lambda.AWSLambdaClient;
/**
* This class is the base class for invoking aws lambda function
*/
public abstract class AbstractAWSLambdaProcessor extends AbstractAWSCredentialsProviderProcessor<AWSLambdaClient> {
public static final PropertyDescriptor AWS_LAMBDA_FUNCTION_NAME = new PropertyDescriptor.Builder()
.name("Amazon Lambda Name")
.description("The Lambda Function Name")
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor AWS_LAMBDA_FUNCTION_QUALIFIER = new PropertyDescriptor.Builder()
.name("Amazon Lambda Qualifier (version)")
.description("The Lambda Function Version")
.defaultValue("$LATEST")
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
/**
* Create client using aws credentials provider. This is the preferred way for creating clients
*/
@Override
protected AWSLambdaClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials provider");
return new AWSLambdaClient(credentialsProvider, config);
}
/**
* Create client using AWSCredentials
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Override
protected AWSLambdaClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials");
return new AWSLambdaClient(credentials, config);
}
}

View File

@ -18,13 +18,15 @@ package org.apache.nifi.processors.aws.s3;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.auth.Signer;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Builder;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.S3ClientOptions;
import com.amazonaws.services.s3.AmazonS3ClientBuilder;
import com.amazonaws.services.s3.model.AccessControlList;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CannedAccessControlList;
@ -40,6 +42,8 @@ import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.ConfigVerificationResult;
import org.apache.nifi.components.ConfigVerificationResult.Outcome;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.resource.ResourceCardinality;
import org.apache.nifi.components.resource.ResourceType;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
@ -48,9 +52,6 @@ import org.apache.nifi.processor.ProcessSession;
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.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.AwsClientDetails;
import org.apache.nifi.processors.aws.AwsPropertyDescriptors;
import org.apache.nifi.processors.aws.signer.AwsCustomSignerUtil;
import org.apache.nifi.processors.aws.signer.AwsSignerType;
@ -59,6 +60,7 @@ import java.util.Collections;
import java.util.EnumSet;
import java.util.List;
import java.util.Map;
import java.util.function.Consumer;
import static java.lang.String.format;
import static org.apache.nifi.processors.aws.signer.AwsSignerType.AWS_S3_V2_SIGNER;
@ -126,13 +128,17 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.defaultValue("${s3.owner}")
.build();
public static final PropertyDescriptor BUCKET = new PropertyDescriptor.Builder()
public static final PropertyDescriptor BUCKET_WITHOUT_DEFAULT_VALUE = new PropertyDescriptor.Builder()
.name("Bucket")
.description("The S3 Bucket to interact with")
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor BUCKET_WITH_DEFAULT_VALUE = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(BUCKET_WITHOUT_DEFAULT_VALUE)
.defaultValue("${s3.bucket}")
.build();
public static final PropertyDescriptor KEY = new PropertyDescriptor.Builder()
.name("Object Key")
.description("The S3 Object Key to use. This is analogous to a filename for traditional file systems.")
@ -155,13 +161,25 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
.build();
public static final PropertyDescriptor S3_CUSTOM_SIGNER_CLASS_NAME = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_CLASS_NAME)
.name("custom-signer-class-name")
.displayName("Custom Signer Class Name")
.description(String.format("Fully qualified class name of the custom signer class. The signer must implement %s interface.", Signer.class.getName()))
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.dependsOn(SIGNER_OVERRIDE, CUSTOM_SIGNER)
.build();
public static final PropertyDescriptor S3_CUSTOM_SIGNER_MODULE_LOCATION = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AwsPropertyDescriptors.CUSTOM_SIGNER_MODULE_LOCATION)
.name("custom-signer-module-location")
.displayName("Custom Signer Module Location")
.description("Comma-separated list of paths to files and/or directories which contain the custom signer's JAR file and its dependencies (if any).")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE, ResourceType.DIRECTORY)
.dependsOn(SIGNER_OVERRIDE, CUSTOM_SIGNER)
.dynamicallyModifiesClasspath(true)
.build();
public static final String S3_REGION_ATTRIBUTE = "s3.region" ;
@ -170,7 +188,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
"Uses '" + S3_REGION_ATTRIBUTE + "' FlowFile attribute as region.");
public static final PropertyDescriptor S3_REGION = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AbstractAWSProcessor.REGION)
.fromPropertyDescriptor(REGION)
.allowableValues(getAvailableS3Regions())
.build();
@ -203,36 +221,46 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
* Create client using credentials provider. This is the preferred way for creating clients
*/
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region,
final ClientConfiguration config, final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
getLogger().info("Creating client with credentials provider");
initializeSignerOverride(context, config);
AmazonS3EncryptionService encryptionService = context.getProperty(ENCRYPTION_SERVICE).asControllerService(AmazonS3EncryptionService.class);
AmazonS3Client s3 = null;
final Consumer<AmazonS3Builder<?, ?>> clientBuilder = builder -> {
if (endpointConfiguration == null) {
builder.withRegion(region.getName());
} else {
builder.withEndpointConfiguration(endpointConfiguration);
}
builder.withClientConfiguration(config);
builder.withCredentials(credentialsProvider);
final Boolean useChunkedEncoding = context.getProperty(USE_CHUNKED_ENCODING).asBoolean();
if (useChunkedEncoding == Boolean.FALSE) {
builder.disableChunkedEncoding();
}
final Boolean usePathStyleAccess = context.getProperty(USE_PATH_STYLE_ACCESS).asBoolean();
final boolean endpointOverrideSet = !StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()).isEmpty();
if (usePathStyleAccess == Boolean.TRUE || endpointOverrideSet) {
builder.withPathStyleAccessEnabled(true);
}
};
AmazonS3 s3Client = null;
if (encryptionService != null) {
s3 = encryptionService.createEncryptionClient(credentialsProvider, config);
s3Client = encryptionService.createEncryptionClient(clientBuilder);
}
if (s3Client == null) {
final AmazonS3ClientBuilder builder = AmazonS3Client.builder();
clientBuilder.accept(builder);
s3Client = builder.build();
}
if (s3 == null) {
s3 = new AmazonS3Client(credentialsProvider, config);
return (AmazonS3Client) s3Client;
}
configureClientOptions(context, s3);
return s3;
}
/**
* Create client using AWSCredentials
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Deprecated
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client with AWS credentials");
return createClient(context, new AWSStaticCredentialsProvider(credentials), config);
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
@ -264,8 +292,8 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
* @return The created S3 client
*/
protected AmazonS3Client getS3Client(final ProcessContext context, final Map<String, String> attributes) {
final AwsClientDetails clientDetails = getAwsClientDetails(context, attributes);
return getClient(context, clientDetails);
final Region region = resolveRegion(context, attributes);
return getClient(context, region);
}
/**
@ -275,8 +303,8 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
* @return The newly created S3 client
*/
protected AmazonS3Client createClient(final ProcessContext context, final Map<String, String> attributes) {
final AwsClientDetails clientDetails = getAwsClientDetails(context, attributes);
return createClient(context, clientDetails);
final Region region = resolveRegion(context, attributes);
return createClient(context, region);
}
@Override
@ -287,28 +315,6 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
}
}
private void configureClientOptions(final ProcessContext context, final AmazonS3Client s3) {
final S3ClientOptions.Builder builder = S3ClientOptions.builder();
// disable chunked encoding if "Use Chunked Encoding" has been set to false, otherwise use the default (not disabled)
final Boolean useChunkedEncoding = context.getProperty(USE_CHUNKED_ENCODING).asBoolean();
if (useChunkedEncoding != null && !useChunkedEncoding) {
builder.disableChunkedEncoding();
}
// use PathStyleAccess if "Use Path Style Access" has been set to true, otherwise use the default (false)
final Boolean usePathStyleAccess = context.getProperty(USE_PATH_STYLE_ACCESS).asBoolean();
if (usePathStyleAccess != null && usePathStyleAccess) {
builder.setPathStyleAccess(true);
}
// if ENDPOINT_OVERRIDE is set, use PathStyleAccess
if (!StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).evaluateAttributeExpressions().getValue()).isEmpty()){
builder.setPathStyleAccess(true);
}
s3.setS3ClientOptions(builder.build());
}
private void initializeSignerOverride(final ProcessContext context, final ClientConfiguration config) {
final String signer = context.getProperty(SIGNER_OVERRIDE).getValue();
@ -323,12 +329,6 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
}
}
@Override
protected boolean isCustomSignerConfigured(final ProcessContext context) {
final String signer = context.getProperty(SIGNER_OVERRIDE).getValue();
final AwsSignerType signerType = AwsSignerType.forValue(signer);
return signerType == CUSTOM_SIGNER;
}
protected Grantee createGrantee(final String value) {
if (StringUtils.isEmpty(value)) {
@ -374,9 +374,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
if (!StringUtils.isEmpty(ownerId)) {
final Owner owner = new Owner();
owner.setId(ownerId);
if (acl == null) {
acl = new AccessControlList();
}
acl.setOwner(owner);
}
@ -423,8 +421,7 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
if (e instanceof AmazonS3Exception) {
flowFile = putAttribute(session, flowFile, "s3.additionalDetails", ((AmazonS3Exception) e).getAdditionalDetails());
}
if (e instanceof AmazonServiceException) {
final AmazonServiceException ase = (AmazonServiceException) e;
if (e instanceof final AmazonServiceException ase) {
flowFile = putAttribute(session, flowFile, "s3.statusCode", ase.getStatusCode());
flowFile = putAttribute(session, flowFile, "s3.errorCode", ase.getErrorCode());
flowFile = putAttribute(session, flowFile, "s3.errorMessage", ase.getErrorMessage());
@ -486,8 +483,4 @@ public abstract class AbstractS3Processor extends AbstractAWSCredentialsProvider
return ArrayUtils.addAll(availableRegions, ATTRIBUTE_DEFINED_REGION);
}
private AwsClientDetails getAwsClientDetails(final ProcessContext context, final Map<String, String> attributes) {
final Region region = resolveRegion(context, attributes);
return new AwsClientDetails(region);
}
}

View File

@ -1,57 +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.sns;
import org.apache.nifi.components.AllowableValue;
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.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sns.SnsClient;
import software.amazon.awssdk.services.sns.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");
protected static final AllowableValue ARN_TYPE_TARGET
= new AllowableValue("Target ARN", "Target ARN", "The ARN is the name of a particular Target, used to notify a specific subscriber");
public static final PropertyDescriptor ARN = new PropertyDescriptor.Builder()
.name("Amazon Resource Name (ARN)")
.description("The name of the resource to which notifications should be published")
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor ARN_TYPE = new PropertyDescriptor.Builder()
.name("ARN Type")
.description("The type of Amazon Resource Name that is being used.")
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.allowableValues(ARN_TYPE_TOPIC, ARN_TYPE_TARGET)
.defaultValue(ARN_TYPE_TOPIC.getValue())
.build();
@Override
protected SnsClientBuilder createClientBuilder(final ProcessContext context) {
return SnsClient.builder();
}
}

View File

@ -1,50 +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.sqs;
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.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sqs.SqsClient;
import software.amazon.awssdk.services.sqs.SqsClientBuilder;
public abstract class AbstractSQSProcessor extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")
.description("The maximum number of messages to send in a single network request")
.required(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("25")
.build();
public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
.name("Queue URL")
.description("The URL of the queue to act upon")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.build();
@Override
protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
return SqsClient.builder();
}
}

View File

@ -24,6 +24,7 @@ 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 software.amazon.awssdk.regions.Region;
import javax.net.ssl.TrustManager;
import java.time.Duration;
@ -54,10 +55,9 @@ public abstract class AbstractAwsAsyncProcessor<
* @param context The process context
* @return The created client
*/
@Override
public T createClient(final ProcessContext context) {
public T createClient(final ProcessContext context, final Region region) {
final U clientBuilder = createClientBuilder(context);
this.configureClientBuilder(clientBuilder, context);
this.configureClientBuilder(clientBuilder, region, context);
return clientBuilder.build();
}
@ -101,4 +101,5 @@ public abstract class AbstractAwsAsyncProcessor<
return builder.build();
}
}

View File

@ -16,6 +16,8 @@
*/
package org.apache.nifi.processors.aws.v2;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
@ -78,8 +80,7 @@ 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>
extends AbstractSessionFactoryProcessor implements VerifiableProcessor, AwsClientProvider<T> {
public abstract class AbstractAwsProcessor<T extends SdkClient> extends AbstractSessionFactoryProcessor implements VerifiableProcessor {
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
@ -97,7 +98,7 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
public static final PropertyDescriptor CREDENTIALS_FILE = CredentialPropertyDescriptors.CREDENTIALS_FILE;
public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY;
public static final PropertyDescriptor ACCESS_KEY = CredentialPropertyDescriptors.ACCESS_KEY_ID;
public static final PropertyDescriptor SECRET_KEY = CredentialPropertyDescriptors.SECRET_KEY;
@ -187,7 +188,7 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
private final AwsClientCache<T> awsClientCache = new AwsClientCache<>();
private final Cache<Region, T> clientCache = Caffeine.newBuilder().build();
/**
* Configure the http client on the builder.
@ -275,14 +276,16 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
@OnStopped
public void onStopped() {
this.awsClientCache.clearCache();
clientCache.asMap().values().forEach(SdkClient::close);
clientCache.invalidateAll();
clientCache.cleanUp();
}
@Override
public List<ConfigVerificationResult> verify(final ProcessContext context, final ComponentLog verificationLogger, final Map<String, String> attributes) {
final List<ConfigVerificationResult> results = new ArrayList<>();
try (final T client = createClient(context)) {
try (final T ignored = createClient(context)) {
results.add(new ConfigVerificationResult.Builder()
.outcome(Outcome.SUCCESSFUL)
.verificationStepName("Create Client")
@ -302,31 +305,27 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
/**
* Creates an AWS service client from the context or returns an existing client from the cache
* @param context The process context
* @param awsClientDetails details of the AWS client
* @return The created client
* @return The created or cached client
*/
protected T getClient(final ProcessContext context, final AwsClientDetails awsClientDetails) {
return this.awsClientCache.getOrCreateClient(context, awsClientDetails, this);
protected T getClient(final ProcessContext context, final Region region) {
return clientCache.get(region, ignored -> createClient(context, region));
}
protected T getClient(final ProcessContext context) {
final AwsClientDetails awsClientDetails = new AwsClientDetails(getRegion(context));
return getClient(context, awsClientDetails);
return getClient(context, getRegion(context));
}
protected <C extends SdkClient, B extends AwsClientBuilder<B, C>>
void configureClientBuilder(final B clientBuilder, final ProcessContext context) {
configureClientBuilder(clientBuilder, context, ENDPOINT_OVERRIDE);
void configureClientBuilder(final B clientBuilder, final Region region, final ProcessContext context) {
configureClientBuilder(clientBuilder, region, context, ENDPOINT_OVERRIDE);
}
protected <C extends SdkClient, B extends AwsClientBuilder<B, C>>
void configureClientBuilder(final B clientBuilder, final ProcessContext context, final PropertyDescriptor endpointOverrideDescriptor) {
void configureClientBuilder(final B clientBuilder, final Region region, 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);
}
@ -391,10 +390,10 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
return region;
}
protected void configureEndpoint(final ProcessContext context, final SdkClientBuilder clientBuilder, final PropertyDescriptor endpointOverrideDescriptor) {
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(endpointOverrideDescriptor)) {
if (endpointOverrideDescriptor != null && getSupportedPropertyDescriptors().contains(endpointOverrideDescriptor)) {
final String endpointOverride = StringUtils.trimToEmpty(context.getProperty(endpointOverrideDescriptor).evaluateAttributeExpressions().getValue());
if (!endpointOverride.isEmpty()) {
@ -405,9 +404,6 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
}
}
protected void configureEndpoint(final ProcessContext context, final SdkClientBuilder clientBuilder) {
configureEndpoint(context, clientBuilder, ENDPOINT_OVERRIDE);
}
/**
* Get credentials provider using the {@link AwsCredentialsProvider}
@ -438,4 +434,19 @@ public abstract class AbstractAwsProcessor<T extends SdkClient>
return AnonymousCredentialsProvider.create();
}
protected T createClient(final ProcessContext context) {
return createClient(context, getRegion(context));
}
/**
* Creates an AWS client using process context and AWS client details.
*
* @param context process context
* @param region the AWS Region
* @return AWS client
*/
protected abstract T createClient(final ProcessContext context, final Region region);
}

View File

@ -24,6 +24,7 @@ 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 software.amazon.awssdk.regions.Region;
import javax.net.ssl.TrustManager;
import java.net.URI;
@ -55,10 +56,13 @@ public abstract class AbstractAwsSyncProcessor<
* @param context The process context
* @return The created client
*/
@Override
public T createClient(final ProcessContext context) {
return createClient(context, getRegion(context));
}
public T createClient(final ProcessContext context, final Region region) {
final U clientBuilder = createClientBuilder(context);
this.configureClientBuilder(clientBuilder, context);
this.configureClientBuilder(clientBuilder, region, context);
return clientBuilder.build();
}

View File

@ -1,42 +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.v2;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import org.apache.nifi.processor.ProcessContext;
import software.amazon.awssdk.core.SdkClient;
public class AwsClientCache<T extends SdkClient> {
private final Cache<AwsClientDetails, T> clientCache = Caffeine.newBuilder().build();
public T getOrCreateClient(final ProcessContext context, final AwsClientDetails clientDetails, final AwsClientProvider<T> provider) {
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

@ -1,59 +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.v2;
import software.amazon.awssdk.regions.Region;
import java.util.Objects;
/**
* This class contains the AWS client details used to distinguish between the various AWS clients stored in the cache.
* The class acts as a cache key for @link AwsClientCache.
* AwsClientDetails contains the region only, since actually the region value may come from the FlowFile attributes.
*/
public class AwsClientDetails {
private Region region;
public AwsClientDetails(Region region) {
this.region = region;
}
public Region getRegion() {
return region;
}
public void setRegion(final Region region) {
this.region = region;
}
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final AwsClientDetails that = (AwsClientDetails) o;
return Objects.equals(region, that.region);
}
@Override
public int hashCode() {
return Objects.hash(region);
}
}

View File

@ -1,31 +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.v2;
import org.apache.nifi.processor.ProcessContext;
import software.amazon.awssdk.core.SdkClient;
public interface AwsClientProvider<T extends SdkClient> {
/**
* Creates an AWS client using process context and AWS client details.
*
* @param context process context
* @return AWS client
*/
T createClient(final ProcessContext context);
}

View File

@ -16,33 +16,12 @@
*/
package org.apache.nifi.processors.aws.wag;
import static org.apache.commons.lang3.StringUtils.trimToEmpty;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.http.AmazonHttpClient;
import com.amazonaws.http.HttpMethodName;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.InputStream;
import java.net.MalformedURLException;
import java.net.URI;
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.io.Charsets;
import org.apache.commons.lang3.StringUtils;
import org.apache.http.NameValuePair;
@ -50,7 +29,7 @@ import org.apache.http.client.utils.URLEncodedUtils;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.expression.AttributeExpression;
import org.apache.nifi.components.Validator;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
@ -60,18 +39,37 @@ import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayClient;
import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayClientBuilder;
import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayRequest;
import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayRequestBuilder;
import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayResponse;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.InputStream;
import java.net.MalformedURLException;
import java.net.URI;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import static org.apache.commons.lang3.StringUtils.trimToEmpty;
/**
* This class is the base class for invoking aws gateway api endpoints
*/
public abstract class AbstractAWSGatewayApiProcessor extends
AbstractAWSCredentialsProviderProcessor<GenericApiGatewayClient> {
public abstract class AbstractAWSGatewayApiProcessor extends AbstractAWSCredentialsProviderProcessor<GenericApiGatewayClient> {
private volatile Set<String> dynamicPropertyNames = new HashSet<>();
private volatile Pattern regexAttributesToSend = null;
@ -105,21 +103,23 @@ public abstract class AbstractAWSGatewayApiProcessor extends
// processing, including when converting http headers, copying attributes, etc.
// This set includes our strings defined above as well as some standard flowfile
// attributes.
public static final Set<String> IGNORED_ATTRIBUTES = Collections.unmodifiableSet(new HashSet<>(
Arrays.asList(STATUS_CODE, STATUS_MESSAGE, RESOURCE_NAME_ATTR, TRANSACTION_ID, "uuid",
"filename", "path")));
public static final Set<String> IGNORED_ATTRIBUTES = Set.of(STATUS_CODE,
STATUS_MESSAGE,
RESOURCE_NAME_ATTR,
TRANSACTION_ID,
CoreAttributes.UUID.key(),
CoreAttributes.FILENAME.key(),
CoreAttributes.PATH.key());
public static final PropertyDescriptor PROP_METHOD = new PropertyDescriptor.Builder()
.name("aws-gateway-http-method")
.displayName("HTTP Method")
.description(
"HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS)."
.description("HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS)."
+ "Methods other than POST, PUT and PATCH will be sent without a message body.")
.required(true)
.defaultValue("GET")
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators
.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PROP_AWS_API_KEY = new PropertyDescriptor.Builder()
@ -140,16 +140,6 @@ public abstract class AbstractAWSGatewayApiProcessor extends
.addValidator(StandardValidators.URL_VALIDATOR)
.build();
// we use our own region, because the way the base sets the region after the client is created
// resets the endpoint and breaks everything
public static final PropertyDescriptor PROP_AWS_GATEWAY_API_REGION = new PropertyDescriptor.Builder()
.name("aws-gateway-region")
.displayName("Amazon Region")
.required(true)
.allowableValues(AbstractAWSProcessor.getAvailableRegions())
.defaultValue(AbstractAWSProcessor.createAllowableValue(Regions.DEFAULT_REGION).getValue())
.build();
public static final PropertyDescriptor PROP_RESOURCE_NAME = new PropertyDescriptor.Builder()
.name("aws-gateway-resource")
.displayName("Amazon Gateway Api ResourceName")
@ -160,13 +150,11 @@ public abstract class AbstractAWSGatewayApiProcessor extends
public static final PropertyDescriptor PROP_QUERY_PARAMS = new PropertyDescriptor.Builder()
.name("aws-gateway-query-parameters")
.displayName("Query Parameters")
.description(
"The query parameters for this request in the form of Name=Value separated by &")
.description("The query parameters for this request in the form of Name=Value separated by &")
.displayName("Query Parameters")
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators
.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor PROP_ATTRIBUTES_TO_SEND = new PropertyDescriptor.Builder()
@ -184,19 +172,16 @@ public abstract class AbstractAWSGatewayApiProcessor extends
public static final PropertyDescriptor PROP_PUT_OUTPUT_IN_ATTRIBUTE = new PropertyDescriptor.Builder()
.name("aws-gateway-put-response-body-in-attribute")
.displayName("Put Response Body In Attribute")
.description(
"If set, the response body received back will be put into an attribute of the original FlowFile instead of a separate "
.description("If set, the response body received back will be put into an attribute of the original FlowFile instead of a separate "
+ "FlowFile. The attribute key to put to is determined by evaluating value of this property. ")
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
AttributeExpression.ResultType.STRING))
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final PropertyDescriptor PROP_OUTPUT_RESPONSE_REGARDLESS = new PropertyDescriptor.Builder()
.name("aws-gateway-always-output-response")
.displayName("Always Output Response")
.description(
"Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is "
.description("Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is "
+ "or if the processor is configured to put the server response body in the request attribute. In the later configuration a request FlowFile with the "
+ "response body in the attribute and a typical response FlowFile will be emitted to their respective relationships.")
.required(false)
@ -207,8 +192,7 @@ public abstract class AbstractAWSGatewayApiProcessor extends
public static final PropertyDescriptor PROP_PENALIZE_NO_RETRY = new PropertyDescriptor.Builder()
.name("aws-gateway-penalize-no-retry")
.displayName("Penalize on \"No Retry\"")
.description("Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" " +
"relationship.")
.description("Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.")
.required(false)
.defaultValue("false")
.allowableValues("true", "false")
@ -235,10 +219,9 @@ public abstract class AbstractAWSGatewayApiProcessor extends
+ "In the case of an empty value after evaluating an expression language expression, Content-Type defaults to "
+ DEFAULT_CONTENT_TYPE)
.required(true)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.defaultValue("${" + CoreAttributes.MIME_TYPE.key() + "}")
.addValidator(StandardValidators
.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
.addValidator(Validator.VALID)
.build();
public static final PropertyDescriptor PROP_SEND_BODY = new PropertyDescriptor.Builder()
@ -262,32 +245,13 @@ public abstract class AbstractAWSGatewayApiProcessor extends
.allowableValues("true", "false")
.build();
public static final PropertyDescriptor PROP_CONNECT_TIMEOUT = new PropertyDescriptor.Builder()
.name("aws-gateway-connection-timeout")
.displayName("Connection Timeout")
.description("Max wait time for connection to remote service.")
.required(false)
.defaultValue("10 secs")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();
public static final PropertyDescriptor PROP_READ_TIMEOUT = new PropertyDescriptor.Builder()
.name("aws-gateway-read-timeout")
.displayName("Read Timeout")
.description("Max wait time for response from remote service.")
.required(false)
.defaultValue("50 secs")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.required(false)
.name(propertyDescriptorName)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(
AttributeExpression.ResultType.STRING,
true))
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.dynamic(true)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES).build();
}
@ -335,22 +299,28 @@ public abstract class AbstractAWSGatewayApiProcessor extends
.getValue();
// user is not expected to encode, that will be done by the aws client
// but we may need to when validating
final String encodedInput = URLEncoder.encode(evaluatedInput, "UTF-8");
final String encodedInput = URLEncoder.encode(evaluatedInput, StandardCharsets.UTF_8);
final String url = String.format("http://www.foo.com?%s", encodedInput);
URI.create(url).toURL();
results.add(new ValidationResult.Builder().subject(PROP_QUERY_PARAMS.getName())
results.add(new ValidationResult.Builder()
.subject(PROP_QUERY_PARAMS.getName())
.input(input)
.explanation("Valid URL params")
.valid(true).build());
.valid(true)
.build());
} catch (final Exception e) {
results.add(new ValidationResult.Builder().subject(PROP_QUERY_PARAMS.getName())
.input(input).explanation(
"Not a valid set of URL params").valid(false).build());
results.add(new ValidationResult.Builder()
.subject(PROP_QUERY_PARAMS.getName())
.input(input)
.explanation("Not a valid set of URL params")
.valid(false)
.build());
}
}
}
final String method = trimToEmpty(validationContext.getProperty(PROP_METHOD).getValue())
.toUpperCase();
final String method = trimToEmpty(validationContext.getProperty(PROP_METHOD).getValue()).toUpperCase();
// if there are expressions do not validate
if (!(validationContext.isExpressionLanguageSupported(PROP_METHOD.getName())
@ -368,30 +338,19 @@ public abstract class AbstractAWSGatewayApiProcessor extends
}
@Override
protected GenericApiGatewayClient createClient(final ProcessContext context,
final AWSCredentialsProvider awsCredentialsProvider,
final ClientConfiguration clientConfiguration) {
protected GenericApiGatewayClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
GenericApiGatewayClientBuilder builder = new GenericApiGatewayClientBuilder()
.withCredentials(awsCredentialsProvider).withClientConfiguration(clientConfiguration)
.withEndpoint(context.getProperty(PROP_AWS_GATEWAY_API_ENDPOINT).getValue()).withRegion(
Region.getRegion(
Regions.fromName(context.getProperty(PROP_AWS_GATEWAY_API_REGION).getValue())));
if (context.getProperty(PROP_AWS_API_KEY).isSet()) {
builder = builder.withApiKey(context.getProperty(PROP_AWS_API_KEY).evaluateAttributeExpressions().getValue());
}
.withCredentials(credentialsProvider).withClientConfiguration(config)
.withEndpoint(context.getProperty(PROP_AWS_GATEWAY_API_ENDPOINT).getValue())
.withRegion(region)
.withApiKey(context.getProperty(PROP_AWS_API_KEY).evaluateAttributeExpressions().getValue());
if (providedClient != null) {
builder = builder.withHttpClient(providedClient);
}
return builder.build();
}
@Override
@Deprecated
protected GenericApiGatewayClient createClient(final ProcessContext context,
final AWSCredentials credentials,
final ClientConfiguration clientConfiguration) {
return createClient(context, new AWSStaticCredentialsProvider(credentials), clientConfiguration);
return builder.build();
}
protected GenericApiGatewayRequest configureRequest(final ProcessContext context,
@ -499,8 +458,7 @@ public abstract class AbstractAWSGatewayApiProcessor extends
}
}
String contentType = context.getProperty(PROP_CONTENT_TYPE)
.evaluateAttributeExpressions(requestAttributes).getValue();
String contentType = context.getProperty(PROP_CONTENT_TYPE).evaluateAttributeExpressions(requestAttributes).getValue();
final boolean sendBody = context.getProperty(PROP_SEND_BODY).asBoolean();
contentType = StringUtils.isBlank(contentType) ? DEFAULT_CONTENT_TYPE : contentType;
if (methodName == HttpMethodName.PUT || methodName == HttpMethodName.POST
@ -557,21 +515,23 @@ public abstract class AbstractAWSGatewayApiProcessor extends
protected Map<String, String> convertAttributesFromHeaders(final GenericApiGatewayResponse responseHttp) {
// create a new hashmap to store the values from the connection
final Map<String, String> map = new HashMap<>();
responseHttp.getHttpResponse().getHeaders().entrySet().forEach((entry) -> {
final String key = entry.getKey();
final String value = entry.getValue();
responseHttp.getHttpResponse().getAllHeaders().forEach((key, headers) -> {
if (key == null) {
return;
}
final String joined = headers.stream()
.map(String::trim)
.filter(str -> !str.isEmpty())
.collect(Collectors.joining(","));
// we ignore any headers with no actual values (rare)
if (StringUtils.isBlank(value)) {
if (StringUtils.isBlank(joined)) {
return;
}
// put the csv into the map
map.put(key, value);
map.put(key, joined);
});
return map;
@ -647,17 +607,17 @@ public abstract class AbstractAWSGatewayApiProcessor extends
protected void logResponse(final ComponentLog logger, final GenericApiGatewayResponse response) {
try {
logger.debug("\nResponse from remote service:\n\t{}\n{}",
new Object[]{response.getHttpResponse().getHttpRequest().getURI().toURL().toExternalForm(), getLogString(response.getHttpResponse().getHeaders())});
new Object[]{response.getHttpResponse().getHttpRequest().getURI().toURL().toExternalForm(), getLogString(response.getHttpResponse().getAllHeaders())});
} catch (MalformedURLException e) {
logger.debug(e.getMessage());
}
}
protected String getLogString(final Map<String, String> map) {
protected String getLogString(final Map<String, ?> map) {
final StringBuilder sb = new StringBuilder();
if (map != null && map.size() > 0) {
for (Map.Entry<String, String> entry : map.entrySet()) {
String value = entry.getValue();
for (Map.Entry<String, ?> entry : map.entrySet()) {
final Object value = entry.getValue();
sb.append("\t");
sb.append(entry.getKey());
sb.append(": ");

View File

@ -2,8 +2,10 @@ package org.apache.nifi.processors.aws.wag.client;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.AmazonWebServiceResponse;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.DefaultRequest;
import com.amazonaws.Response;
import com.amazonaws.auth.AWS4Signer;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.http.AmazonHttpClient;
@ -12,6 +14,7 @@ import com.amazonaws.http.HttpMethodName;
import com.amazonaws.http.HttpResponseHandler;
import com.amazonaws.http.JsonResponseHandler;
import com.amazonaws.internal.auth.DefaultSignerProvider;
import com.amazonaws.protocol.json.JsonErrorResponseMetadata;
import com.amazonaws.protocol.json.JsonOperationMetadata;
import com.amazonaws.protocol.json.SdkStructuredPlainJsonFactory;
import com.amazonaws.regions.Region;
@ -19,9 +22,9 @@ import com.amazonaws.transform.JsonErrorUnmarshaller;
import com.amazonaws.transform.JsonUnmarshallerContext;
import com.amazonaws.transform.Unmarshaller;
import com.fasterxml.jackson.databind.JsonNode;
import java.io.InputStream;
import java.net.URI;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -35,12 +38,14 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
private final AWSCredentialsProvider credentials;
private final String apiKey;
private final AWS4Signer signer;
private final URI endpoint;
private final String region;
GenericApiGatewayClient(ClientConfiguration clientConfiguration, String endpoint, Region region,
AWSCredentialsProvider credentials, String apiKey, AmazonHttpClient httpClient) {
super(clientConfiguration);
setRegion(region);
setEndpoint(endpoint);
this.endpoint = URI.create(endpoint);
this.region = region.getName();
this.credentials = credentials;
this.apiKey = apiKey;
this.signer = new AWS4Signer();
@ -50,14 +55,16 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
final JsonOperationMetadata metadata = new JsonOperationMetadata().withHasStreamingSuccessResponse(false).withPayloadJson(false);
final Unmarshaller<GenericApiGatewayResponse, JsonUnmarshallerContext> responseUnmarshaller = in -> new GenericApiGatewayResponse(in.getHttpResponse());
this.responseHandler = SdkStructuredPlainJsonFactory.SDK_JSON_FACTORY.createResponseHandler(metadata, responseUnmarshaller);
JsonErrorUnmarshaller defaultErrorUnmarshaller = new JsonErrorUnmarshaller(GenericApiGatewayException.class, null) {
final JsonErrorResponseMetadata errorResponseMetadata = new JsonErrorResponseMetadata();
final JsonErrorUnmarshaller defaultErrorUnmarshaller = new JsonErrorUnmarshaller(GenericApiGatewayException.class, null) {
@Override
public AmazonServiceException unmarshall(JsonNode jsonContent) throws Exception {
public AmazonServiceException unmarshall(final JsonNode jsonContent) {
return new GenericApiGatewayException(jsonContent.toString());
}
};
this.errorResponseHandler = SdkStructuredPlainJsonFactory.SDK_JSON_FACTORY.createErrorResponseHandler(
Collections.singletonList(defaultErrorUnmarshaller), null);
this.errorResponseHandler = SdkStructuredPlainJsonFactory.SDK_JSON_FACTORY.createErrorResponseHandler(errorResponseMetadata, List.of(defaultErrorUnmarshaller));
if (httpClient != null) {
super.client = httpClient;
@ -80,18 +87,25 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
if (parameters != null) {
request.setParameters(parameters);
}
return this.client.execute(request, responseHandler, errorResponseHandler, executionContext).getAwsResponse();
final Response<AmazonWebServiceResponse<GenericApiGatewayResponse>> response = client.requestExecutionBuilder()
.request(request)
.errorResponseHandler(errorResponseHandler)
.executionContext(executionContext)
.execute(responseHandler);
return response.getAwsResponse().getResult();
}
private ExecutionContext buildExecutionContext() {
final ExecutionContext executionContext = ExecutionContext.builder().withSignerProvider(
new DefaultSignerProvider(this, signer)).build();
final ExecutionContext executionContext = ExecutionContext.builder()
.withSignerProvider(new DefaultSignerProvider(this, signer))
.build();
executionContext.setCredentialsProvider(credentials);
executionContext.setSigner(signer);
return executionContext;
}
private Map<String, String> buildRequestHeaders(Map<String, String> headers, String apiKey) {
private Map<String, String> buildRequestHeaders(Map<String, String> headers, final String apiKey) {
if (headers == null) {
headers = new HashMap<>();
}
@ -105,6 +119,10 @@ public class GenericApiGatewayClient extends AmazonWebServiceClient {
return this.endpoint;
}
public String getRegion() {
return region;
}
@Override
protected String getServiceNameIntern() {
return API_GATEWAY_SERVICE_NAME;

View File

@ -34,6 +34,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
import org.apache.http.conn.ssl.DefaultHostnameVerifier;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.AllowableValue;
@ -52,8 +53,6 @@ import org.apache.nifi.ssl.SSLContextService;
import javax.net.ssl.SSLContext;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -119,13 +118,13 @@ public class AwsSecretsManagerParameterProvider extends AbstractParameterProvide
private static final String DEFAULT_USER_AGENT = "NiFi";
private static final Protocol DEFAULT_PROTOCOL = Protocol.HTTPS;
private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
private static final List<PropertyDescriptor> PROPERTIES = List.of(
SECRET_NAME_PATTERN,
REGION,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TIMEOUT,
SSL_CONTEXT_SERVICE
));
);
private final ObjectMapper objectMapper = new ObjectMapper();
@ -249,7 +248,7 @@ public class AwsSecretsManagerParameterProvider extends AbstractParameterProvide
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
if (sslContextService != null) {
final SSLContext sslContext = sslContextService.createContext();
SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, SdkTLSSocketFactory.BROWSER_COMPATIBLE_HOSTNAME_VERIFIER);
SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
}

View File

@ -141,6 +141,12 @@
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-textract</artifactId>
</dependency>
<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>localstack</artifactId>
<version>1.19.1</version>
<scope>test</scope>
</dependency>
</dependencies>
<build>

View File

@ -17,8 +17,8 @@
package org.apache.nifi.processors.aws.credentials.provider.factory.strategies;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.internal.StaticCredentialsProvider;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
@ -36,22 +36,22 @@ public class AccessKeyPairCredentialsStrategy extends AbstractCredentialsStrateg
public AccessKeyPairCredentialsStrategy() {
super("Access Key Pair", new PropertyDescriptor[] {
CredentialPropertyDescriptors.ACCESS_KEY,
CredentialPropertyDescriptors.ACCESS_KEY_ID,
CredentialPropertyDescriptors.SECRET_KEY
});
}
@Override
public AWSCredentialsProvider getCredentialsProvider(final PropertyContext propertyContext) {
final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY).evaluateAttributeExpressions().getValue();
final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID).evaluateAttributeExpressions().getValue();
final String secretKey = propertyContext.getProperty(CredentialPropertyDescriptors.SECRET_KEY).evaluateAttributeExpressions().getValue();
final BasicAWSCredentials credentials = new BasicAWSCredentials(accessKey, secretKey);
return new StaticCredentialsProvider(credentials);
return new AWSStaticCredentialsProvider(credentials);
}
@Override
public AwsCredentialsProvider getAwsCredentialsProvider(final PropertyContext propertyContext) {
final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY).evaluateAttributeExpressions().getValue();
final String accessKey = propertyContext.getProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID).evaluateAttributeExpressions().getValue();
final String secretKey = propertyContext.getProperty(CredentialPropertyDescriptors.SECRET_KEY).evaluateAttributeExpressions().getValue();
return software.amazon.awssdk.auth.credentials.StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, secretKey));
}

View File

@ -17,8 +17,8 @@
package org.apache.nifi.processors.aws.credentials.provider.factory.strategies;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.auth.AnonymousAWSCredentials;
import com.amazonaws.internal.StaticCredentialsProvider;
import org.apache.nifi.context.PropertyContext;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
@ -40,7 +40,7 @@ public class AnonymousCredentialsStrategy extends AbstractBooleanCredentialsStra
@Override
public AWSCredentialsProvider getCredentialsProvider(final PropertyContext propertyContext) {
AnonymousAWSCredentials credentials = new AnonymousAWSCredentials();
return new StaticCredentialsProvider(credentials);
return new AWSStaticCredentialsProvider(credentials);
}
@Override

View File

@ -19,7 +19,9 @@ package org.apache.nifi.processors.aws.credentials.provider.factory.strategies;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient;
import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
@ -150,7 +152,6 @@ public class AssumeRoleCredentialsStrategy extends AbstractCredentialsStrategy {
final String assumeRoleSTSSigner = propertyContext.getProperty(ASSUME_ROLE_STS_SIGNER_OVERRIDE).getValue();
final SSLContextService sslContextService = propertyContext.getProperty(ASSUME_ROLE_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
STSAssumeRoleSessionCredentialsProvider.Builder builder;
final ClientConfiguration config = new ClientConfiguration();
if (sslContextService != null) {
@ -175,26 +176,25 @@ public class AssumeRoleCredentialsStrategy extends AbstractCredentialsStrategy {
config.withSignerOverride(assumeRoleSTSSigner);
}
AWSSecurityTokenServiceClient securityTokenService = new AWSSecurityTokenServiceClient(primaryCredentialsProvider, config);
AWSSecurityTokenServiceClientBuilder securityTokenServiceBuilder = AWSSecurityTokenServiceClient.builder()
.withCredentials(primaryCredentialsProvider)
.withRegion(assumeRoleSTSRegion)
.withClientConfiguration(config);
if (assumeRoleSTSEndpoint != null && !assumeRoleSTSEndpoint.isEmpty()) {
if (assumeRoleSTSSignerType == CUSTOM_SIGNER) {
securityTokenService.setEndpoint(assumeRoleSTSEndpoint, securityTokenService.getServiceName(), assumeRoleSTSRegion);
} else {
securityTokenService.setEndpoint(assumeRoleSTSEndpoint);
}
AwsClientBuilder.EndpointConfiguration endpointConfiguration = new AwsClientBuilder.EndpointConfiguration(assumeRoleSTSEndpoint, assumeRoleSTSRegion);
securityTokenServiceBuilder.withEndpointConfiguration(endpointConfiguration);
}
builder = new STSAssumeRoleSessionCredentialsProvider
.Builder(assumeRoleArn, assumeRoleName)
.withStsClient(securityTokenService)
STSAssumeRoleSessionCredentialsProvider.Builder builder = new STSAssumeRoleSessionCredentialsProvider.Builder(assumeRoleArn, assumeRoleName)
.withStsClient(securityTokenServiceBuilder.build())
.withRoleSessionDurationSeconds(maxSessionTime);
if (assumeRoleExternalId != null && !assumeRoleExternalId.isEmpty()) {
builder = builder.withExternalId(assumeRoleExternalId);
builder.withExternalId(assumeRoleExternalId);
}
final AWSCredentialsProvider credsProvider = builder.build();
return credsProvider;
}

View File

@ -38,7 +38,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ACCESS_KEY;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ACCESS_KEY_ID;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_EXTERNAL_ID;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_PROXY_HOST;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_PROXY_PORT;
@ -83,7 +83,7 @@ public class AWSCredentialsProviderControllerService extends AbstractControllerS
static {
final List<PropertyDescriptor> props = new ArrayList<>();
props.add(USE_DEFAULT_CREDENTIALS);
props.add(ACCESS_KEY);
props.add(ACCESS_KEY_ID);
props.add(SECRET_KEY);
props.add(CREDENTIALS_FILE);
props.add(PROFILE_NAME);

View File

@ -16,12 +16,14 @@
*/
package org.apache.nifi.processors.aws.dynamodb;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
import com.amazonaws.services.dynamodbv2.model.AttributeValue;
import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult;
import com.amazonaws.services.dynamodbv2.model.WriteRequest;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@ -38,13 +40,11 @@ import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
import com.amazonaws.services.dynamodbv2.model.AttributeValue;
import com.amazonaws.services.dynamodbv2.model.WriteRequest;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@SupportsBatching
@SeeAlso({GetDynamoDB.class, PutDynamoDB.class, PutDynamoDBRecord.class})
@ -69,7 +69,7 @@ import com.amazonaws.services.dynamodbv2.model.WriteRequest;
@ReadsAttribute(attribute = AbstractDynamoDBProcessor.DYNAMODB_ITEM_HASH_KEY_VALUE, description = "Items hash key value" ),
@ReadsAttribute(attribute = AbstractDynamoDBProcessor.DYNAMODB_ITEM_RANGE_KEY_VALUE, description = "Items range key value" ),
})
public class DeleteDynamoDB extends AbstractWriteDynamoDBProcessor {
public class DeleteDynamoDB extends AbstractDynamoDBProcessor {
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(TABLE, HASH_KEY_NAME, RANGE_KEY_NAME, HASH_KEY_VALUE, RANGE_KEY_VALUE,
@ -128,10 +128,20 @@ public class DeleteDynamoDB extends AbstractWriteDynamoDBProcessor {
final DynamoDB dynamoDB = getDynamoDB(context);
try {
BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
final BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
final BatchWriteItemResult result = outcome.getBatchWriteItemResult();
handleUnprocessedItems(session, keysToFlowFileMap, table, hashKeyName, hashKeyValueType, rangeKeyName,
rangeKeyValueType, outcome);
// Handle unprocessed items
final List<WriteRequest> unprocessedItems = result.getUnprocessedItems().get(table);
if (unprocessedItems != null && unprocessedItems.size() > 0) {
for (final WriteRequest request : unprocessedItems) {
final Map<String, AttributeValue> item = request.getDeleteRequest().getKey();
final Object hashKeyValue = getValue(item, hashKeyName, hashKeyValueType);
final Object rangeKeyValue = getValue(item, rangeKeyName, rangeKeyValueType);
sendUnprocessedToUnprocessedRelationship(session, keysToFlowFileMap, hashKeyValue, rangeKeyValue);
}
}
// All non unprocessed items are successful
for (FlowFile flowFile : keysToFlowFileMap.values()) {
@ -153,10 +163,4 @@ public class DeleteDynamoDB extends AbstractWriteDynamoDBProcessor {
}
}
/**
* {@inheritDoc}
*/
protected Map<String, AttributeValue> getRequestItem(WriteRequest writeRequest) {
return writeRequest.getDeleteRequest().getKey();
}
}

View File

@ -16,22 +16,24 @@
*/
package org.apache.nifi.processors.aws.dynamodb;
import java.io.ByteArrayOutputStream;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.document.Item;
import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
import com.amazonaws.services.dynamodbv2.model.AttributeValue;
import com.amazonaws.services.dynamodbv2.model.BatchWriteItemResult;
import com.amazonaws.services.dynamodbv2.model.WriteRequest;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.ReadsAttribute;
import org.apache.nifi.annotation.behavior.ReadsAttributes;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.behavior.SystemResource;
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
@ -42,14 +44,12 @@ import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.dynamodbv2.document.BatchWriteItemOutcome;
import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.document.Item;
import com.amazonaws.services.dynamodbv2.document.TableWriteItems;
import com.amazonaws.services.dynamodbv2.model.AttributeValue;
import com.amazonaws.services.dynamodbv2.model.WriteRequest;
import java.io.ByteArrayOutputStream;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@SupportsBatching
@SeeAlso({DeleteDynamoDB.class, GetDynamoDB.class, PutDynamoDBRecord.class})
@ -78,7 +78,7 @@ import com.amazonaws.services.dynamodbv2.model.WriteRequest;
@ReadsAttribute(attribute = AbstractDynamoDBProcessor.DYNAMODB_ITEM_RANGE_KEY_VALUE, description = "Items range key value")
})
@SystemResourceConsideration(resource = SystemResource.MEMORY)
public class PutDynamoDB extends AbstractWriteDynamoDBProcessor {
public class PutDynamoDB extends AbstractDynamoDBProcessor {
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(TABLE, HASH_KEY_NAME, RANGE_KEY_NAME, HASH_KEY_VALUE, RANGE_KEY_VALUE,
@ -155,10 +155,20 @@ public class PutDynamoDB extends AbstractWriteDynamoDBProcessor {
final DynamoDB dynamoDB = getDynamoDB(context);
try {
BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
final BatchWriteItemOutcome outcome = dynamoDB.batchWriteItem(tableWriteItems);
final BatchWriteItemResult result = outcome.getBatchWriteItemResult();
handleUnprocessedItems(session, keysToFlowFileMap, table, hashKeyName, hashKeyValueType, rangeKeyName,
rangeKeyValueType, outcome);
// Handle unprocessed items
final List<WriteRequest> unprocessedItems = result.getUnprocessedItems().get(table);
if (unprocessedItems != null && unprocessedItems.size() > 0) {
for (final WriteRequest request : unprocessedItems) {
final Map<String, AttributeValue> item = request.getPutRequest().getItem();
final Object hashKeyValue = getValue(item, hashKeyName, hashKeyValueType);
final Object rangeKeyValue = getValue(item, rangeKeyName, rangeKeyValueType);
sendUnprocessedToUnprocessedRelationship(session, keysToFlowFileMap, hashKeyValue, rangeKeyValue);
}
}
// Handle any remaining flowfiles
for (FlowFile flowFile : keysToFlowFileMap.values()) {
@ -184,11 +194,4 @@ public class PutDynamoDB extends AbstractWriteDynamoDBProcessor {
return (flowFile.getSize() + jsonDocument.length()) < DYNAMODB_MAX_ITEM_SIZE;
}
/**
* {@inheritDoc}
*/
protected Map<String, AttributeValue> getRequestItem(WriteRequest writeRequest) {
return writeRequest.getPutRequest().getItem();
}
}

View File

@ -25,21 +25,23 @@ import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
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 org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.kinesis.KinesisProcessorUtils;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.core.SdkBytes;
import software.amazon.awssdk.services.firehose.FirehoseClient;
import software.amazon.awssdk.services.firehose.FirehoseClientBuilder;
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;
@ -53,37 +55,67 @@ import java.util.Map;
@WritesAttribute(attribute = "aws.kinesis.firehose.error.message", description = "Error message on posting message to AWS Kinesis Firehose"),
@WritesAttribute(attribute = "aws.kinesis.firehose.error.code", description = "Error code for the message when posting to AWS Kinesis Firehose"),
@WritesAttribute(attribute = "aws.kinesis.firehose.record.id", description = "Record id of the message posted to Kinesis Firehose")})
public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
public class PutKinesisFirehose extends AbstractAwsSyncProcessor<FirehoseClient, FirehoseClientBuilder> {
/**
* Kinesis put record response error message
*/
public static final String AWS_KINESIS_FIREHOSE_ERROR_MESSAGE = "aws.kinesis.firehose.error.message";
/**
* Kinesis put record response error code
*/
public static final String AWS_KINESIS_FIREHOSE_ERROR_CODE = "aws.kinesis.firehose.error.code";
/**
* Kinesis put record response record id
*/
public static final String AWS_KINESIS_FIREHOSE_RECORD_ID = "aws.kinesis.firehose.record.id";
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, BATCH_SIZE, MAX_MESSAGE_BUFFER_SIZE_MB, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD, ENDPOINT_OVERRIDE));
public static final PropertyDescriptor KINESIS_FIREHOSE_DELIVERY_STREAM_NAME = new PropertyDescriptor.Builder()
.name("Amazon Kinesis Firehose Delivery Stream Name")
.description("The name of kinesis firehose delivery stream")
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
/**
* Max buffer size 1 MB
*/
public static final int MAX_MESSAGE_SIZE = 1000 * 1024;
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")
.description("Batch size for messages (1-500).")
.defaultValue("250")
.required(false)
.addValidator(StandardValidators.createLongValidator(1, 500, true))
.sensitive(false)
.build();
public static final PropertyDescriptor MAX_MESSAGE_BUFFER_SIZE_MB = new PropertyDescriptor.Builder()
.name("Max message buffer size")
.description("Max message buffer")
.defaultValue("1 MB")
.required(false)
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
.sensitive(false)
.build();
private static final List<PropertyDescriptor> properties = List.of(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME,
BATCH_SIZE,
MAX_MESSAGE_BUFFER_SIZE_MB,
REGION,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TIMEOUT,
PROXY_CONFIGURATION_SERVICE,
PROXY_HOST,
PROXY_HOST_PORT,
PROXY_USERNAME,
PROXY_PASSWORD,
ENDPOINT_OVERRIDE);
public static final int MAX_MESSAGE_SIZE = KinesisProcessorUtils.MAX_MESSAGE_SIZE;
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
}
@Override
protected FirehoseClientBuilder createClientBuilder(final ProcessContext context) {
return FirehoseClient.builder();
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
@ -101,27 +133,19 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
final List<FlowFile> successfulFlowFiles = new ArrayList<>();
// Prepare batch of records
for (int i = 0; i < flowFiles.size(); i++) {
FlowFile flowFile = flowFiles.get(i);
for (final FlowFile flowFile : flowFiles) {
final String firehoseStreamName = context.getProperty(KINESIS_FIREHOSE_DELIVERY_STREAM_NAME).evaluateAttributeExpressions(flowFile).getValue();
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<>());
recordHash.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>());
final List<FlowFile> flowFilesForStream = hashFlowFiles.computeIfAbsent(firehoseStreamName, k -> new ArrayList<>());
flowFilesForStream.add(flowFile);
}
if (hashFlowFiles.containsKey(firehoseStreamName) == false) {
hashFlowFiles.put(firehoseStreamName, new ArrayList<>());
}
hashFlowFiles.get(firehoseStreamName).add(flowFile);
}
for (final Map.Entry<String, List<Record>> entryRecord : recordHash.entrySet()) {
final String streamName = entryRecord.getKey();
final List<Record> records = entryRecord.getValue();
for (final Map.Entry<String, List<Record>> entry : recordHash.entrySet()) {
final String streamName = entry.getKey();
final List<Record> records = entry.getValue();
if (records.size() > 0) {
// Send the batch
@ -135,15 +159,15 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
final List<PutRecordBatchResponseEntry> responseEntries = response.requestResponses();
for (int i = 0; i < responseEntries.size(); i++ ) {
final PutRecordBatchResponseEntry entry = responseEntries.get(i);
final PutRecordBatchResponseEntry responseEntry = responseEntries.get(i);
FlowFile flowFile = hashFlowFiles.get(streamName).get(i);
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());
attributes.put(AWS_KINESIS_FIREHOSE_RECORD_ID, responseEntry.recordId());
flowFile = session.putAttribute(flowFile, AWS_KINESIS_FIREHOSE_RECORD_ID, responseEntry.recordId());
if (StringUtils.isNotBlank(responseEntry.errorCode())) {
attributes.put(AWS_KINESIS_FIREHOSE_ERROR_CODE, responseEntry.errorCode());
attributes.put(AWS_KINESIS_FIREHOSE_ERROR_MESSAGE, responseEntry.errorMessage());
flowFile = session.putAllAttributes(flowFile, attributes);
failedFlowFiles.add(flowFile);
} else {
@ -158,15 +182,14 @@ public class PutKinesisFirehose extends AbstractKinesisFirehoseProcessor {
if (failedFlowFiles.size() > 0) {
session.transfer(failedFlowFiles, REL_FAILURE);
getLogger().error("Failed to publish to kinesis firehose {}", new Object[]{failedFlowFiles});
getLogger().error("Failed to publish to kinesis firehose {}", failedFlowFiles);
}
if (successfulFlowFiles.size() > 0) {
session.transfer(successfulFlowFiles, REL_SUCCESS);
getLogger().info("Successfully published to kinesis firehose {}", new Object[]{successfulFlowFiles});
getLogger().info("Successfully published to kinesis firehose {}", successfulFlowFiles);
}
} catch (final Exception exception) {
getLogger().error("Failed to publish to kinesis firehose {} with exception {}", new Object[]{flowFiles, exception});
getLogger().error("Failed to publish to kinesis firehose {} with exception {}", flowFiles, exception);
session.transfer(flowFiles, REL_FAILURE);
context.yield();
}

View File

@ -50,6 +50,7 @@ import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.kinesis.stream.record.AbstractKinesisRecordProcessor;
import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRaw;
import org.apache.nifi.processors.aws.kinesis.stream.record.KinesisRecordProcessorRecord;
import org.apache.nifi.processors.aws.v2.AbstractAwsAsyncProcessor;
import org.apache.nifi.processors.aws.v2.AbstractAwsProcessor;
import org.apache.nifi.serialization.RecordReaderFactory;
import org.apache.nifi.serialization.RecordSetWriterFactory;
@ -60,6 +61,7 @@ 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.awssdk.services.kinesis.KinesisAsyncClientBuilder;
import software.amazon.kinesis.checkpoint.CheckpointConfig;
import software.amazon.kinesis.common.ConfigsBuilder;
import software.amazon.kinesis.common.InitialPositionInStream;
@ -142,7 +144,8 @@ 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 AbstractKinesisStreamAsyncProcessor {
public class ConsumeKinesisStream extends AbstractAwsAsyncProcessor<KinesisAsyncClient, KinesisAsyncClientBuilder> {
private static final String CHECKPOINT_CONFIG = "checkpointConfig";
private static final String COORDINATOR_CONFIG = "coordinatorConfig";
private static final String LEASE_MANAGEMENT_CONFIG = "leaseManagementConfig";
@ -166,6 +169,14 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
InitialPositionInStream.AT_TIMESTAMP.toString(), "Start reading from the position denoted by a specific time stamp, provided in the value Timestamp."
);
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();
public static final PropertyDescriptor APPLICATION_NAME = new PropertyDescriptor.Builder()
.displayName("Application Name")
.name("amazon-kinesis-stream-application-name")
@ -305,21 +316,35 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
" the contents of the message will be routed to this Relationship as its own individual FlowFile.")
.build();
public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = Collections.unmodifiableList(
Arrays.asList(
public static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS = List.of(
// Kinesis Stream specific properties
KINESIS_STREAM_NAME, APPLICATION_NAME, RECORD_READER, RECORD_WRITER, REGION, ENDPOINT_OVERRIDE,
DYNAMODB_ENDPOINT_OVERRIDE, INITIAL_STREAM_POSITION, STREAM_POSITION_TIMESTAMP, TIMESTAMP_FORMAT,
FAILOVER_TIMEOUT, GRACEFUL_SHUTDOWN_TIMEOUT, CHECKPOINT_INTERVAL, NUM_RETRIES, RETRY_WAIT, REPORT_CLOUDWATCH_METRICS,
KINESIS_STREAM_NAME,
APPLICATION_NAME,
RECORD_READER,
RECORD_WRITER,
REGION,
ENDPOINT_OVERRIDE,
DYNAMODB_ENDPOINT_OVERRIDE,
INITIAL_STREAM_POSITION,
STREAM_POSITION_TIMESTAMP,
TIMESTAMP_FORMAT,
FAILOVER_TIMEOUT,
GRACEFUL_SHUTDOWN_TIMEOUT,
CHECKPOINT_INTERVAL,
NUM_RETRIES,
RETRY_WAIT,
REPORT_CLOUDWATCH_METRICS,
// generic AWS processor properties
TIMEOUT, AWS_CREDENTIALS_PROVIDER_SERVICE, PROXY_CONFIGURATION_SERVICE
)
TIMEOUT,
AWS_CREDENTIALS_PROVIDER_SERVICE,
PROXY_CONFIGURATION_SERVICE
);
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 Map<String, PropertyDescriptor> DISALLOWED_DYNAMIC_KCL_PROPERTIES = Map.of(
"leaseManagementConfig.initialPositionInStream", INITIAL_STREAM_POSITION,
"leaseManagementConfig.failoverTimeMillis", FAILOVER_TIMEOUT
);
private static final Object WORKER_LOCK = new Object();
private static final String SCHEDULER_THREAD_NAME_TEMPLATE = ConsumeKinesisStream.class.getSimpleName() + "-" + Scheduler.class.getSimpleName() + "-";
@ -459,11 +484,11 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
private ValidationResult buildDynamicPropertyBeanValidationResult(final ValidationResult.Builder validationResult,
final String subject, final String input, final String message) {
return validationResult
.explanation(
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();
.input(input)
.subject(subject)
.explanation("Kinesis Client Configuration Builder property with name %s cannot be used with value \"%s\" : %s".formatted(StringUtils.capitalize(subject), input, message))
.valid(false)
.build();
}
@Override
@ -702,13 +727,13 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
private CloudWatchAsyncClient getCloudwatchClient(final ProcessContext context) {
final CloudWatchAsyncClientBuilder builder = CloudWatchAsyncClient.builder();
configureClientBuilder(builder, context, null);
configureClientBuilder(builder, getRegion(context), context, null);
return builder.build();
}
private DynamoDbAsyncClient getDynamoClient(final ProcessContext context) {
final DynamoDbAsyncClientBuilder dynamoClientBuilder = DynamoDbAsyncClient.builder();
configureClientBuilder(dynamoClientBuilder, context, DYNAMODB_ENDPOINT_OVERRIDE);
configureClientBuilder(dynamoClientBuilder, getRegion(context), context, DYNAMODB_ENDPOINT_OVERRIDE);
return dynamoClientBuilder.build();
}
@ -807,4 +832,9 @@ public class ConsumeKinesisStream extends AbstractKinesisStreamAsyncProcessor {
.toInstant().toEpochMilli() // convert to epoch milliseconds for creating Date
);
}
@Override
protected KinesisAsyncClientBuilder createClientBuilder(final ProcessContext context) {
return KinesisAsyncClient.builder();
}
}

View File

@ -33,8 +33,10 @@ 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 org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.core.SdkBytes;
import software.amazon.awssdk.services.kinesis.KinesisClient;
import software.amazon.awssdk.services.kinesis.KinesisClientBuilder;
import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest;
import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
import software.amazon.awssdk.services.kinesis.model.PutRecordsResponse;
@ -42,8 +44,6 @@ import software.amazon.awssdk.services.kinesis.model.PutRecordsResultEntry;
import java.io.ByteArrayOutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -60,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 AbstractKinesisStreamSyncProcessor {
public class PutKinesisStream extends AbstractAwsSyncProcessor<KinesisClient, KinesisClientBuilder> {
/**
* Kinesis put record response error message
*/
@ -104,14 +104,32 @@ public class PutKinesisStream extends AbstractKinesisStreamSyncProcessor {
.sensitive(false)
.build();
public static final PropertyDescriptor KINESIS_STREAM_NAME = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(AbstractKinesisStreamSyncProcessor.KINESIS_STREAM_NAME)
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)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(KINESIS_STREAM_NAME, KINESIS_PARTITION_KEY, BATCH_SIZE, MAX_MESSAGE_BUFFER_SIZE_MB, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT, PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD, ENDPOINT_OVERRIDE));
public static final List<PropertyDescriptor> properties = List.of(
KINESIS_STREAM_NAME,
KINESIS_PARTITION_KEY,
BATCH_SIZE,
MAX_MESSAGE_BUFFER_SIZE_MB,
REGION,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TIMEOUT,
PROXY_CONFIGURATION_SERVICE,
PROXY_HOST,
PROXY_HOST_PORT,
PROXY_USERNAME,
PROXY_PASSWORD,
ENDPOINT_OVERRIDE);
/** A random number generator for cases where partition key is not available */
protected Random randomPartitionKeyGenerator = new Random();
@ -206,4 +224,10 @@ public class PutKinesisStream extends AbstractKinesisStreamSyncProcessor {
context.yield();
}
}
@Override
protected KinesisClientBuilder createClientBuilder(final ProcessContext context) {
return KinesisClient.builder();
}
}

View File

@ -16,29 +16,11 @@
*/
package org.apache.nifi.processors.aws.lambda;
import java.io.ByteArrayOutputStream;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.lambda.AWSLambdaClient;
import com.amazonaws.services.lambda.model.InvalidParameterValueException;
import com.amazonaws.services.lambda.model.InvalidRequestContentException;
@ -51,6 +33,28 @@ import com.amazonaws.services.lambda.model.ResourceNotFoundException;
import com.amazonaws.services.lambda.model.TooManyRequestsException;
import com.amazonaws.services.lambda.model.UnsupportedMediaTypeException;
import com.amazonaws.util.Base64;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
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.AbstractAWSCredentialsProviderProcessor;
import java.io.ByteArrayOutputStream;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
@InputRequirement(Requirement.INPUT_REQUIRED)
@Tags({"amazon", "aws", "lambda", "put"})
@ -69,66 +73,52 @@ import com.amazonaws.util.Base64;
@WritesAttribute(attribute = "aws.lambda.exception.status.code", description = "Exception status code on invoking from AWS Lambda"),
@WritesAttribute(attribute = "aws.lambda.exception.error.type", description = "Exception error type on invoking from AWS Lambda")
})
public class PutLambda extends AbstractAWSLambdaProcessor {
/**
* Lambda result function error message
*/
public class PutLambda extends AbstractAWSCredentialsProviderProcessor<AWSLambdaClient> {
public static final String AWS_LAMBDA_RESULT_FUNCTION_ERROR = "aws.lambda.result.function.error";
/**
* Lambda response status code
*/
public static final String AWS_LAMBDA_RESULT_STATUS_CODE = "aws.lambda.result.status.code";
/**
* Lambda response log tail (4kb)
*/
public static final String AWS_LAMBDA_RESULT_LOG = "aws.lambda.result.log";
/**
* Lambda payload in response
*/
public static final String AWS_LAMBDA_RESULT_PAYLOAD = "aws.lambda.result.payload";
/**
* Lambda exception field
*/
public static final String AWS_LAMBDA_EXCEPTION_MESSAGE = "aws.lambda.exception.message";
/**
* Lambda exception field
*/
public static final String AWS_LAMBDA_EXCEPTION_CAUSE = "aws.lambda.exception.cause";
/**
* Lambda exception field
*/
public static final String AWS_LAMBDA_EXCEPTION_ERROR_CODE = "aws.lambda.exception.error.code";
/**
* Lambda exception field
*/
public static final String AWS_LAMBDA_EXCEPTION_REQUEST_ID = "aws.lambda.exception.request.id";
/**
* Lambda exception field
*/
public static final String AWS_LAMBDA_EXCEPTION_STATUS_CODE = "aws.lambda.exception.status.code";
/**
* Lambda exception field
*/
public static final String AWS_LAMBDA_EXCEPTION_ERROR_TYPE = "aws.lambda.exception.error.type";
/**
* Max request body size
*/
public static final long MAX_REQUEST_SIZE = 6 * 1000 * 1000;
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(AWS_LAMBDA_FUNCTION_NAME, AWS_LAMBDA_FUNCTION_QUALIFIER, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
PROXY_CONFIGURATION_SERVICE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD, ENDPOINT_OVERRIDE));
static final PropertyDescriptor AWS_LAMBDA_FUNCTION_NAME = new PropertyDescriptor.Builder()
.name("Amazon Lambda Name")
.description("The Lambda Function Name")
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
static final PropertyDescriptor AWS_LAMBDA_FUNCTION_QUALIFIER = new PropertyDescriptor.Builder()
.name("Amazon Lambda Qualifier (version)")
.description("The Lambda Function Version")
.defaultValue("$LATEST")
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final List<PropertyDescriptor> properties = List.of(
AWS_LAMBDA_FUNCTION_NAME,
AWS_LAMBDA_FUNCTION_QUALIFIER,
REGION,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TIMEOUT,
PROXY_CONFIGURATION_SERVICE,
PROXY_HOST,
PROXY_HOST_PORT,
PROXY_USERNAME,
PROXY_PASSWORD,
ENDPOINT_OVERRIDE);
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@ -137,20 +127,17 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
FlowFile flowFile = session.get();
if (flowFile == null) {
return;
}
final String functionName = context.getProperty(AWS_LAMBDA_FUNCTION_NAME).getValue();
final String qualifier = context.getProperty(AWS_LAMBDA_FUNCTION_QUALIFIER).getValue();
// Max size of message is 6 MB
if (flowFile.getSize() > MAX_REQUEST_SIZE) {
getLogger().error("Max size for request body is 6mb but was {} for flow file {} for function {}",
new Object[]{flowFile.getSize(), flowFile, functionName});
getLogger().error("Max size for request body is 6mb but was {} for flow file {} for function {}", flowFile.getSize(), flowFile, functionName);
session.transfer(flowFile, REL_FAILURE);
return;
}
@ -161,14 +148,14 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
session.exportTo(flowFile, baos);
InvokeRequest invokeRequest = new InvokeRequest()
final InvokeRequest invokeRequest = new InvokeRequest()
.withFunctionName(functionName)
.withLogType(LogType.Tail).withInvocationType(InvocationType.RequestResponse)
.withPayload(ByteBuffer.wrap(baos.toByteArray()))
.withQualifier(qualifier);
long startTime = System.nanoTime();
InvokeResult result = client.invoke(invokeRequest);
final long startTime = System.nanoTime();
final InvokeResult result = client.invoke(invokeRequest);
flowFile = session.putAttribute(flowFile, AWS_LAMBDA_RESULT_STATUS_CODE, result.getStatusCode().toString());
@ -188,31 +175,23 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
final long totalTimeMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
session.getProvenanceReporter().send(flowFile, functionName, totalTimeMillis);
}
} catch (final InvalidRequestContentException
| InvalidParameterValueException
| RequestTooLargeException
| ResourceNotFoundException
| UnsupportedMediaTypeException unrecoverableException) {
getLogger().error("Failed to invoke lambda {} with unrecoverable exception {} for flow file {}",
new Object[]{functionName, unrecoverableException, flowFile});
} catch (final InvalidRequestContentException | InvalidParameterValueException | RequestTooLargeException | ResourceNotFoundException | UnsupportedMediaTypeException unrecoverableException) {
getLogger().error("Failed to invoke lambda {} with unrecoverable exception {} for flow file {}", functionName, unrecoverableException, flowFile);
flowFile = populateExceptionAttributes(session, flowFile, unrecoverableException);
session.transfer(flowFile, REL_FAILURE);
} catch (final TooManyRequestsException retryableServiceException) {
getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}, therefore penalizing flowfile",
new Object[]{functionName, retryableServiceException, flowFile});
getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}, therefore penalizing flowfile", functionName, retryableServiceException, flowFile);
flowFile = populateExceptionAttributes(session, flowFile, retryableServiceException);
flowFile = session.penalize(flowFile);
session.transfer(flowFile, REL_FAILURE);
context.yield();
} catch (final AmazonServiceException unrecoverableServiceException) {
getLogger().error("Failed to invoke lambda {} with exception {} for flow file {} sending to fail",
new Object[]{functionName, unrecoverableServiceException, flowFile});
getLogger().error("Failed to invoke lambda {} with exception {} for flow file {} sending to fail", functionName, unrecoverableServiceException, flowFile);
flowFile = populateExceptionAttributes(session, flowFile, unrecoverableServiceException);
session.transfer(flowFile, REL_FAILURE);
context.yield();
} catch (final Exception exception) {
getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}",
new Object[]{functionName, exception, flowFile});
getLogger().error("Failed to invoke lambda {} with exception {} for flow file {}", functionName, exception, flowFile);
session.transfer(flowFile, REL_FAILURE);
context.yield();
}
@ -227,7 +206,7 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
*/
private FlowFile populateExceptionAttributes(final ProcessSession session, FlowFile flowFile,
final AmazonServiceException exception) {
Map<String,String> attributes = new HashMap<>();
final Map<String, String> attributes = new HashMap<>();
attributes.put(AWS_LAMBDA_EXCEPTION_MESSAGE, exception.getErrorMessage());
attributes.put(AWS_LAMBDA_EXCEPTION_ERROR_CODE, exception.getErrorCode());
attributes.put(AWS_LAMBDA_EXCEPTION_REQUEST_ID, exception.getRequestId());
@ -240,4 +219,15 @@ public class PutLambda extends AbstractAWSLambdaProcessor {
return flowFile;
}
@Override
protected AWSLambdaClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AWSLambdaClient) AWSLambdaClient.builder()
.withRegion(region.getName())
.withEndpointConfiguration(endpointConfiguration)
.withCredentials(credentialsProvider)
.withClientConfiguration(config)
.build();
}
}

View File

@ -17,26 +17,18 @@
package org.apache.nifi.processors.aws.ml;
import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.AmazonWebServiceRequest;
import com.amazonaws.AmazonWebServiceResult;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.MapperFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.json.JsonMapper;
import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
@ -48,6 +40,14 @@ import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import java.io.IOException;
import java.io.InputStream;
import java.util.List;
import java.util.Set;
import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
extends AbstractAWSCredentialsProviderProcessor<T> {
public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
@ -74,21 +74,20 @@ public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceCli
.description("Upon successful completion, the original FlowFile will be routed to this relationship.")
.autoTerminateDefault(true)
.build();
protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
protected static final List<PropertyDescriptor> PROPERTIES = List.of(
MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
REGION,
TIMEOUT,
JSON_PAYLOAD,
SSL_CONTEXT_SERVICE,
ENDPOINT_OVERRIDE));
ENDPOINT_OVERRIDE);
private final static ObjectMapper MAPPER = JsonMapper.builder()
.configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
.build();
private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
REL_ORIGINAL,
private static final Set<Relationship> relationships = Set.of(REL_ORIGINAL,
REL_SUCCESS,
REL_FAILURE
)));
REL_FAILURE);
@Override
public Set<Relationship> getRelationships() {
@ -138,8 +137,9 @@ public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceCli
}
@Override
protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
throw new UnsupportedOperationException("createClient(ProcessContext, AWSCredentials, ClientConfiguration) is not supported");
protected T createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
throw new UnsupportedOperationException();
}
protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {

View File

@ -17,23 +17,18 @@
package org.apache.nifi.processors.aws.ml;
import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.ResponseMetadata;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.http.SdkHttpMetadata;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.fasterxml.jackson.databind.MapperFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.json.JsonMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
@ -42,6 +37,11 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import java.util.List;
import java.util.Set;
import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
extends AbstractAWSCredentialsProviderProcessor<T> {
public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
@ -89,14 +89,14 @@ public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebSe
.defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
.build();
public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
protected static final List<PropertyDescriptor> PROPERTIES = List.of(
TASK_ID,
MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
REGION,
TIMEOUT,
SSL_CONTEXT_SERVICE,
ENDPOINT_OVERRIDE,
PROXY_CONFIGURATION_SERVICE));
PROXY_CONFIGURATION_SERVICE);
private static final ObjectMapper MAPPER = JsonMapper.builder()
.configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
.build();
@ -116,13 +116,13 @@ public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebSe
return relationships;
}
private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
private static final Set<Relationship> relationships = Set.of(
REL_ORIGINAL,
REL_SUCCESS,
REL_RUNNING,
REL_THROTTLED,
REL_FAILURE
)));
);
@Override
public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@ -130,8 +130,9 @@ public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebSe
}
@Override
protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
throw new UnsupportedOperationException("Client creation not supported");
protected T createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
throw new UnsupportedOperationException();
}
protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {

View File

@ -19,14 +19,13 @@ package org.apache.nifi.processors.aws.ml.polly;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.polly.AmazonPollyClient;
import com.amazonaws.services.polly.AmazonPollyClientBuilder;
import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
import com.amazonaws.services.polly.model.TaskStatus;
import com.amazonaws.services.textract.model.ThrottlingException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
@ -38,6 +37,9 @@ import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
@SeeAlso({StartAwsPollyJob.class})
@ -54,10 +56,13 @@ public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<A
private static final String AWS_S3_KEY = "filename";
@Override
protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
protected AmazonPollyClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonPollyClient) AmazonPollyClient.builder()
.withCredentials(credentialsProvider)
.withRegion(context.getProperty(REGION).getValue())
.withEndpointConfiguration(endpointConfiguration)
.withClientConfiguration(config)
.build();
}

View File

@ -19,8 +19,9 @@ package org.apache.nifi.processors.aws.ml.polly;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.polly.AmazonPollyClient;
import com.amazonaws.services.polly.AmazonPollyClientBuilder;
import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
@ -34,11 +35,16 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
@CapabilityDescription("Trigger a AWS Polly job. It should be followed by GetAwsPollyJobStatus processor in order to monitor job status.")
@SeeAlso({GetAwsPollyJobStatus.class})
public class StartAwsPollyJob extends AwsMachineLearningJobStarter<AmazonPollyClient, StartSpeechSynthesisTaskRequest, StartSpeechSynthesisTaskResult> {
@Override
protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
protected AmazonPollyClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonPollyClient) AmazonPollyClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withCredentials(credentialsProvider)
.withClientConfiguration(config)
.withEndpointConfiguration(endpointConfiguration)
.build();
}

View File

@ -17,20 +17,16 @@
package org.apache.nifi.processors.aws.ml.textract;
import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.textract.AmazonTextractClient;
import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
import com.amazonaws.services.textract.model.JobStatus;
import com.amazonaws.services.textract.model.ThrottlingException;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
@ -43,6 +39,13 @@ import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
@SeeAlso({StartAwsTextractJob.class})
@ -66,9 +69,12 @@ public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcesso
}
@Override
protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonTextractClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonTextractClient) AmazonTextractClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withClientConfiguration(config)
.withEndpointConfiguration(endpointConfiguration)
.withCredentials(credentialsProvider)
.build();
}
@ -99,44 +105,25 @@ public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcesso
} catch (ThrottlingException e) {
getLogger().info("Request Rate Limit exceeded", e);
session.transfer(flowFile, REL_THROTTLED);
return;
} catch (Exception e) {
getLogger().warn("Failed to get Textract Job status", e);
session.transfer(flowFile, REL_FAILURE);
return;
}
}
private Object getTask(TextractType typeOfTextract, AmazonTextractClient client, String awsTaskId) {
Object job = null;
switch (typeOfTextract) {
case DOCUMENT_ANALYSIS:
job = client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId));
break;
case DOCUMENT_TEXT_DETECTION:
job = client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId));
break;
case EXPENSE_ANALYSIS:
job = client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId));
break;
}
return job;
return switch (typeOfTextract) {
case DOCUMENT_ANALYSIS -> client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId));
case DOCUMENT_TEXT_DETECTION -> client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId));
case EXPENSE_ANALYSIS -> client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId));
};
}
private JobStatus getTaskStatus(TextractType typeOfTextract, AmazonTextractClient client, String awsTaskId) {
JobStatus jobStatus = JobStatus.IN_PROGRESS;
switch (typeOfTextract) {
case DOCUMENT_ANALYSIS:
jobStatus = JobStatus.fromValue(client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
break;
case DOCUMENT_TEXT_DETECTION:
jobStatus = JobStatus.fromValue(client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId)).getJobStatus());
break;
case EXPENSE_ANALYSIS:
jobStatus = JobStatus.fromValue(client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
break;
}
return jobStatus;
return switch (typeOfTextract) {
case DOCUMENT_ANALYSIS -> JobStatus.fromValue(client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
case DOCUMENT_TEXT_DETECTION -> JobStatus.fromValue(client.getDocumentTextDetection(new GetDocumentTextDetectionRequest().withJobId(awsTaskId)).getJobStatus());
case EXPENSE_ANALYSIS -> JobStatus.fromValue(client.getExpenseAnalysis(new GetExpenseAnalysisRequest().withJobId(awsTaskId)).getJobStatus());
};
}
}

View File

@ -17,12 +17,12 @@
package org.apache.nifi.processors.aws.ml.textract;
import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
import com.amazonaws.AmazonWebServiceRequest;
import com.amazonaws.AmazonWebServiceResult;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.textract.AmazonTextractClient;
import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
@ -30,10 +30,6 @@ import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
@ -47,6 +43,13 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
@SeeAlso({GetAwsTextractJobStatus.class})
@ -86,69 +89,43 @@ public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonText
}
@Override
protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonTextractClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonTextractClient) AmazonTextractClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withCredentials(credentialsProvider)
.withClientConfiguration(config)
.withEndpointConfiguration(endpointConfiguration)
.build();
}
@Override
protected AmazonWebServiceResult sendRequest(AmazonWebServiceRequest request, ProcessContext context, FlowFile flowFile) {
TextractType textractType =
TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
AmazonWebServiceResult result;
switch (textractType) {
case DOCUMENT_ANALYSIS :
result = getClient(context).startDocumentAnalysis((StartDocumentAnalysisRequest) request);
break;
case DOCUMENT_TEXT_DETECTION:
result = getClient(context).startDocumentTextDetection((StartDocumentTextDetectionRequest) request);
break;
case EXPENSE_ANALYSIS:
result = getClient(context).startExpenseAnalysis((StartExpenseAnalysisRequest) request);
break;
default: throw new UnsupportedOperationException("Unsupported textract type: " + textractType);
}
return result;
TextractType textractType = TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
return switch (textractType) {
case DOCUMENT_ANALYSIS -> getClient(context).startDocumentAnalysis((StartDocumentAnalysisRequest) request);
case DOCUMENT_TEXT_DETECTION -> getClient(context).startDocumentTextDetection((StartDocumentTextDetectionRequest) request);
case EXPENSE_ANALYSIS -> getClient(context).startExpenseAnalysis((StartExpenseAnalysisRequest) request);
};
}
@Override
protected Class<? extends AmazonWebServiceRequest> getAwsRequestClass(ProcessContext context, FlowFile flowFile) {
TextractType typeOfTextract =
TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
Class<? extends AmazonWebServiceRequest> result = null;
switch (typeOfTextract) {
case DOCUMENT_ANALYSIS:
result = StartDocumentAnalysisRequest.class;
break;
case DOCUMENT_TEXT_DETECTION:
result = StartDocumentTextDetectionRequest.class;
break;
case EXPENSE_ANALYSIS:
result = StartExpenseAnalysisRequest.class;
break;
}
return result;
final TextractType typeOfTextract = TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
return switch (typeOfTextract) {
case DOCUMENT_ANALYSIS -> StartDocumentAnalysisRequest.class;
case DOCUMENT_TEXT_DETECTION -> StartDocumentTextDetectionRequest.class;
case EXPENSE_ANALYSIS -> StartExpenseAnalysisRequest.class;
};
}
@Override
protected String getAwsTaskId(ProcessContext context, AmazonWebServiceResult amazonWebServiceResult, FlowFile flowFile) {
TextractType textractType =
TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
String result;
switch (textractType) {
case DOCUMENT_ANALYSIS:
result = ((StartDocumentAnalysisResult) amazonWebServiceResult).getJobId();
break;
case DOCUMENT_TEXT_DETECTION:
result = ((StartDocumentTextDetectionResult) amazonWebServiceResult).getJobId();
break;
case EXPENSE_ANALYSIS:
result = ((StartExpenseAnalysisResult) amazonWebServiceResult).getJobId();
break;
default: throw new UnsupportedOperationException("Unsupported textract type.");
}
return result;
final TextractType textractType = TextractType.fromString(context.getProperty(TEXTRACT_TYPE.getName()).evaluateAttributeExpressions(flowFile).getValue());
return switch (textractType) {
case DOCUMENT_ANALYSIS -> ((StartDocumentAnalysisResult) amazonWebServiceResult).getJobId();
case DOCUMENT_TEXT_DETECTION -> ((StartDocumentTextDetectionResult) amazonWebServiceResult).getJobId();
case EXPENSE_ANALYSIS -> ((StartExpenseAnalysisResult) amazonWebServiceResult).getJobId();
};
}
}

View File

@ -19,6 +19,8 @@ package org.apache.nifi.processors.aws.ml.transcribe;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.textract.model.ThrottlingException;
import com.amazonaws.services.transcribe.AmazonTranscribeClient;
import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
@ -42,11 +44,15 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
@WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
})
public class GetAwsTranscribeJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTranscribeClient> {
@Override
protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonTranscribeClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withCredentials(credentialsProvider)
.withEndpointConfiguration(endpointConfiguration)
.withClientConfiguration(config)
.build();
}

View File

@ -18,8 +18,9 @@
package org.apache.nifi.processors.aws.ml.transcribe;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.transcribe.AmazonTranscribeClient;
import com.amazonaws.services.transcribe.model.StartTranscriptionJobRequest;
import com.amazonaws.services.transcribe.model.StartTranscriptionJobResult;
@ -34,17 +35,16 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
@CapabilityDescription("Trigger a AWS Transcribe job. It should be followed by GetAwsTranscribeStatus processor in order to monitor job status.")
@SeeAlso({GetAwsTranscribeJobStatus.class})
public class StartAwsTranscribeJob extends AwsMachineLearningJobStarter<AmazonTranscribeClient, StartTranscriptionJobRequest, StartTranscriptionJobResult> {
@Override
protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withCredentials(credentialsProvider)
.build();
}
@Override
protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
return (AmazonTranscribeClient) AmazonTranscribeClient.builder().build();
protected AmazonTranscribeClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withClientConfiguration(config)
.withEndpointConfiguration(endpointConfiguration)
.withCredentials(credentialsProvider)
.build();
}
@Override

View File

@ -19,6 +19,8 @@ package org.apache.nifi.processors.aws.ml.translate;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.textract.model.ThrottlingException;
import com.amazonaws.services.translate.AmazonTranslateClient;
import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
@ -42,11 +44,15 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
@WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
})
public class GetAwsTranslateJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTranslateClient> {
@Override
protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonTranslateClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonTranslateClient) AmazonTranslateClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withCredentials(credentialsProvider)
.withClientConfiguration(config)
.withEndpointConfiguration(endpointConfiguration)
.build();
}
@ -76,11 +82,9 @@ public class GetAwsTranslateJobStatus extends AwsMachineLearningJobStatusProcess
} catch (ThrottlingException e) {
getLogger().info("Request Rate Limit exceeded", e);
session.transfer(flowFile, REL_THROTTLED);
return;
} catch (Exception e) {
getLogger().warn("Failed to get Polly Job status", e);
session.transfer(flowFile, REL_FAILURE);
return;
}
}

View File

@ -19,6 +19,8 @@ package org.apache.nifi.processors.aws.ml.translate;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.translate.AmazonTranslateClient;
import com.amazonaws.services.translate.model.StartTextTranslationJobRequest;
import com.amazonaws.services.translate.model.StartTextTranslationJobResult;
@ -33,11 +35,15 @@ import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
@CapabilityDescription("Trigger a AWS Translate job. It should be followed by GetAwsTranslateJobStatus processor in order to monitor job status.")
@SeeAlso({GetAwsTranslateJobStatus.class})
public class StartAwsTranslateJob extends AwsMachineLearningJobStarter<AmazonTranslateClient, StartTextTranslationJobRequest, StartTextTranslationJobResult> {
@Override
protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonTranslateClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return (AmazonTranslateClient) AmazonTranslateClient.builder()
.withRegion(context.getProperty(REGION).getValue())
.withCredentials(credentialsProvider)
.withClientConfiguration(config)
.withEndpointConfiguration(endpointConfiguration)
.build();
}

View File

@ -63,13 +63,13 @@ public class DeleteS3Object extends AbstractS3Processor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
BUCKET_WITH_DEFAULT_VALUE,
KEY,
BUCKET,
AWS_CREDENTIALS_PROVIDER_SERVICE,
S3_REGION,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
S3_REGION,
TIMEOUT,
VERSION_ID,
FULL_CONTROL_USER_LIST,
@ -114,7 +114,7 @@ public class DeleteS3Object extends AbstractS3Processor {
final long startNanos = System.nanoTime();
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(flowFile).getValue();

View File

@ -214,13 +214,13 @@ public class FetchS3Object extends AbstractS3Processor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
BUCKET,
BUCKET_WITH_DEFAULT_VALUE,
KEY,
S3_REGION,
AWS_CREDENTIALS_PROVIDER_SERVICE,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TIMEOUT,
VERSION_ID,
SSL_CONTEXT_SERVICE,
@ -268,7 +268,7 @@ public class FetchS3Object extends AbstractS3Processor {
public List<ConfigVerificationResult> verify(ProcessContext context, ComponentLog verificationLogger, Map<String, String> attributes) {
final List<ConfigVerificationResult> results = new ArrayList<>(super.verify(context, verificationLogger, attributes));
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
final String key = context.getProperty(KEY).evaluateAttributeExpressions(attributes).getValue();
final AmazonS3Client client = createClient(context, attributes);
@ -319,7 +319,7 @@ public class FetchS3Object extends AbstractS3Processor {
if (encryptionService != null) {
attributes.put("s3.encryptionStrategy", encryptionService.getStrategyName());
}
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
final GetObjectRequest request = createGetObjectRequest(context, flowFile.getAttributes());
@ -333,7 +333,7 @@ public class FetchS3Object extends AbstractS3Processor {
final ObjectMetadata metadata = s3Object.getObjectMetadata();
if (metadata.getContentDisposition() != null) {
final String contentDisposition = URLDecoder.decode(metadata.getContentDisposition(), StandardCharsets.UTF_8.name());
final String contentDisposition = URLDecoder.decode(metadata.getContentDisposition(), StandardCharsets.UTF_8);
if (contentDisposition.equals(PutS3Object.CONTENT_DISPOSITION_INLINE) || contentDisposition.startsWith("attachment; filename=")) {
setFilePathAttributes(attributes, key);
@ -401,7 +401,7 @@ public class FetchS3Object extends AbstractS3Processor {
}
private GetObjectMetadataRequest createGetObjectMetadataRequest(final ProcessContext context, final Map<String, String> attributes) {
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
final String key = context.getProperty(KEY).evaluateAttributeExpressions(attributes).getValue();
final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(attributes).getValue();
final boolean requesterPays = context.getProperty(REQUESTER_PAYS).asBoolean();
@ -417,7 +417,7 @@ public class FetchS3Object extends AbstractS3Processor {
}
private GetObjectRequest createGetObjectRequest(final ProcessContext context, final Map<String, String> attributes) {
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
final String key = context.getProperty(KEY).evaluateAttributeExpressions(attributes).getValue();
final String versionId = context.getProperty(VERSION_ID).evaluateAttributeExpressions(attributes).getValue();
final boolean requesterPays = context.getProperty(REQUESTER_PAYS).asBoolean();

View File

@ -64,8 +64,8 @@ import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.VerifiableProcessor;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processor.util.list.ListableEntityWrapper;
import org.apache.nifi.processor.util.list.ListedEntity;
@ -285,22 +285,22 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
BUCKET_WITHOUT_DEFAULT_VALUE,
REGION,
AWS_CREDENTIALS_PROVIDER_SERVICE,
LISTING_STRATEGY,
TRACKING_STATE_CACHE,
INITIAL_LISTING_TARGET,
TRACKING_TIME_WINDOW,
BUCKET,
REGION,
ACCESS_KEY,
SECRET_KEY,
RECORD_WRITER,
MIN_AGE,
MAX_AGE,
BATCH_SIZE,
WRITE_OBJECT_TAGS,
WRITE_USER_METADATA,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TIMEOUT,
SSL_CONTEXT_SERVICE,
ENDPOINT_OVERRIDE,
@ -482,7 +482,7 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
final Long maxAgeMilliseconds = context.getProperty(MAX_AGE) != null ? context.getProperty(MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS) : null;
final long listingTimestamp = System.currentTimeMillis();
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions().getValue();
final String bucket = context.getProperty(BUCKET_WITHOUT_DEFAULT_VALUE).evaluateAttributeExpressions().getValue();
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
final ListingSnapshot currentListing = listing.get();
@ -703,7 +703,7 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
final boolean requesterPays = context.getProperty(REQUESTER_PAYS).asBoolean();
final boolean useVersions = context.getProperty(USE_VERSIONS).asBoolean();
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions().getValue();
final String bucket = context.getProperty(BUCKET_WITHOUT_DEFAULT_VALUE).evaluateAttributeExpressions().getValue();
final String delimiter = context.getProperty(DELIMITER).getValue();
final String prefix = context.getProperty(PREFIX).evaluateAttributeExpressions().getValue();
@ -1138,7 +1138,7 @@ public class ListS3 extends AbstractS3Processor implements VerifiableProcessor {
final AmazonS3Client client = createClient(context);
final List<ConfigVerificationResult> results = new ArrayList<>(super.verify(context, logger, attributes));
final String bucketName = context.getProperty(BUCKET).evaluateAttributeExpressions(attributes).getValue();
final String bucketName = context.getProperty(BUCKET_WITHOUT_DEFAULT_VALUE).evaluateAttributeExpressions(attributes).getValue();
final long minAgeMilliseconds = context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
final Long maxAgeMilliseconds = context.getProperty(MAX_AGE) != null ? context.getProperty(MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS) : null;

View File

@ -16,35 +16,29 @@
*/
package org.apache.nifi.processors.aws.s3;
import java.io.BufferedInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.Serializable;
import java.net.URLEncoder;
import java.nio.file.Files;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
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 java.util.Map.Entry;
import java.util.Properties;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
import com.amazonaws.services.s3.model.AccessControlList;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CannedAccessControlList;
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
import com.amazonaws.services.s3.model.MultipartUpload;
import com.amazonaws.services.s3.model.MultipartUploadListing;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.ObjectTagging;
import com.amazonaws.services.s3.model.PartETag;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.PutObjectResult;
import com.amazonaws.services.s3.model.StorageClass;
import com.amazonaws.services.s3.model.Tag;
import com.amazonaws.services.s3.model.UploadPartRequest;
import com.amazonaws.services.s3.model.UploadPartResult;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@ -68,48 +62,38 @@ import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
import com.amazonaws.services.s3.model.AccessControlList;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CannedAccessControlList;
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
import com.amazonaws.services.s3.model.InitiateMultipartUploadResult;
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
import com.amazonaws.services.s3.model.MultipartUpload;
import com.amazonaws.services.s3.model.MultipartUploadListing;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.PartETag;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.PutObjectResult;
import com.amazonaws.services.s3.model.StorageClass;
import com.amazonaws.services.s3.model.UploadPartRequest;
import com.amazonaws.services.s3.model.UploadPartResult;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.Serializable;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
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 java.util.Map.Entry;
import java.util.Properties;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
@SupportsBatching
@SeeAlso({FetchS3Object.class, DeleteS3Object.class, ListS3.class})
@InputRequirement(Requirement.INPUT_REQUIRED)
@Tags({"Amazon", "S3", "AWS", "Archive", "Put"})
@CapabilityDescription("Puts FlowFiles to an Amazon S3 Bucket.\n" +
"The upload uses either the PutS3Object method or the PutS3MultipartUpload method. The PutS3Object method " +
"sends the file in a single synchronous call, but it has a 5GB size limit. Larger files are sent using the " +
"PutS3MultipartUpload method. This multipart process " +
"saves state after each step so that a large upload can be resumed with minimal loss if the processor or " +
"cluster is stopped and restarted.\n" +
"A multipart upload consists of three steps:\n" +
" 1) initiate upload,\n" +
" 2) upload the parts, and\n" +
" 3) complete the upload.\n" +
"For multipart uploads, the processor saves state locally tracking the upload ID and parts uploaded, which " +
"must both be provided to complete the upload.\n" +
"The AWS libraries select an endpoint URL based on the AWS region, but this can be overridden with the " +
"'Endpoint Override URL' property for use with other S3-compatible endpoints.\n" +
"The S3 API specifies that the maximum file size for a PutS3Object upload is 5GB. It also requires that " +
"parts in a multipart upload must be at least 5MB in size, except for the last part. These limits " +
"establish the bounds for the Multipart Upload Threshold and Part Size properties.")
@CapabilityDescription("Writes the contents of a FlowFile as an S3 Object to an Amazon S3 Bucket.")
@DynamicProperty(name = "The name of a User-Defined Metadata field to add to the S3 Object",
value = "The value of a User-Defined Metadata field to add to the S3 Object",
description = "Allows user-defined metadata to be added to the S3 object as key/value pairs",
@ -277,19 +261,21 @@ public class PutS3Object extends AbstractS3Processor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
BUCKET_WITH_DEFAULT_VALUE,
KEY,
BUCKET,
S3_REGION,
AWS_CREDENTIALS_PROVIDER_SERVICE,
STORAGE_CLASS,
ENCRYPTION_SERVICE,
SERVER_SIDE_ENCRYPTION,
CONTENT_TYPE,
CONTENT_DISPOSITION,
CACHE_CONTROL,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
OBJECT_TAGS_PREFIX,
REMOVE_TAG_PREFIX,
STORAGE_CLASS,
S3_REGION,
TIMEOUT,
EXPIRATION_RULE_ID,
FULL_CONTROL_USER_LIST,
@ -309,8 +295,6 @@ public class PutS3Object extends AbstractS3Processor {
MULTIPART_S3_AGEOFF_INTERVAL,
MULTIPART_S3_MAX_AGE,
MULTIPART_TEMP_DIR,
SERVER_SIDE_ENCRYPTION,
ENCRYPTION_SERVICE,
USE_CHUNKED_ENCODING,
USE_PATH_STYLE_ACCESS,
PROXY_CONFIGURATION_SERVICE,
@ -366,7 +350,7 @@ public class PutS3Object extends AbstractS3Processor {
return new File(this.tempDirMultipart + File.separator + getIdentifier());
}
protected boolean localUploadExistsInS3(final AmazonS3Client s3, final String bucket, final MultipartState localState) {
protected boolean localUploadExistsInS3(final AmazonS3 s3, final String bucket, final MultipartState localState) {
ListMultipartUploadsRequest listRequest = new ListMultipartUploadsRequest(bucket);
MultipartUploadListing listing = s3.listMultipartUploads(listRequest);
@ -378,7 +362,7 @@ public class PutS3Object extends AbstractS3Processor {
return false;
}
protected synchronized MultipartState getLocalStateIfInS3(final AmazonS3Client s3, final String bucket,
protected synchronized MultipartState getLocalStateIfInS3(final AmazonS3 s3, final String bucket,
final String s3ObjectKey) throws IOException {
MultipartState currState = getLocalState(s3ObjectKey);
if (currState == null) {
@ -522,7 +506,7 @@ public class PutS3Object extends AbstractS3Processor {
final long startNanos = System.nanoTime();
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
final String cacheKey = getIdentifier() + "/" + bucket + "/" + key;
@ -549,8 +533,7 @@ public class PutS3Object extends AbstractS3Processor {
final FlowFile flowFileCopy = flowFile;
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream rawIn) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn)) {
public void process(final InputStream in) throws IOException {
final ObjectMetadata objectMetadata = new ObjectMetadata();
objectMetadata.setContentLength(ff.getSize());
@ -569,7 +552,7 @@ public class PutS3Object extends AbstractS3Processor {
}
final String contentDisposition = context.getProperty(CONTENT_DISPOSITION).getValue();
String fileName = URLEncoder.encode(ff.getAttribute(CoreAttributes.FILENAME.key()), "UTF-8");
String fileName = URLEncoder.encode(ff.getAttribute(CoreAttributes.FILENAME.key()), StandardCharsets.UTF_8);
if (contentDisposition != null && contentDisposition.equals(CONTENT_DISPOSITION_INLINE)) {
objectMetadata.setContentDisposition(CONTENT_DISPOSITION_INLINE);
attributes.put(S3_CONTENT_DISPOSITION, CONTENT_DISPOSITION_INLINE);
@ -661,7 +644,7 @@ public class PutS3Object extends AbstractS3Processor {
attributes.put(S3_API_METHOD_ATTR_KEY, S3_API_METHOD_PUTOBJECT);
} catch (AmazonClientException e) {
getLogger().info("Failure completing upload flowfile={} bucket={} key={} reason={}",
new Object[]{ffFilename, bucket, key, e.getMessage()});
ffFilename, bucket, key, e.getMessage());
throw (e);
}
} else {
@ -681,21 +664,21 @@ public class PutS3Object extends AbstractS3Processor {
getLogger().info("Resuming upload for flowfile='{}' bucket='{}' key='{}' " +
"uploadID='{}' filePosition='{}' partSize='{}' storageClass='{}' " +
"contentLength='{}' partsLoaded={} lastPart={}/{}",
new Object[]{ffFilename, bucket, key, currentState.getUploadId(),
ffFilename, bucket, key, currentState.getUploadId(),
currentState.getFilePosition(), currentState.getPartSize(),
currentState.getStorageClass().toString(),
currentState.getContentLength(),
currentState.getPartETags().size(),
Integer.toString(lastETag.getPartNumber()),
lastETag.getETag()});
lastETag.getETag());
} else {
getLogger().info("Resuming upload for flowfile='{}' bucket='{}' key='{}' " +
"uploadID='{}' filePosition='{}' partSize='{}' storageClass='{}' " +
"contentLength='{}' no partsLoaded",
new Object[]{ffFilename, bucket, key, currentState.getUploadId(),
ffFilename, bucket, key, currentState.getUploadId(),
currentState.getFilePosition(), currentState.getPartSize(),
currentState.getStorageClass().toString(),
currentState.getContentLength()});
currentState.getContentLength());
}
} else {
currentState = new MultipartState();
@ -705,7 +688,7 @@ public class PutS3Object extends AbstractS3Processor {
currentState.setContentLength(ff.getSize());
persistLocalState(cacheKey, currentState);
getLogger().info("Starting new upload for flowfile='{}' bucket='{}' key='{}'",
new Object[]{ffFilename, bucket, key});
ffFilename, bucket, key);
}
} catch (IOException e) {
getLogger().error("IOException initiating cache state while processing flow files: " +
@ -870,7 +853,6 @@ public class PutS3Object extends AbstractS3Processor {
}
}
}
}
});
if (!attributes.isEmpty()) {
@ -907,20 +889,20 @@ public class PutS3Object extends AbstractS3Processor {
private final AtomicLong lastS3AgeOff = new AtomicLong(0L);
private final DateFormat logFormat = new SimpleDateFormat();
protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
protected void ageoffS3Uploads(final ProcessContext context, final AmazonS3 s3, final long now, String bucket) {
MultipartUploadListing oldUploads = getS3AgeoffListAndAgeoffLocalState(context, s3, now, bucket);
for (MultipartUpload upload : oldUploads.getMultipartUploads()) {
abortS3MultipartUpload(s3, oldUploads.getBucketName(), upload);
}
}
protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3Client s3, final long now, String bucket) {
final long ageoff_interval = context.getProperty(MULTIPART_S3_AGEOFF_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
protected MultipartUploadListing getS3AgeoffListAndAgeoffLocalState(final ProcessContext context, final AmazonS3 s3, final long now, String bucket) {
final long ageoffInterval = context.getProperty(MULTIPART_S3_AGEOFF_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS);
final Long maxAge = context.getProperty(MULTIPART_S3_MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
final long ageCutoff = now - maxAge;
final List<MultipartUpload> ageoffList = new ArrayList<>();
if ((lastS3AgeOff.get() < now - ageoff_interval) && s3BucketLock.tryLock()) {
if ((lastS3AgeOff.get() < now - ageoffInterval) && s3BucketLock.tryLock()) {
try {
ListMultipartUploadsRequest listRequest = new ListMultipartUploadsRequest(bucket);
@ -959,7 +941,7 @@ public class PutS3Object extends AbstractS3Processor {
return result;
}
protected void abortS3MultipartUpload(final AmazonS3Client s3, final String bucket, final MultipartUpload upload) {
protected void abortS3MultipartUpload(final AmazonS3 s3, final String bucket, final MultipartUpload upload) {
final String uploadKey = upload.getKey();
final String uploadId = upload.getUploadId();
final AbortMultipartUploadRequest abortRequest = new AbortMultipartUploadRequest(
@ -1106,7 +1088,7 @@ public class PutS3Object extends AbstractS3Processor {
} else {
first = false;
}
buf.append(String.format("%d/%s", tag.getPartNumber(), tag.getETag()));
buf.append(java.lang.String.format("%d/%s", tag.getPartNumber(), tag.getETag()));
}
}
buf.append(SEPARATOR)

View File

@ -62,8 +62,7 @@ import java.util.stream.Collectors;
@SeeAlso({PutS3Object.class, FetchS3Object.class, ListS3.class})
@Tags({"Amazon", "S3", "AWS", "Archive", "Tag"})
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Sets tags on a FlowFile within an Amazon S3 Bucket. " +
"If attempting to tag a file that does not exist, FlowFile is routed to success.")
@CapabilityDescription("Adds or updates a tag on an Amazon S3 Object.")
public class TagS3Object extends AbstractS3Processor {
public static final PropertyDescriptor TAG_KEY = new PropertyDescriptor.Builder()
@ -107,17 +106,17 @@ public class TagS3Object extends AbstractS3Processor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays.asList(
BUCKET_WITH_DEFAULT_VALUE,
KEY,
BUCKET,
VERSION_ID,
S3_REGION,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TAG_KEY,
TAG_VALUE,
APPEND_TAG,
VERSION_ID,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
S3_REGION,
TIMEOUT,
SSL_CONTEXT_SERVICE,
ENDPOINT_OVERRIDE,
@ -155,13 +154,13 @@ public class TagS3Object extends AbstractS3Processor {
final long startNanos = System.nanoTime();
final String bucket = context.getProperty(BUCKET).evaluateAttributeExpressions(flowFile).getValue();
final String bucket = context.getProperty(BUCKET_WITH_DEFAULT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
final String key = context.getProperty(KEY).evaluateAttributeExpressions(flowFile).getValue();
final String newTagKey = context.getProperty(TAG_KEY).evaluateAttributeExpressions(flowFile).getValue();
final String newTagVal = context.getProperty(TAG_VALUE).evaluateAttributeExpressions(flowFile).getValue();
if(StringUtils.isBlank(bucket)){
failFlowWithBlankEvaluatedProperty(session, flowFile, BUCKET);
failFlowWithBlankEvaluatedProperty(session, flowFile, BUCKET_WITH_DEFAULT_VALUE);
return;
}

View File

@ -16,10 +16,13 @@
*/
package org.apache.nifi.processors.aws.s3.encryption;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3EncryptionClient;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Builder;
import com.amazonaws.services.s3.AmazonS3EncryptionClientV2;
import com.amazonaws.services.s3.AmazonS3EncryptionClientV2Builder;
import com.amazonaws.services.s3.model.CryptoConfigurationV2;
import com.amazonaws.services.s3.model.EncryptionMaterials;
import com.amazonaws.services.s3.model.StaticEncryptionMaterialsProvider;
import org.apache.commons.codec.binary.Base64;
@ -27,6 +30,7 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.ValidationResult;
import javax.crypto.spec.SecretKeySpec;
import java.util.function.Consumer;
/**
* This strategy uses a client master key to perform client-side encryption. Use this strategy when you want the client to perform the encryption,
@ -39,26 +43,31 @@ public class ClientSideCEncryptionStrategy implements S3EncryptionStrategy {
/**
* Create an encryption client.
*
* @param credentialsProvider AWS credentials provider.
* @param clientConfiguration Client configuration
* @param clientBuilder A consumer that is responsible for configuring the client builder
* @param kmsRegion not used by this encryption strategy
* @param keyIdOrMaterial client master key, always base64 encoded
* @return AWS S3 client
*/
@Override
public AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration, String kmsRegion, String keyIdOrMaterial) {
ValidationResult keyValidationResult = validateKey(keyIdOrMaterial);
public AmazonS3 createEncryptionClient(final Consumer<AmazonS3Builder<?, ?>> clientBuilder, final String kmsRegion, final String keyIdOrMaterial) {
final ValidationResult keyValidationResult = validateKey(keyIdOrMaterial);
if (!keyValidationResult.isValid()) {
throw new IllegalArgumentException("Invalid client key; " + keyValidationResult.getExplanation());
}
byte[] keyMaterial = Base64.decodeBase64(keyIdOrMaterial);
SecretKeySpec symmetricKey = new SecretKeySpec(keyMaterial, "AES");
StaticEncryptionMaterialsProvider encryptionMaterialsProvider = new StaticEncryptionMaterialsProvider(new EncryptionMaterials(symmetricKey));
final byte[] keyMaterial = Base64.decodeBase64(keyIdOrMaterial);
final SecretKeySpec symmetricKey = new SecretKeySpec(keyMaterial, "AES");
final StaticEncryptionMaterialsProvider encryptionMaterialsProvider = new StaticEncryptionMaterialsProvider(new EncryptionMaterials(symmetricKey));
AmazonS3EncryptionClient client = new AmazonS3EncryptionClient(credentialsProvider, encryptionMaterialsProvider);
final CryptoConfigurationV2 cryptoConfig = new CryptoConfigurationV2();
cryptoConfig.setAwsKmsRegion(Region.getRegion(Regions.fromName(kmsRegion)));
return client;
final AmazonS3EncryptionClientV2Builder builder = AmazonS3EncryptionClientV2.encryptionBuilder()
.disableChunkedEncoding()
.withCryptoConfiguration(cryptoConfig)
.withEncryptionMaterialsProvider(encryptionMaterialsProvider);
clientBuilder.accept(builder);
return builder.build();
}
@Override

View File

@ -1,61 +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.s3.encryption;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3EncryptionClient;
import com.amazonaws.services.s3.model.CryptoConfiguration;
import com.amazonaws.services.s3.model.KMSEncryptionMaterialsProvider;
import org.apache.commons.lang3.StringUtils;
/**
* This strategy uses KMS key id to perform client-side encryption. Use this strategy when you want the client to perform the encryption,
* (thus incurring the cost of processing) and manage the key in a KMS instance.
*
* See https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingClientSideEncryption.html#client-side-encryption-kms-managed-master-key-intro
*
*/
public class ClientSideKMSEncryptionStrategy implements S3EncryptionStrategy {
/**
* Create an encryption client.
*
* @param credentialsProvider AWS credentials provider.
* @param clientConfiguration Client configuration
* @param kmsRegion AWS KMS region
* @param keyIdOrMaterial KMS key id
* @return AWS S3 client
*/
@Override
public AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration, String kmsRegion, String keyIdOrMaterial) {
KMSEncryptionMaterialsProvider materialProvider = new KMSEncryptionMaterialsProvider(keyIdOrMaterial);
boolean haveKmsRegion = StringUtils.isNotBlank(kmsRegion);
CryptoConfiguration cryptoConfig = new CryptoConfiguration();
if (haveKmsRegion) {
Region awsRegion = Region.getRegion(Regions.fromName(kmsRegion));
cryptoConfig.setAwsKmsRegion(awsRegion);
}
AmazonS3EncryptionClient client = new AmazonS3EncryptionClient(credentialsProvider, materialProvider, cryptoConfig);
return client;
}
}

View File

@ -16,9 +16,8 @@
*/
package org.apache.nifi.processors.aws.s3.encryption;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Builder;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
import com.amazonaws.services.s3.model.ObjectMetadata;
@ -26,6 +25,8 @@ import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.UploadPartRequest;
import org.apache.nifi.components.ValidationResult;
import java.util.function.Consumer;
/**
* This interface defines the API for S3 encryption strategies. The methods have empty defaults
* to minimize the burden on implementations.
@ -72,11 +73,8 @@ public interface S3EncryptionStrategy {
/**
* Create an S3 encryption client.
*
* @param credentialsProvider AWS credentials provider.
* @param clientConfiguration Client configuration.
* @return {@link AmazonS3Client}, perhaps an {@link com.amazonaws.services.s3.AmazonS3EncryptionClient}
*/
default AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration, String kmsRegion, String keyIdOrMaterial) {
default AmazonS3 createEncryptionClient(final Consumer<AmazonS3Builder<?, ?>> clientBuilder, String kmsRegion, String keyIdOrMaterial) {
return null;
}

View File

@ -30,8 +30,7 @@ import org.apache.nifi.components.ValidationResult;
* This strategy uses a customer key to perform server-side encryption. Use this strategy when you want the server to perform the encryption,
* (meaning you pay cost of processing) and when you want to manage the key material yourself.
*
* See https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html
*
* See <a href="https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html">Server Side Encryption Customer Keys</a>.
*/
public class ServerSideCEncryptionStrategy implements S3EncryptionStrategy {
@Override

View File

@ -25,7 +25,7 @@ import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams;
* This strategy uses a KMS key to perform server-side encryption. Use this strategy when you want the server to perform the encryption,
* (meaning you pay the cost of processing) and when you want to use a KMS key.
*
* See https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html
* See <a href="https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html">Using KMS Encryption</a>
*
*/
public class ServerSideKMSEncryptionStrategy implements S3EncryptionStrategy {

View File

@ -26,7 +26,7 @@ import com.amazonaws.services.s3.model.PutObjectRequest;
* perform the encryption (meaning you pay the cost of processing) and you want AWS to completely manage the key.
*
*
* See https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html
* See <a href="https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html">Using Server Side Encryption</a>
*
*/
public class ServerSideS3EncryptionStrategy implements S3EncryptionStrategy {

View File

@ -16,10 +16,9 @@
*/
package org.apache.nifi.processors.aws.s3.encryption;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Builder;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
import com.amazonaws.services.s3.model.ObjectMetadata;
@ -33,13 +32,12 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.processors.aws.s3.AmazonS3EncryptionService;
import org.apache.nifi.processors.aws.s3.AbstractS3Processor;
import org.apache.nifi.processors.aws.s3.AmazonS3EncryptionService;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.StringUtils;
import org.slf4j.Logger;
@ -51,6 +49,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Consumer;
@Tags({"service", "aws", "s3", "encryption", "encrypt", "decryption", "decrypt", "key"})
@ -58,14 +57,13 @@ import java.util.Map;
public class StandardS3EncryptionService extends AbstractControllerService implements AmazonS3EncryptionService {
private static final Logger logger = LoggerFactory.getLogger(StandardS3EncryptionService.class);
private static final Map<String, S3EncryptionStrategy> NAMED_STRATEGIES = new HashMap<String, S3EncryptionStrategy>() {{
put(STRATEGY_NAME_NONE, new NoOpEncryptionStrategy());
put(STRATEGY_NAME_SSE_S3, new ServerSideS3EncryptionStrategy());
put(STRATEGY_NAME_SSE_KMS, new ServerSideKMSEncryptionStrategy());
put(STRATEGY_NAME_SSE_C, new ServerSideCEncryptionStrategy());
put(STRATEGY_NAME_CSE_KMS, new ClientSideKMSEncryptionStrategy());
put(STRATEGY_NAME_CSE_C, new ClientSideCEncryptionStrategy());
}};
private static final Map<String, S3EncryptionStrategy> NAMED_STRATEGIES = Map.of(
STRATEGY_NAME_NONE, new NoOpEncryptionStrategy(),
STRATEGY_NAME_SSE_S3, new ServerSideS3EncryptionStrategy(),
STRATEGY_NAME_SSE_KMS, new ServerSideKMSEncryptionStrategy(),
STRATEGY_NAME_SSE_C, new ServerSideCEncryptionStrategy(),
STRATEGY_NAME_CSE_C, new ClientSideCEncryptionStrategy()
);
private static final AllowableValue NONE = new AllowableValue(STRATEGY_NAME_NONE, "None","No encryption.");
private static final AllowableValue SSE_S3 = new AllowableValue(STRATEGY_NAME_SSE_S3, "Server-side S3","Use server-side, S3-managed encryption.");
@ -100,7 +98,7 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
"In case of Server-side Customer Key, the key must be an AES-256 key. In case of Client-side Customer Key, it can be an AES-256, AES-192 or AES-128 key.")
.required(false)
.sensitive(true)
.addValidator((subject, input, context) -> new ValidationResult.Builder().valid(true).build()) // will be validated in customValidate()
.addValidator(Validator.VALID) // will be validated in customValidate()
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.build();
@ -123,11 +121,8 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
final String newStrategyName = context.getProperty(ENCRYPTION_STRATEGY).getValue();
final String newKeyValue = context.getProperty(ENCRYPTION_VALUE).evaluateAttributeExpressions().getValue();
final S3EncryptionStrategy newEncryptionStrategy = NAMED_STRATEGIES.get(newStrategyName);
String newKmsRegion = null;
if (context.getProperty(KMS_REGION) != null ) {
newKmsRegion = context.getProperty(KMS_REGION).getValue();
}
kmsRegion = context.getProperty(KMS_REGION).getValue();
if (newEncryptionStrategy == null) {
final String msg = "No encryption strategy found for name: " + strategyName;
@ -138,7 +133,6 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
strategyName = newStrategyName;
encryptionStrategy = newEncryptionStrategy;
keyValue = newKeyValue;
kmsRegion = newKmsRegion;
}
@Override
@ -224,8 +218,8 @@ public class StandardS3EncryptionService extends AbstractControllerService imple
}
@Override
public AmazonS3Client createEncryptionClient(AWSCredentialsProvider credentialsProvider, ClientConfiguration clientConfiguration) {
return encryptionStrategy.createEncryptionClient(credentialsProvider, clientConfiguration, kmsRegion, keyValue);
public AmazonS3 createEncryptionClient(final Consumer<AmazonS3Builder<?, ?>> clientBuilder) {
return encryptionStrategy.createEncryptionClient(clientBuilder, kmsRegion, keyValue);
}
@Override

View File

@ -24,22 +24,25 @@ import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.sqs.GetSQS;
import org.apache.nifi.processors.aws.sqs.PutSQS;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sns.SnsClient;
import software.amazon.awssdk.services.sns.SnsClientBuilder;
import software.amazon.awssdk.services.sns.model.MessageAttributeValue;
import software.amazon.awssdk.services.sns.model.PublishRequest;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -51,7 +54,7 @@ import java.util.Map;
@CapabilityDescription("Sends the content of a FlowFile as a notification to the Amazon Simple Notification Service")
@DynamicProperty(name = "A name of an attribute to be added to the notification", value = "The attribute value", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
description = "User specified dynamic Properties are added as attributes to the notification")
public class PutSNS extends AbstractSNSProcessor {
public class PutSNS extends AbstractAwsSyncProcessor<SnsClient, SnsClientBuilder> {
public static final PropertyDescriptor CHARACTER_ENCODING = new PropertyDescriptor.Builder()
.name("Character Set")
@ -61,6 +64,7 @@ public class PutSNS extends AbstractSNSProcessor {
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.required(true)
.build();
public static final PropertyDescriptor USE_JSON_STRUCTURE = new PropertyDescriptor.Builder()
.name("Use JSON Structure")
.description("If true, the contents of the FlowFile must be JSON with a top-level element named 'default'."
@ -70,6 +74,7 @@ public class PutSNS extends AbstractSNSProcessor {
.allowableValues("true", "false")
.required(true)
.build();
public static final PropertyDescriptor SUBJECT = new PropertyDescriptor.Builder()
.name("E-mail Subject")
.description("The optional subject to use for any subscribers that are subscribed via E-mail")
@ -96,10 +101,45 @@ public class PutSNS extends AbstractSNSProcessor {
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(ARN, ARN_TYPE, SUBJECT, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
USE_JSON_STRUCTURE, CHARACTER_ENCODING, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD,
MESSAGEGROUPID, MESSAGEDEDUPLICATIONID));
public static final PropertyDescriptor ARN = new PropertyDescriptor.Builder()
.name("Amazon Resource Name (ARN)")
.description("The name of the resource to which notifications should be published")
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
protected static final AllowableValue ARN_TYPE_TOPIC = new AllowableValue("Topic ARN", "Topic ARN", "The ARN is the name of a topic");
protected static final AllowableValue ARN_TYPE_TARGET = new AllowableValue("Target ARN", "Target ARN", "The ARN is the name of a particular Target, used to notify a specific subscriber");
public static final PropertyDescriptor ARN_TYPE = new PropertyDescriptor.Builder()
.name("ARN Type")
.description("The type of Amazon Resource Name that is being used.")
.expressionLanguageSupported(ExpressionLanguageScope.NONE)
.required(true)
.allowableValues(ARN_TYPE_TOPIC, ARN_TYPE_TARGET)
.defaultValue(ARN_TYPE_TOPIC.getValue())
.build();
public static final List<PropertyDescriptor> properties = List.of(
ARN,
ARN_TYPE,
SUBJECT,
REGION,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
TIMEOUT,
USE_JSON_STRUCTURE,
CHARACTER_ENCODING,
PROXY_HOST,
PROXY_HOST_PORT,
PROXY_USERNAME,
PROXY_PASSWORD,
MESSAGEGROUPID,
MESSAGEDEDUPLICATIONID);
public static final int MAX_SIZE = 256 * 1024;
@ -134,9 +174,13 @@ public class PutSNS extends AbstractSNSProcessor {
final Charset charset = Charset.forName(context.getProperty(CHARACTER_ENCODING).evaluateAttributeExpressions(flowFile).getValue());
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final String message;
try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
session.exportTo(flowFile, baos);
final String message = new String(baos.toByteArray(), charset);
message = baos.toString(charset);
} catch (final IOException ioe) {
throw new ProcessException("Failed to read FlowFile content", ioe);
}
final SnsClient client = getClient(context);
@ -196,4 +240,9 @@ public class PutSNS extends AbstractSNSProcessor {
}
}
@Override
protected SnsClientBuilder createClientBuilder(final ProcessContext context) {
return SnsClient.builder();
}
}

View File

@ -30,13 +30,13 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sqs.SqsClient;
import software.amazon.awssdk.services.sqs.SqsClientBuilder;
import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest;
import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry;
import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@SupportsBatching
@ -44,7 +44,15 @@ import java.util.List;
@InputRequirement(Requirement.INPUT_REQUIRED)
@Tags({"Amazon", "AWS", "SQS", "Queue", "Delete"})
@CapabilityDescription("Deletes a message from an Amazon Simple Queuing Service Queue")
public class DeleteSQS extends AbstractSQSProcessor {
public class DeleteSQS extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
.name("Queue URL")
.description("The URL of the queue delete from")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.build();
public static final PropertyDescriptor RECEIPT_HANDLE = new PropertyDescriptor.Builder()
.name("Receipt Handle")
@ -55,9 +63,20 @@ public class DeleteSQS extends AbstractSQSProcessor {
.defaultValue("${sqs.receipt.handle}")
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(QUEUE_URL, RECEIPT_HANDLE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE,
REGION, TIMEOUT, ENDPOINT_OVERRIDE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME, PROXY_PASSWORD));
public static final List<PropertyDescriptor> properties = List.of(
QUEUE_URL,
RECEIPT_HANDLE,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
REGION,
TIMEOUT,
ENDPOINT_OVERRIDE,
PROXY_HOST,
PROXY_HOST_PORT,
PROXY_USERNAME,
PROXY_PASSWORD);
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@ -101,8 +120,11 @@ public class DeleteSQS extends AbstractSQSProcessor {
getLogger().error("Failed to delete message from SQS due to {}", new Object[] { e });
flowFile = session.penalize(flowFile);
session.transfer(flowFile, REL_FAILURE);
return;
}
}
@Override
protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
return SqsClient.builder();
}
}

View File

@ -26,14 +26,15 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.expression.AttributeExpression.ResultType;
import org.apache.nifi.expression.ExpressionLanguageScope;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sqs.SqsClient;
import software.amazon.awssdk.services.sqs.SqsClientBuilder;
import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest;
import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry;
import software.amazon.awssdk.services.sqs.model.Message;
@ -45,7 +46,6 @@ import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@ -64,7 +64,15 @@ import java.util.concurrent.TimeUnit;
@WritesAttribute(attribute = "sqs.message.id", description = "The unique identifier of the SQS message"),
@WritesAttribute(attribute = "sqs.receipt.handle", description = "The SQS Receipt Handle that is to be used to delete the message from the queue")
})
public class GetSQS extends AbstractSQSProcessor {
public class GetSQS extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
.name("Queue URL")
.description("The URL of the queue to get messages from")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.required(true)
.build();
public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
.name("Character Set")
@ -99,12 +107,6 @@ public class GetSQS extends AbstractSQSProcessor {
.defaultValue("10")
.build();
public static final PropertyDescriptor DYNAMIC_QUEUE_URL = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(QUEUE_URL)
.expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT)
.addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
.build();
public static final PropertyDescriptor RECEIVE_MSG_WAIT_TIME = new PropertyDescriptor.Builder()
.name("Receive Message Wait Time")
.description("The maximum amount of time to wait on a long polling receive call. Setting this to a value of 1 second or greater will "
@ -115,11 +117,24 @@ public class GetSQS extends AbstractSQSProcessor {
.addValidator(StandardValidators.createTimePeriodValidator(0, TimeUnit.SECONDS, 20, TimeUnit.SECONDS)) // 20 seconds is the maximum allowed by SQS
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(DYNAMIC_QUEUE_URL, AUTO_DELETE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE, REGION, BATCH_SIZE, TIMEOUT, ENDPOINT_OVERRIDE,
CHARSET, VISIBILITY_TIMEOUT, RECEIVE_MSG_WAIT_TIME, PROXY_HOST, PROXY_HOST_PORT,
PROXY_USERNAME, PROXY_PASSWORD));
public static final List<PropertyDescriptor> properties = List.of(
QUEUE_URL,
AUTO_DELETE,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
REGION,
BATCH_SIZE,
TIMEOUT,
ENDPOINT_OVERRIDE,
CHARSET,
VISIBILITY_TIMEOUT,
RECEIVE_MSG_WAIT_TIME,
PROXY_HOST,
PROXY_HOST_PORT,
PROXY_USERNAME,
PROXY_PASSWORD);
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
@ -133,9 +148,7 @@ public class GetSQS extends AbstractSQSProcessor {
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
final String queueUrl = context.getProperty(DYNAMIC_QUEUE_URL).evaluateAttributeExpressions()
.getValue();
final String queueUrl = context.getProperty(QUEUE_URL).evaluateAttributeExpressions().getValue();
final SqsClient client = getClient(context);
final ReceiveMessageRequest request = ReceiveMessageRequest.builder()
@ -220,4 +233,10 @@ public class GetSQS extends AbstractSQSProcessor {
+ " may be duplicated. Reason for deletion failure: {}", new Object[]{messages.size(), e});
}
}
@Override
protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
return SqsClient.builder();
}
}

View File

@ -32,7 +32,9 @@ import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.v2.AbstractAwsSyncProcessor;
import software.amazon.awssdk.services.sqs.SqsClient;
import software.amazon.awssdk.services.sqs.SqsClientBuilder;
import software.amazon.awssdk.services.sqs.model.MessageAttributeValue;
import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequest;
import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry;
@ -40,7 +42,6 @@ import software.amazon.awssdk.services.sqs.model.SendMessageBatchResponse;
import java.io.ByteArrayOutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@ -55,9 +56,25 @@ import java.util.concurrent.TimeUnit;
@DynamicProperty(name = "The name of a Message Attribute to add to the message", value = "The value of the Message Attribute",
description = "Allows the user to add key/value pairs as Message Attributes by adding a property whose name will become the name of "
+ "the Message Attribute and value will become the value of the Message Attribute", expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
public class PutSQS extends AbstractSQSProcessor {
public class PutSQS extends AbstractAwsSyncProcessor<SqsClient, SqsClientBuilder> {
private static final String STRING_DATA_TYPE = "String";
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")
.description("The maximum number of messages to send in a single network request")
.required(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("25")
.build();
public static final PropertyDescriptor QUEUE_URL = new PropertyDescriptor.Builder()
.name("Queue URL")
.description("The URL of the queue to act upon")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.required(true)
.build();
public static final PropertyDescriptor DELAY = new PropertyDescriptor.Builder()
.name("Delay")
.displayName("Delay")
@ -85,10 +102,22 @@ public class PutSQS extends AbstractSQSProcessor {
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(QUEUE_URL, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE,
REGION, DELAY, TIMEOUT, ENDPOINT_OVERRIDE, PROXY_HOST, PROXY_HOST_PORT, PROXY_USERNAME,
PROXY_PASSWORD, MESSAGEGROUPID, MESSAGEDEDUPLICATIONID));
public static final List<PropertyDescriptor> properties = List.of(
QUEUE_URL,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE,
REGION,
DELAY,
TIMEOUT,
ENDPOINT_OVERRIDE,
PROXY_HOST,
PROXY_HOST_PORT,
PROXY_USERNAME,
PROXY_PASSWORD,
MESSAGEGROUPID,
MESSAGEDEDUPLICATIONID);
private volatile List<PropertyDescriptor> userDefinedProperties = Collections.emptyList();
@ -158,7 +187,7 @@ public class PutSQS extends AbstractSQSProcessor {
.build();
try {
SendMessageBatchResponse response = client.sendMessageBatch(request);
final SendMessageBatchResponse response = client.sendMessageBatch(request);
// check for errors
if (!response.failed().isEmpty()) {
@ -177,4 +206,8 @@ public class PutSQS extends AbstractSQSProcessor {
session.getProvenanceReporter().send(flowFile, queueUrl, transmissionMillis);
}
@Override
protected SqsClientBuilder createClientBuilder(final ProcessContext context) {
return SqsClient.builder();
}
}

View File

@ -17,17 +17,6 @@
package org.apache.nifi.processors.aws.wag;
import com.amazonaws.http.AmazonHttpClient;
import java.io.ByteArrayInputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.http.impl.EnglishReasonPhraseCatalog;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
@ -54,6 +43,18 @@ import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayRequest;
import org.apache.nifi.processors.aws.wag.client.GenericApiGatewayResponse;
import org.apache.nifi.stream.io.StreamUtils;
import java.io.ByteArrayInputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
@SupportsBatching
@InputRequirement(Requirement.INPUT_ALLOWED)
@Tags({"Amazon", "AWS", "Client", "Gateway-API", "Rest", "http", "https"})
@ -76,7 +77,7 @@ public class InvokeAWSGatewayApi extends AbstractAWSGatewayApiProcessor {
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(Arrays
.asList(
PROP_METHOD,
PROP_AWS_GATEWAY_API_REGION,
REGION,
ACCESS_KEY,
SECRET_KEY,
CREDENTIALS_FILE,
@ -249,17 +250,12 @@ public class InvokeAWSGatewayApi extends AbstractAWSGatewayApiProcessor {
// transfer the message body to the payload
// can potentially be null in edge cases
if (bodyExists) {
final String contentType = response.getHttpResponse().getHeaders()
.get("Content-Type");
if (!(contentType == null) && !contentType.trim().isEmpty()) {
responseFlowFile = session
.putAttribute(responseFlowFile, CoreAttributes.MIME_TYPE.key(),
contentType.trim());
final List<String> contentTypes = response.getHttpResponse().getHeaderValues("Content-Type");
if (contentTypes != null && !contentTypes.isEmpty()) {
responseFlowFile = session.putAttribute(responseFlowFile, CoreAttributes.MIME_TYPE.key(), contentTypes.get(0).trim());
}
responseFlowFile = session
.importFrom(new ByteArrayInputStream(response.getBody().getBytes()),
responseFlowFile);
responseFlowFile = session.importFrom(new ByteArrayInputStream(response.getBody().getBytes()), responseFlowFile);
// emit provenance event
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);

View File

@ -21,6 +21,26 @@
<link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
</head>
<body>
<h2>Multi-part Upload Details</h2>
<p>
The upload uses either the PutS3Object method or the PutS3MultipartUpload method. The PutS3Object method sends the file in a single synchronous call,
but it has a 5GB size limit. Larger files are sent using the PutS3MultipartUpload method. This multipart process saves state after each step so that
a large upload can be resumed with minimal loss if the processor or cluster is stopped and restarted.
A multipart upload consists of three steps:
</p>
<ol>
<li>Initiate upload</li>
<li>Upload the parts</li>
<li>Complete the upload</li>
</ol>
<p>
For multipart uploads, the processor saves state locally tracking the upload ID and parts uploaded, which must both be provided to complete the upload.
The AWS libraries select an endpoint URL based on the AWS region, but this can be overridden with the 'Endpoint Override URL' property for use with other
S3-compatible endpoints. The S3 API specifies that the maximum file size for a PutS3Object upload is 5GB. It also requires that parts in a multipart upload
must be at least 5MB in size, except for the last part. These limits establish the bounds for the Multipart Upload Threshold and Part Size properties.
</p>
<h2>Configuration Details</h2>
<h3>Object Key</h3>
<p>The Object Key property value should not start with "/".</p>

View File

@ -1,138 +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;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.AnonymousAWSCredentials;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.auth.PropertiesCredentials;
import com.amazonaws.internal.StaticCredentialsProvider;
import com.amazonaws.services.s3.AmazonS3Client;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.reporting.InitializationException;
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 java.util.Arrays;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
/**
* Unit tests for AWS Credential specification based on {@link AbstractAWSProcessor} and
* [@link AbstractAWSCredentialsProviderProcessor}, without interaction with S3.
*/
public class TestAWSCredentials {
private TestRunner runner = null;
private AbstractAWSProcessor mockAwsProcessor = null;
private AWSCredentials awsCredentials = null;
private AWSCredentialsProvider awsCredentialsProvider = null;
@BeforeEach
public void setUp() {
mockAwsProcessor = new AbstractAWSCredentialsProviderProcessor<AmazonS3Client>() {
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return Arrays.asList(
AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE,
AbstractAWSProcessor.CREDENTIALS_FILE,
AbstractAWSProcessor.ACCESS_KEY,
AbstractAWSProcessor.SECRET_KEY,
AbstractAWSProcessor.TIMEOUT
);
}
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
awsCredentials = credentials;
final AmazonS3Client s3 = new AmazonS3Client(credentials, config);
return s3;
}
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
awsCredentialsProvider = credentialsProvider;
final AmazonS3Client s3 = new AmazonS3Client(credentialsProvider, config);
return s3;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
}
};
runner = TestRunners.newTestRunner(mockAwsProcessor);
}
@Test
public void testAnonymousByDefault() {
runner.assertValid();
runner.run(1);
assertEquals(AnonymousAWSCredentials.class, awsCredentials.getClass());
assertNull(awsCredentialsProvider);
}
@Test
public void testAccessKeySecretKey() {
runner.setProperty(AbstractAWSProcessor.ACCESS_KEY, "testAccessKey");
runner.setProperty(AbstractAWSProcessor.SECRET_KEY, "testSecretKey");
runner.assertValid();
runner.run(1);
assertEquals(BasicAWSCredentials.class, awsCredentials.getClass());
assertNull(awsCredentialsProvider);
}
@Test
public void testCredentialsFile() {
runner.setProperty(AbstractAWSProcessor.CREDENTIALS_FILE, "src/test/resources/mock-aws-credentials.properties");
runner.assertValid();
runner.run(1);
assertEquals(PropertiesCredentials.class, awsCredentials.getClass());
assertNull(awsCredentialsProvider);
}
@Test
public void testCredentialsProviderControllerService() throws InitializationException {
final AWSCredentialsProviderControllerService credsService = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", credsService);
runner.setProperty(credsService, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(credsService, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.enableControllerService(credsService);
runner.setProperty(AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.assertValid();
runner.run(1);
assertEquals(StaticCredentialsProvider.class, awsCredentialsProvider.getClass());
assertNull(awsCredentials);
}
}

View File

@ -1,100 +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;
import static com.amazonaws.regions.Regions.US_EAST_1;
import static com.amazonaws.regions.Regions.US_WEST_2;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertSame;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import org.apache.nifi.processor.ProcessContext;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class TestAwsClientCache {
@Mock
private ProcessContext contextMock;
@Mock
private AwsClientProvider<AmazonWebServiceClient> awsClientProviderMock;
@Mock
private AmazonWebServiceClient awsClientMock1;
@Mock
private AmazonWebServiceClient awsClientMock2;
private AwsClientCache<AmazonWebServiceClient> clientCache;
@BeforeEach
public void setup() {
clientCache = new AwsClientCache<>();
}
@Test
public void testSameRegionUseExistingClientFromCache() {
final AwsClientDetails clientDetails = getClientDetails(US_WEST_2);
when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock1);
final AmazonWebServiceClient client1 = clientCache.getOrCreateClient(contextMock, clientDetails, awsClientProviderMock);
final AwsClientDetails newClientDetails = getClientDetails(US_WEST_2);
final AmazonWebServiceClient client2 = clientCache.getOrCreateClient(contextMock, newClientDetails, awsClientProviderMock);
verify(awsClientProviderMock, times(1)).createClient(eq(contextMock), any(AwsClientDetails.class));
assertSame(client1, client2);
}
@Test
public void testRegionChangeNewClientIsCreated() {
final AwsClientDetails clientDetails = getClientDetails(US_WEST_2);
when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock1);
final AmazonWebServiceClient client1 = clientCache.getOrCreateClient(contextMock, clientDetails, awsClientProviderMock);
final AwsClientDetails newClientDetails = getClientDetails(US_EAST_1);
when(awsClientProviderMock.createClient(contextMock, newClientDetails)).thenReturn(awsClientMock2);
final AmazonWebServiceClient client2 = clientCache.getOrCreateClient(contextMock, newClientDetails, awsClientProviderMock);
verify(awsClientProviderMock, times(2)).createClient(eq(contextMock), any(AwsClientDetails.class));
assertNotEquals(client1, client2);
}
@Test
public void testSameRegionClientCacheIsClearedNewClientIsCreated() {
final AwsClientDetails clientDetails = getClientDetails(US_WEST_2);
when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock1);
final AmazonWebServiceClient client1 = clientCache.getOrCreateClient(contextMock, clientDetails, awsClientProviderMock);
clientCache.clearCache();
final AwsClientDetails newClientDetails = getClientDetails(US_WEST_2);
when(awsClientProviderMock.createClient(contextMock, clientDetails)).thenReturn(awsClientMock2);
final AmazonWebServiceClient client2 = clientCache.getOrCreateClient(contextMock, newClientDetails, awsClientProviderMock);
verify(awsClientProviderMock, times(2)).createClient(eq(contextMock), any(AwsClientDetails.class));
assertNotEquals(client1, client2);
}
private static AwsClientDetails getClientDetails(Regions region) {
return new AwsClientDetails(Region.getRegion(region));
}
}

View File

@ -1,49 +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;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import org.junit.jupiter.api.Test;
public class TestAwsClientDetails {
@Test
public void clientDetailsEqual() {
final AwsClientDetails details1 = getDefaultClientDetails(Regions.US_WEST_2);
final AwsClientDetails details2 = getDefaultClientDetails(Regions.US_WEST_2);
assertEquals(details1, details2);
assertEquals(details1.hashCode(), details2.hashCode());
}
@Test
public void clientDetailsDifferInRegion() {
final AwsClientDetails details1 = getDefaultClientDetails(Regions.US_WEST_2);
final AwsClientDetails details2 = getDefaultClientDetails(Regions.US_EAST_1);
assertNotEquals(details1, details2);
assertNotEquals(details1.hashCode(), details2.hashCode());
}
private AwsClientDetails getDefaultClientDetails(Regions region) {
return new AwsClientDetails(Region.getRegion(region));
}
}

View File

@ -17,8 +17,9 @@
package org.apache.nifi.processors.aws.credentials.provider.factory;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.s3.AmazonS3Client;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
@ -38,8 +39,8 @@ import static org.apache.nifi.processors.aws.credentials.provider.factory.Creden
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_PROXY_PORT;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_CUSTOM_SIGNER_CLASS_NAME;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_CUSTOM_SIGNER_MODULE_LOCATION;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_REGION;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_ENDPOINT;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_REGION;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.ASSUME_ROLE_STS_SIGNER_OVERRIDE;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.MAX_SESSION_TIME;
import static org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors.PROFILE_NAME;
@ -89,24 +90,9 @@ public class MockAWSProcessor extends AbstractAWSCredentialsProviderProcessor<Am
return validationFailureResults;
}
/**
* Create client using credentials provider. This is the preferred way for creating clients
*/
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
getLogger().info("Creating client with credentials provider");
return new AmazonS3Client(credentialsProvider, config);
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return null;
}
/**
* Create client using AWSCredentials
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client with awd credentials");
return new AmazonS3Client(credentials, config);
}
}

View File

@ -26,7 +26,6 @@ import com.amazonaws.auth.PropertiesFileCredentialsProvider;
import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
import com.amazonaws.auth.Signer;
import com.amazonaws.auth.profile.ProfileCredentialsProvider;
import com.amazonaws.internal.StaticCredentialsProvider;
import org.apache.nifi.processors.aws.credentials.provider.PropertiesCredentialsProvider;
import org.apache.nifi.processors.aws.s3.FetchS3Object;
import org.apache.nifi.processors.aws.signer.AwsSignerType;
@ -89,7 +88,7 @@ public class TestCredentialsProviderFactory {
public void testExplicitDefaultCredentialsExclusive() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(MockAWSProcessor.class);
runner.setProperty(CredentialPropertyDescriptors.USE_DEFAULT_CREDENTIALS, "true");
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
runner.assertNotValid();
}
@ -97,7 +96,7 @@ public class TestCredentialsProviderFactory {
public void testAccessKeyPairCredentials() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(MockAWSProcessor.class);
runner.setProperty(CredentialPropertyDescriptors.USE_DEFAULT_CREDENTIALS, "false");
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
runner.setProperty(CredentialPropertyDescriptors.SECRET_KEY, "BogusSecretKey");
runner.assertValid();
@ -105,8 +104,6 @@ public class TestCredentialsProviderFactory {
final CredentialsProviderFactory factory = new CredentialsProviderFactory();
final AWSCredentialsProvider credentialsProvider = factory.getCredentialsProvider(runner.getProcessContext());
assertNotNull(credentialsProvider);
assertEquals(StaticCredentialsProvider.class,
credentialsProvider.getClass(), "credentials provider should be equal");
final AwsCredentialsProvider credentialsProviderV2 = factory.getAwsCredentialsProvider(runner.getProcessContext());
assertNotNull(credentialsProviderV2);
@ -117,14 +114,14 @@ public class TestCredentialsProviderFactory {
@Test
public void testAccessKeyPairIncomplete() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(MockAWSProcessor.class);
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
runner.assertNotValid();
}
@Test
public void testAccessKeyPairIncompleteS3() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY, "BogusAccessKey");
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, "BogusAccessKey");
runner.assertNotValid();
}

View File

@ -20,8 +20,6 @@ import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
import com.amazonaws.auth.PropertiesFileCredentialsProvider;
import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
import com.amazonaws.internal.StaticCredentialsProvider;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
import org.apache.nifi.processors.aws.s3.FetchS3Object;
import org.apache.nifi.util.TestRunner;
@ -59,8 +57,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
@ -69,8 +67,6 @@ public class AWSCredentialsProviderControllerServiceTest {
assertNotNull(service);
final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
assertNotNull(credentialsProvider);
assertEquals(StaticCredentialsProvider.class,
credentialsProvider.getClass(), "credentials provider should be equal");
}
@Test
@ -78,8 +74,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@ -100,8 +96,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@ -123,8 +119,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@ -139,8 +135,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
@ -155,8 +151,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "899");
@ -168,8 +164,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "899");
@ -181,8 +177,8 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.assertNotValid(serviceImpl);
@ -193,7 +189,7 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_STS_REGION, Region.US_WEST_1.id());
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
@ -247,7 +243,7 @@ public class AWSCredentialsProviderControllerServiceTest {
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "awsSecretKey");
runner.assertNotValid(serviceImpl);
@ -260,7 +256,7 @@ public class AWSCredentialsProviderControllerServiceTest {
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.assertNotValid(serviceImpl);
}
@ -282,7 +278,7 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "awsAccessKey");
runner.assertNotValid(serviceImpl);
}
@ -302,7 +298,7 @@ public class AWSCredentialsProviderControllerServiceTest {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY, "${literal(\"awsAccessKey\")}");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.ACCESS_KEY_ID, "${literal(\"awsAccessKey\")}");
runner.setProperty(serviceImpl, CredentialPropertyDescriptors.SECRET_KEY, "${literal(\"awsSecretKey\")}");
runner.enableControllerService(serviceImpl);
@ -313,7 +309,7 @@ public class AWSCredentialsProviderControllerServiceTest {
assertEquals(
"awsAccessKey", service.getCredentialsProvider().getCredentials().getAWSAccessKeyId(),
"Expression language should be supported for " + CredentialPropertyDescriptors.ACCESS_KEY.getName());
"Expression language should be supported for " + CredentialPropertyDescriptors.ACCESS_KEY_ID.getName());
assertEquals(
"awsSecretKey", service.getCredentialsProvider().getCredentials().getAWSSecretKey(),
"Expression language should be supported for " + CredentialPropertyDescriptors.SECRET_KEY.getName());

View File

@ -16,7 +16,7 @@
*/
package org.apache.nifi.processors.aws.credentials.provider.service;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import org.apache.nifi.processors.aws.s3.FetchS3Object;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
@ -29,79 +29,68 @@ public class AWSProcessorProxyTest {
private TestRunner runner;
@BeforeEach
public void testSetup() throws Throwable {
public void testSetup() {
runner = TestRunners.newTestRunner(FetchS3Object.class);
runner.setProperty(FetchS3Object.BUCKET, "bucket");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "bucket");
runner.assertValid();
}
@AfterEach
public void testTearDown() throws Throwable {
public void testTearDown() {
runner = null;
}
@SuppressWarnings("deprecation")
@Test
public void testProxyHostOnlyInvalid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_HOST, "proxyHost");
runner.assertNotValid();
}
@SuppressWarnings("deprecation")
@Test
public void testProxyHostPortOnlyInvalid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "1");
public void testProxyHostOnlyInvalid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST, "proxyHost");
runner.assertNotValid();
}
@SuppressWarnings("deprecation")
@Test
public void testProxyHostPortNonNumberInvalid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "a");
public void testProxyHostPortOnlyInvalid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "1");
runner.assertNotValid();
}
@SuppressWarnings("deprecation")
@Test
public void testProxyHostAndPortValid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "1");
runner.setProperty(AbstractAWSProcessor.PROXY_HOST, "proxyHost");
public void testProxyHostPortNonNumberInvalid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "a");
runner.assertNotValid();
}
@Test
public void testProxyHostAndPortValid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "1");
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST, "proxyHost");
runner.assertValid();
}
@SuppressWarnings("deprecation")
@Test
public void testProxyUserNoPasswordInValid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_USERNAME, "foo");
public void testProxyUserNoPasswordInValid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_USERNAME, "foo");
runner.assertNotValid();
}
@SuppressWarnings("deprecation")
@Test
public void testProxyNoUserPasswordInValid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_PASSWORD, "foo");
public void testProxyNoUserPasswordInValid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_PASSWORD, "foo");
runner.assertNotValid();
}
@SuppressWarnings("deprecation")
@Test
public void testProxyUserPasswordNoHostInValid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_USERNAME, "foo");
runner.setProperty(AbstractAWSProcessor.PROXY_PASSWORD, "foo");
public void testProxyUserPasswordNoHostInValid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_USERNAME, "foo");
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_PASSWORD, "foo");
runner.assertNotValid();
}
@SuppressWarnings("deprecation")
@Test
public void testProxyUserPasswordHostValid() throws Throwable {
runner.setProperty(AbstractAWSProcessor.PROXY_HOST_PORT, "1");
runner.setProperty(AbstractAWSProcessor.PROXY_HOST, "proxyHost");
runner.setProperty(AbstractAWSProcessor.PROXY_USERNAME, "foo");
runner.setProperty(AbstractAWSProcessor.PROXY_PASSWORD, "foo");
public void testProxyUserPasswordHostValid() {
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST_PORT, "1");
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_HOST, "proxyHost");
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_USERNAME, "foo");
runner.setProperty(AbstractAWSCredentialsProviderProcessor.PROXY_PASSWORD, "foo");
runner.assertValid();
}
}

View File

@ -18,6 +18,9 @@ package org.apache.nifi.processors.aws.dynamodb;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.AmazonServiceException.ErrorType;
import org.apache.nifi.util.MockFlowFile;
import java.util.List;
/**
* Provides reused elements and utilities for the AWS DynamoDB related tests
@ -27,6 +30,21 @@ import com.amazonaws.AmazonServiceException.ErrorType;
* @see DeleteDynamoDBTest
*/
public abstract class AbstractDynamoDBTest {
public static final String REGION = "us-west-2";
public static final String stringHashStringRangeTableName = "StringHashStringRangeTable";
private static final List<String> errorAttributes = List.of(
AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_CODE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_MESSAGE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_TYPE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_SERVICE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_REQUEST_ID,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_STATUS_CODE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE,
AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE
);
protected AmazonServiceException getSampleAwsServiceException() {
final AmazonServiceException testServiceException = new AmazonServiceException("Test AWS Service Exception");
@ -38,4 +56,9 @@ public abstract class AbstractDynamoDBTest {
return testServiceException;
}
protected static void validateServiceExceptionAttributes(final MockFlowFile flowFile) {
errorAttributes.forEach(flowFile::assertAttributeExists);
}
}

View File

@ -40,8 +40,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.REGION;
import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.stringHashStringRangeTableName;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
@ -101,7 +99,7 @@ public class DeleteDynamoDBTest extends AbstractDynamoDBTest {
List<MockFlowFile> flowFiles = deleteRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
for (MockFlowFile flowFile : flowFiles) {
ITAbstractDynamoDBTest.validateServiceExceptionAttribute(flowFile);
validateServiceExceptionAttributes(flowFile);
}
}

View File

@ -39,7 +39,6 @@ import org.mockito.Mockito;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@ -47,8 +46,6 @@ import java.util.Map;
import static org.apache.nifi.components.ConfigVerificationResult.Outcome.FAILED;
import static org.apache.nifi.components.ConfigVerificationResult.Outcome.SUCCESSFUL;
import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.REGION;
import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.stringHashStringRangeTableName;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
@ -411,7 +408,7 @@ public class GetDynamoDBTest extends AbstractDynamoDBTest {
List<MockFlowFile> flowFiles = getRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
for (MockFlowFile flowFile : flowFiles) {
ITAbstractDynamoDBTest.validateServiceExceptionAttribute(flowFile);
validateServiceExceptionAttributes(flowFile);
}
}

View File

@ -1,138 +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.dynamodb;
import com.amazonaws.auth.PropertiesCredentials;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
import com.amazonaws.services.dynamodbv2.document.DynamoDB;
import com.amazonaws.services.dynamodbv2.document.Table;
import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
import com.amazonaws.services.dynamodbv2.model.CreateTableRequest;
import com.amazonaws.services.dynamodbv2.model.DeleteTableResult;
import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
import com.amazonaws.services.dynamodbv2.model.KeyType;
import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
import org.apache.nifi.flowfile.FlowFile;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import java.io.FileInputStream;
import java.util.ArrayList;
import static org.junit.jupiter.api.Assertions.assertNotNull;
public class ITAbstractDynamoDBTest {
protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
protected static DynamoDB dynamoDB;
protected static AmazonDynamoDBClient amazonDynamoDBClient;
protected static String stringHashStringRangeTableName = "StringHashStringRangeTable";
protected static String numberHashNumberRangeTableName = "NumberHashNumberRangeTable";
protected static String numberHashOnlyTableName = "NumberHashOnlyTable";
protected final static String REGION = "us-west-2";
@BeforeAll
public static void beforeClass() throws Exception {
FileInputStream fis = new FileInputStream(CREDENTIALS_FILE);
final PropertiesCredentials credentials = new PropertiesCredentials(fis);
amazonDynamoDBClient = new AmazonDynamoDBClient(credentials);
dynamoDB = new DynamoDB(amazonDynamoDBClient);
amazonDynamoDBClient.setRegion(Region.getRegion(Regions.US_WEST_2));
ArrayList<AttributeDefinition> attributeDefinitions = new ArrayList<AttributeDefinition>();
attributeDefinitions
.add(new AttributeDefinition().withAttributeName("hashS").withAttributeType("S"));
attributeDefinitions
.add(new AttributeDefinition().withAttributeName("rangeS").withAttributeType("S"));
ArrayList<KeySchemaElement> keySchema = new ArrayList<KeySchemaElement>();
keySchema.add(new KeySchemaElement().withAttributeName("hashS").withKeyType(KeyType.HASH));
keySchema.add(new KeySchemaElement().withAttributeName("rangeS").withKeyType(KeyType.RANGE));
CreateTableRequest request = new CreateTableRequest()
.withTableName(stringHashStringRangeTableName)
.withKeySchema(keySchema)
.withAttributeDefinitions(attributeDefinitions)
.withProvisionedThroughput(new ProvisionedThroughput()
.withReadCapacityUnits(5L)
.withWriteCapacityUnits(6L));
Table stringHashStringRangeTable = dynamoDB.createTable(request);
stringHashStringRangeTable.waitForActive();
attributeDefinitions = new ArrayList<AttributeDefinition>();
attributeDefinitions
.add(new AttributeDefinition().withAttributeName("hashN").withAttributeType("N"));
attributeDefinitions
.add(new AttributeDefinition().withAttributeName("rangeN").withAttributeType("N"));
keySchema = new ArrayList<KeySchemaElement>();
keySchema.add(new KeySchemaElement().withAttributeName("hashN").withKeyType(KeyType.HASH));
keySchema.add(new KeySchemaElement().withAttributeName("rangeN").withKeyType(KeyType.RANGE));
request = new CreateTableRequest()
.withTableName(numberHashNumberRangeTableName)
.withKeySchema(keySchema)
.withAttributeDefinitions(attributeDefinitions)
.withProvisionedThroughput(new ProvisionedThroughput()
.withReadCapacityUnits(5L)
.withWriteCapacityUnits(6L));
Table numberHashNumberRangeTable = dynamoDB.createTable(request);
numberHashNumberRangeTable.waitForActive();
attributeDefinitions = new ArrayList<AttributeDefinition>();
attributeDefinitions
.add(new AttributeDefinition().withAttributeName("hashN").withAttributeType("N"));
keySchema = new ArrayList<KeySchemaElement>();
keySchema.add(new KeySchemaElement().withAttributeName("hashN").withKeyType(KeyType.HASH));
request = new CreateTableRequest()
.withTableName(numberHashOnlyTableName)
.withKeySchema(keySchema)
.withAttributeDefinitions(attributeDefinitions)
.withProvisionedThroughput(new ProvisionedThroughput()
.withReadCapacityUnits(5L)
.withWriteCapacityUnits(6L));
Table numberHashOnlyTable = dynamoDB.createTable(request);
numberHashOnlyTable.waitForActive();
}
protected static void validateServiceExceptionAttribute(FlowFile flowFile) {
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_CODE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_MESSAGE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_TYPE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_SERVICE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_REQUEST_ID));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_STATUS_CODE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_EXCEPTION_MESSAGE));
assertNotNull(flowFile.getAttribute(AbstractDynamoDBProcessor.DYNAMODB_ERROR_RETRYABLE));
}
@AfterAll
public static void afterClass() {
DeleteTableResult result = amazonDynamoDBClient.deleteTable(stringHashStringRangeTableName);
result = amazonDynamoDBClient.deleteTable(numberHashNumberRangeTableName);
result = amazonDynamoDBClient.deleteTable(numberHashOnlyTableName);
}
}

View File

@ -36,12 +36,11 @@ import org.mockito.ArgumentMatchers;
import org.mockito.Mockito;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.REGION;
import static org.apache.nifi.processors.aws.dynamodb.ITAbstractDynamoDBTest.stringHashStringRangeTableName;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
@ -54,7 +53,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
@BeforeEach
public void setUp() {
outcome = new BatchWriteItemOutcome(result);
result.setUnprocessedItems(new HashMap<String, List<WriteRequest>>());
result.setUnprocessedItems(new HashMap<>());
final DynamoDB mockDynamoDB = new DynamoDB(Regions.AP_NORTHEAST_1) {
@Override
public BatchWriteItemOutcome batchWriteItem(TableWriteItems... tableWriteItems) {
@ -68,7 +67,6 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
return mockDynamoDB;
}
};
}
@Test
@ -103,7 +101,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
for (MockFlowFile flowFile : flowFiles) {
ITAbstractDynamoDBTest.validateServiceExceptionAttribute(flowFile);
validateServiceExceptionAttributes(flowFile);
}
}
@ -195,15 +193,15 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
String document = "{\"name\":\"john\"}";
putRunner.enqueue(document.getBytes());
putRunner.run(1);
putRunner.assertAllFlowFilesTransferred(AbstractWriteDynamoDBProcessor.REL_SUCCESS, 1);
putRunner.assertAllFlowFilesTransferred(AbstractDynamoDBProcessor.REL_SUCCESS, 1);
List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_SUCCESS);
List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_SUCCESS);
for (MockFlowFile flowFile : flowFiles) {
System.out.println(flowFile.getAttributes());
assertEquals(document, new String(flowFile.toByteArray()));
@ -223,27 +221,25 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
String document = "{\"name\":\"john\"}";
putRunner.enqueue(document.getBytes());
byte [] item = new byte[PutDynamoDB.DYNAMODB_MAX_ITEM_SIZE + 1];
for (int i = 0; i < item.length; i++) {
item[i] = 'a';
}
Arrays.fill(item, (byte) 'a');
String document2 = new String(item);
putRunner.enqueue(document2.getBytes());
putRunner.run(2,true,true);
List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_FAILURE);
List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
for (MockFlowFile flowFile : flowFilesFailed) {
System.out.println(flowFile.getAttributes());
flowFile.assertAttributeExists(PutDynamoDB.AWS_DYNAMO_DB_ITEM_SIZE_ERROR);
assertEquals(item.length,flowFile.getSize());
}
List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_SUCCESS);
List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_SUCCESS);
for (MockFlowFile flowFile : flowFilesSuccessful) {
System.out.println(flowFile.getAttributes());
assertEquals(document, new String(flowFile.toByteArray()));
@ -262,27 +258,25 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
String document = "{\"name\":\"john\"}";
putRunner.enqueue(document.getBytes());
byte [] item = new byte[PutDynamoDB.DYNAMODB_MAX_ITEM_SIZE + 1];
for (int i = 0; i < item.length; i++) {
item[i] = 'a';
}
Arrays.fill(item, (byte) 'a');
String document2 = new String(item);
putRunner.enqueue(document2.getBytes());
putRunner.run(1);
List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_FAILURE);
List<MockFlowFile> flowFilesFailed = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
for (MockFlowFile flowFile : flowFilesFailed) {
System.out.println(flowFile.getAttributes());
flowFile.assertAttributeExists(PutDynamoDB.AWS_DYNAMO_DB_ITEM_SIZE_ERROR);
assertEquals(item.length,flowFile.getSize());
}
List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_SUCCESS);
List<MockFlowFile> flowFilesSuccessful = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_SUCCESS);
for (MockFlowFile flowFile : flowFilesSuccessful) {
System.out.println(flowFile.getAttributes());
assertEquals(document, new String(flowFile.toByteArray()));
@ -301,26 +295,23 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
byte [] item = new byte[PutDynamoDB.DYNAMODB_MAX_ITEM_SIZE + 1];
for (int i = 0; i < item.length; i++) {
item[i] = 'a';
}
Arrays.fill(item, (byte) 'a');
String document = new String(item);
putRunner.enqueue(document.getBytes());
putRunner.run(1);
putRunner.assertAllFlowFilesTransferred(AbstractWriteDynamoDBProcessor.REL_FAILURE, 1);
putRunner.assertAllFlowFilesTransferred(AbstractDynamoDBProcessor.REL_FAILURE, 1);
List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractWriteDynamoDBProcessor.REL_FAILURE);
List<MockFlowFile> flowFiles = putRunner.getFlowFilesForRelationship(AbstractDynamoDBProcessor.REL_FAILURE);
assertEquals(1,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
System.out.println(flowFile.getAttributes());
flowFile.assertAttributeExists(PutDynamoDB.AWS_DYNAMO_DB_ITEM_SIZE_ERROR);
assertEquals(item.length,flowFile.getSize());
}
}
@Test
@ -348,7 +339,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
String document = "{\"name\":\"john\"}";
putRunner.enqueue(document.getBytes());
@ -388,7 +379,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
String document = "{\"name\":\"john\"}";
putRunner.enqueue(document.getBytes());
@ -427,7 +418,7 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
putRunner.setProperty(AbstractDynamoDBProcessor.HASH_KEY_VALUE, "h1");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_NAME, "rangeS");
putRunner.setProperty(AbstractDynamoDBProcessor.RANGE_KEY_VALUE, "r1");
putRunner.setProperty(AbstractWriteDynamoDBProcessor.JSON_DOCUMENT, "document");
putRunner.setProperty(AbstractDynamoDBProcessor.JSON_DOCUMENT, "document");
String document = "{\"name\":\"john\"}";
putRunner.enqueue(document.getBytes());
@ -443,9 +434,8 @@ public class PutDynamoDBTest extends AbstractDynamoDBTest {
@Test
public void testStringHashStringRangePutSuccessfulWithMockOneUnprocessed() {
Map<String, List<WriteRequest>> unprocessed =
new HashMap<String, List<WriteRequest>>();
PutRequest put = new PutRequest();
final Map<String, List<WriteRequest>> unprocessed = new HashMap<>();
final PutRequest put = new PutRequest();
put.addItemEntry("hashS", new AttributeValue("h1"));
put.addItemEntry("rangeS", new AttributeValue("r1"));
WriteRequest write = new WriteRequest(put);

View File

@ -20,10 +20,12 @@ import org.apache.nifi.processors.aws.s3.FetchS3Object;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assumptions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -32,44 +34,38 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
/**
* This test contains both unit and integration test (integration tests are ignored by default)
*/
public class ITPutKinesisFirehose {
public class ITPutKinesisFirehose {
private final byte[] ONE_MB = new byte[1000 * 1024];
private TestRunner runner;
protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
@BeforeAll
public static void testSkipping() {
Assumptions.assumeTrue(new File(CREDENTIALS_FILE).exists());
}
@BeforeEach
public void setUp() throws Exception {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.ACCESS_KEY, "abcd");
runner.setProperty(PutKinesisFirehose.SECRET_KEY, "secret key");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "deliveryName");
runner.assertValid();
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
}
@AfterEach
public void tearDown() throws Exception {
runner = null;
}
/**
* Comment out ignore for integration tests (requires creds files)
*/
@Test
public void testIntegrationSuccess() throws Exception {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
runner.enqueue("test".getBytes());
runner.run(1);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 1);
final List<MockFlowFile> ffs = runner.getFlowFilesForRelationship(FetchS3Object.REL_SUCCESS);
final MockFlowFile out = ffs.iterator().next();
out.assertContentEquals("test".getBytes());
out.assertContentEquals("test");
}
/**
@ -77,57 +73,33 @@ public class ITPutKinesisFirehose {
*/
@Test
public void testIntegrationFailedBadStreamName() throws Exception {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "bad-firehose-s3-test");
runner.assertValid();
runner.enqueue("test".getBytes());
runner.run(1);
runner.enqueue("test");
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_FAILURE, 1);
}
@Test
public void testOneMessageWithMaxBufferSizeGreaterThan1MBOneSuccess() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "2");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.run(1);
runner.enqueue(ONE_MB);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 1);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(1,flowFiles.size());
}
@Test
public void testTwoMessageBatch5WithMaxBufferSize1MBRunOnceTwoMessageSent() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "5");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "2 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.run(1);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(2,flowFiles.size());
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
for (MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
@ -135,73 +107,43 @@ public class ITPutKinesisFirehose {
@Test
public void testThreeMessageWithBatch10MaxBufferSize1MBTRunOnceTwoMessageSent() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.enqueue(bytes.clone());
runner.run(1);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(2,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
}
@Test
public void testTwoMessageWithBatch10MaxBufferSize1MBTRunOnceTwoMessageSent() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.run(1);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(2,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
}
@Test
public void testThreeMessageWithBatch2MaxBufferSize1MBRunTwiceThreeMessageSent() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "2");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.enqueue(bytes.clone());
runner.run(2, true, true);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.run(2);
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 3);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(3,flowFiles.size());
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
for (MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
@ -209,193 +151,134 @@ public class ITPutKinesisFirehose {
@Test
public void testThreeMessageHello2MBThereWithBatch10MaxBufferSize1MBRunOnceTwoMessageSuccessOneFailed() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE * 2)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue("hello".getBytes());
runner.enqueue(bytes);
runner.enqueue(ONE_MB);
runner.enqueue("there".getBytes());
runner.run(1, true, true);
runner.run();
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(2,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
assertEquals(1,flowFilesFailed.size());
for (MockFlowFile flowFileFailed : flowFilesFailed) {
assertNotNull(flowFileFailed.getAttribute(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE));
for (final MockFlowFile flowFileFailed : flowFilesFailed) {
flowFileFailed.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE);
}
}
@Test
public void testTwoMessageHello2MBWithBatch10MaxBufferSize1MBRunOnceOneSuccessOneFailed() throws Exception {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE * 2)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue("hello".getBytes());
runner.enqueue(bytes);
runner.enqueue(ONE_MB);
runner.run(1, true, true);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(1,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
flowFile.assertContentEquals("hello".getBytes());
}
List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
assertEquals(1,flowFilesFailed.size());
for (MockFlowFile flowFileFailed : flowFilesFailed) {
for (final MockFlowFile flowFileFailed : flowFilesFailed) {
assertNotNull(flowFileFailed.getAttribute(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE));
}
}
@Test
public void testTwoMessage2MBHelloWorldWithBatch10MaxBufferSize1MBRunOnceTwoMessageSent() throws Exception {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE * 2)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(ONE_MB);
runner.enqueue("HelloWorld".getBytes());
runner.run(1, true, true);
runner.run();
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(1,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
flowFile.assertContentEquals("HelloWorld".getBytes());
}
List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
assertEquals(1,flowFilesFailed.size());
for (MockFlowFile flowFileFailed : flowFilesFailed) {
for (final MockFlowFile flowFileFailed : flowFilesFailed) {
assertNotNull(flowFileFailed.getAttribute(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_ERROR_MESSAGE));
}
}
@Test
public void testTwoMessageHelloWorldWithBatch10MaxBufferSize1MBRunOnceTwoMessageSent() throws Exception {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
runner.enqueue("Hello".getBytes());
runner.enqueue("World".getBytes());
runner.run(1, true, true);
runner.run();
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(2,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
flowFiles.get(0).assertContentEquals("Hello".getBytes());
flowFiles.get(1).assertContentEquals("World".getBytes());
List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
final List<MockFlowFile> flowFilesFailed = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_FAILURE);
assertEquals(0,flowFilesFailed.size());
}
@Test
public void testThreeMessageWithBatch5MaxBufferSize1MBRunOnceTwoMessageSent() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "5");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.enqueue(bytes.clone());
runner.run(1, true, true);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(2,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
}
@Test
public void test5MessageWithBatch10MaxBufferSize10MBRunOnce5MessageSent() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "10");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[10];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.enqueue(bytes.clone());
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.run(1, true, true);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 5);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(5,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
}
@Test
public void test5MessageWithBatch2MaxBufferSize10MBRunOnce2MessageSent() {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "2");
runner.setProperty(PutKinesisFirehose.MAX_MESSAGE_BUFFER_SIZE_MB, "1 MB");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "testkinesis");
runner.assertValid();
byte [] bytes = new byte[10];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.enqueue(bytes.clone());
runner.enqueue(bytes);
runner.enqueue(bytes.clone());
runner.run(1, true, true);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.enqueue(ONE_MB);
runner.run();
runner.assertAllFlowFilesTransferred(PutKinesisFirehose.REL_SUCCESS, 2);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutKinesisFirehose.REL_SUCCESS);
assertEquals(2,flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
for (final MockFlowFile flowFile : flowFiles) {
flowFile.assertAttributeExists(PutKinesisFirehose.AWS_KINESIS_FIREHOSE_RECORD_ID);
}
}

View File

@ -29,13 +29,10 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
public class TestPutKinesisFirehose {
private TestRunner runner;
protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
@BeforeEach
public void setUp() throws Exception {
runner = TestRunners.newTestRunner(PutKinesisFirehose.class);
runner.setProperty(PutKinesisFirehose.ACCESS_KEY, "abcd");
runner.setProperty(PutKinesisFirehose.SECRET_KEY, "secret key");
runner.setProperty(PutKinesisFirehose.KINESIS_FIREHOSE_DELIVERY_STREAM_NAME, "deliveryName");
runner.assertValid();
}
@ -67,9 +64,6 @@ public class TestPutKinesisFirehose {
runner.setProperty(PutKinesisFirehose.BATCH_SIZE, "1");
runner.assertValid();
byte[] bytes = new byte[(PutKinesisFirehose.MAX_MESSAGE_SIZE + 1)];
for (int i = 0; i < bytes.length; i++) {
bytes[i] = 'a';
}
runner.enqueue(bytes);
runner.run(1);

View File

@ -74,7 +74,7 @@ public class TestConsumeKinesisStream {
public void testValidWithCredentials() throws InitializationException {
final ControllerService credentialsProvider = new AWSCredentialsProviderControllerService();
runner.addControllerService("credentials-provider", credentialsProvider);
runner.setProperty(credentialsProvider, CredentialPropertyDescriptors.ACCESS_KEY, "access-key");
runner.setProperty(credentialsProvider, CredentialPropertyDescriptors.ACCESS_KEY_ID, "access-key");
runner.setProperty(credentialsProvider, CredentialPropertyDescriptors.SECRET_KEY, "secret-key");
runner.assertValid(credentialsProvider);
runner.enableControllerService(credentialsProvider);
@ -232,10 +232,6 @@ public class TestConsumeKinesisStream {
runWorker(true, true);
}
@Test
public void testRunWorkerWithoutCredentials() throws UnknownHostException, InitializationException, InterruptedException {
runWorker(false, false);
}
@Test
public void testInvalidDynamicKCLProperties() {
@ -341,6 +337,7 @@ public class TestConsumeKinesisStream {
private void runWorker(final boolean withCredentials, final boolean waitForFailure) throws UnknownHostException, InitializationException, InterruptedException {
final TestRunner mockConsumeKinesisStreamRunner = TestRunners.newTestRunner(MockConsumeKinesisStream.class);
mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.GRACEFUL_SHUTDOWN_TIMEOUT, "50 millis");
mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.KINESIS_STREAM_NAME, "test-stream");
mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.APPLICATION_NAME, "test-application");
mockConsumeKinesisStreamRunner.setProperty(ConsumeKinesisStream.REGION, Regions.EU_WEST_2.getName());
@ -349,7 +346,7 @@ public class TestConsumeKinesisStream {
final AWSCredentialsProviderService awsCredentialsProviderService = new AWSCredentialsProviderControllerService();
mockConsumeKinesisStreamRunner.addControllerService("aws-credentials", awsCredentialsProviderService);
if (withCredentials) {
mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.ACCESS_KEY, "test-access");
mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.ACCESS_KEY_ID, "test-access");
mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.SECRET_KEY, "test-secret");
} else {
mockConsumeKinesisStreamRunner.setProperty(awsCredentialsProviderService, CredentialPropertyDescriptors.USE_ANONYMOUS_CREDENTIALS, "true");
@ -380,16 +377,12 @@ public class TestConsumeKinesisStream {
if (!waitForFailure) {
// re-trigger the processor to ensure the Worker isn't re-initialised when already running
mockConsumeKinesisStreamRunner.run(1, false, false);
mockConsumeKinesisStreamRunner.run(1, true, false);
assertTrue(((MockProcessContext) mockConsumeKinesisStreamRunner.getProcessContext()).isYieldCalled());
// stop the processor
mockConsumeKinesisStreamRunner.stop();
} else {
for (int runs = 0; runs < 10; runs++) {
try {
mockConsumeKinesisStreamRunner.run(1, false, false);
Thread.sleep(1_000);
} catch (AssertionError e) {
assertThat(e.getCause(), instanceOf(ProcessException.class));
assertThat(e.getCause().getMessage(), equalTo("Worker has shutdown unexpectedly, possibly due to a configuration issue; check logs for details"));

View File

@ -17,24 +17,15 @@
package org.apache.nifi.processors.aws.ml.polly;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_ORIGINAL;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.polly.AmazonPollyClient;
import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
import com.amazonaws.services.polly.model.SynthesisTask;
import com.amazonaws.services.polly.model.TaskStatus;
import java.util.Collections;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
import org.apache.nifi.reporting.InitializationException;
@ -48,6 +39,18 @@ import org.mockito.Captor;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import java.util.Collections;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_ORIGINAL;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
public class GetAwsPollyStatusTest {
private static final String TEST_TASK_ID = "testTaskId";
@ -66,7 +69,8 @@ public class GetAwsPollyStatusTest {
final GetAwsPollyJobStatus mockGetAwsPollyStatus = new GetAwsPollyJobStatus() {
@Override
protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonPollyClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockPollyClient;
}
};

View File

@ -17,17 +17,10 @@
package org.apache.nifi.processors.aws.ml.textract;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TEXTRACT_TYPE;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.textract.AmazonTextractClient;
import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
@ -46,6 +39,15 @@ import org.mockito.Captor;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TEXTRACT_TYPE;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
public class GetAwsTextractJobStatusTest {
private static final String TEST_TASK_ID = "testTaskId";
@ -62,7 +64,8 @@ public class GetAwsTextractJobStatusTest {
when(mockAwsCredentialsProvider.getIdentifier()).thenReturn("awsCredentialProvider");
final GetAwsTextractJobStatus awsTextractJobStatusGetter = new GetAwsTextractJobStatus() {
@Override
protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonTextractClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockTextractClient;
}
};

View File

@ -17,24 +17,16 @@
package org.apache.nifi.processors.aws.ml.transcribe;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.transcribe.AmazonTranscribeClient;
import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
import com.amazonaws.services.transcribe.model.Transcript;
import com.amazonaws.services.transcribe.model.TranscriptionJob;
import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
import java.util.Collections;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
import org.apache.nifi.reporting.InitializationException;
@ -48,6 +40,17 @@ import org.mockito.Captor;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import java.util.Collections;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
public class GetAwsTranscribeJobStatusTest {
private static final String TEST_TASK_ID = "testTaskId";
@ -68,7 +71,8 @@ public class GetAwsTranscribeJobStatusTest {
when(mockAwsCredentialsProvider.getIdentifier()).thenReturn(AWS_CREDENTIAL_PROVIDER_NAME);
final GetAwsTranscribeJobStatus mockPollyFetcher = new GetAwsTranscribeJobStatus() {
@Override
protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentials, ClientConfiguration config) {
protected AmazonTranscribeClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockTranscribeClient;
}
};

View File

@ -17,24 +17,16 @@
package org.apache.nifi.processors.aws.ml.translate;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.translate.AmazonTranslateClient;
import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
import com.amazonaws.services.translate.model.JobStatus;
import com.amazonaws.services.translate.model.OutputDataConfig;
import com.amazonaws.services.translate.model.TextTranslationJobProperties;
import java.util.Collections;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
import org.apache.nifi.reporting.InitializationException;
@ -48,6 +40,17 @@ import org.mockito.Captor;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import java.util.Collections;
import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_FAILURE;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_SUCCESS;
import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
public class GetAwsTranslateJobStatusTest {
private static final String TEST_TASK_ID = "testTaskId";
@ -68,7 +71,8 @@ public class GetAwsTranslateJobStatusTest {
when(mockAwsCredentialsProvider.getIdentifier()).thenReturn(AWS_CREDENTIALS_PROVIDER_NAME);
final GetAwsTranslateJobStatus mockPollyFetcher = new GetAwsTranslateJobStatus() {
@Override
protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
protected AmazonTranslateClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockTranslateClient;
}
};

View File

@ -16,17 +16,17 @@
*/
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.auth.PropertiesCredentials;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.amazonaws.auth.AWSStaticCredentialsProvider;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.services.kms.AWSKMS;
import com.amazonaws.services.kms.AWSKMSClient;
import com.amazonaws.services.kms.model.CreateKeyRequest;
import com.amazonaws.services.kms.model.CreateKeyResult;
import com.amazonaws.services.kms.model.GenerateDataKeyRequest;
import com.amazonaws.services.kms.model.GenerateDataKeyResult;
import com.amazonaws.services.kms.model.ScheduleKeyDeletionRequest;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3ClientBuilder;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CreateBucketRequest;
import com.amazonaws.services.s3.model.DeleteBucketRequest;
@ -34,23 +34,32 @@ import com.amazonaws.services.s3.model.ObjectListing;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.ObjectTagging;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.PutObjectResult;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.amazonaws.services.s3.model.Tag;
import org.apache.nifi.util.file.FileUtils;
import org.apache.nifi.components.PropertyDescriptor;
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.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.testcontainers.containers.localstack.LocalStackContainer;
import org.testcontainers.utility.DockerImageName;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import static org.junit.jupiter.api.Assertions.fail;
@ -63,57 +72,70 @@ import static org.junit.jupiter.api.Assertions.fail;
* @see ITListS3
*/
public abstract class AbstractS3IT {
protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
protected final static String SAMPLE_FILE_RESOURCE_NAME = "/hello.txt";
protected final static String REGION = System.getProperty("it.aws.region", "us-west-1");
// Adding REGION to bucket prevents errors of
// "A conflicting conditional operation is currently in progress against this resource."
// when bucket is rapidly added/deleted and consistency propagation causes this error.
// (Should not be necessary if REGION remains static, but added to prevent future frustration.)
// [see http://stackoverflow.com/questions/13898057/aws-error-message-a-conflicting-conditional-operation-is-currently-in-progress]
protected final static String BUCKET_NAME = "test-bucket-" + System.currentTimeMillis() + "-" + REGION;
protected final static String BUCKET_NAME = "test-bucket-" + System.currentTimeMillis();
private static AmazonS3 client;
private static AWSKMS kmsClient;
private final List<String> addedKeys = new ArrayList<>();
private static final DockerImageName localstackImage = DockerImageName.parse("localstack/localstack:latest");
private static final LocalStackContainer localstack = new LocalStackContainer(localstackImage)
.withServices(LocalStackContainer.Service.S3, LocalStackContainer.Service.KMS);
// Static so multiple Tests can use same client
protected static AmazonS3Client client;
protected static AWSKMS kmsClient;
@BeforeAll
public static void oneTimeSetup() {
// Creates a client and bucket for this test
localstack.start();
final FileInputStream fis;
try {
fis = new FileInputStream(CREDENTIALS_FILE);
} catch (FileNotFoundException e1) {
fail("Could not open credentials file " + CREDENTIALS_FILE + ": " + e1.getLocalizedMessage());
client = AmazonS3ClientBuilder.standard()
.withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(localstack.getEndpoint().toString(), localstack.getRegion()))
.withCredentials(new AWSStaticCredentialsProvider(new BasicAWSCredentials(localstack.getAccessKey(), localstack.getSecretKey())))
.build();
kmsClient = AWSKMSClient.builder()
.withEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(localstack.getEndpoint().toString(), localstack.getRegion()))
.withCredentials(new AWSStaticCredentialsProvider(new BasicAWSCredentials(localstack.getAccessKey(), localstack.getSecretKey())))
.build();
final CreateBucketRequest request = new CreateBucketRequest(BUCKET_NAME);
client.createBucket(request);
}
protected AmazonS3 getClient() {
return client;
}
protected AWSKMS getKmsClient() {
return kmsClient;
}
protected String getEndpointOverride() {
return localstack.getEndpointOverride(LocalStackContainer.Service.S3).toString();
}
protected static String getRegion() {
return localstack.getRegion();
}
protected static void setSecureProperties(final TestRunner runner, final PropertyDescriptor serviceDescriptor) throws InitializationException {
if (runner.getProcessContext().getProperty(serviceDescriptor).isSet()) {
return;
}
try {
final PropertiesCredentials credentials = new PropertiesCredentials(fis);
client = new AmazonS3Client(credentials);
kmsClient = new AWSKMSClient(credentials);
kmsClient.setRegion(Region.getRegion(Regions.fromName(REGION)));
if (client.doesBucketExist(BUCKET_NAME)) {
fail("Bucket " + BUCKET_NAME + " exists. Choose a different bucket name to continue test");
final AWSCredentialsProviderControllerService creds = new AWSCredentialsProviderControllerService();
runner.addControllerService("creds", creds);
runner.setProperty(CredentialPropertyDescriptors.ACCESS_KEY_ID, localstack.getAccessKey());
runner.setProperty(CredentialPropertyDescriptors.SECRET_KEY, localstack.getSecretKey());
runner.enableControllerService(creds);
runner.setProperty(serviceDescriptor, "creds");
}
CreateBucketRequest request = REGION.contains("east")
? new CreateBucketRequest(BUCKET_NAME) // See https://github.com/boto/boto3/issues/125
: new CreateBucketRequest(BUCKET_NAME, REGION);
client.createBucket(request);
} catch (final AmazonS3Exception e) {
fail("Can't create the key " + BUCKET_NAME + ": " + e.getLocalizedMessage());
} catch (final IOException e) {
fail("Caught IOException preparing tests: " + e.getLocalizedMessage());
} finally {
FileUtils.closeQuietly(fis);
}
if (!client.doesBucketExist(BUCKET_NAME)) {
fail("Setup incomplete, tests will fail");
}
@BeforeEach
public void clearKeys() {
addedKeys.clear();
}
@AfterAll
@ -144,14 +166,13 @@ public abstract class AbstractS3IT {
System.err.println("Unable to delete bucket " + BUCKET_NAME + e.toString());
}
if (client.doesBucketExist(BUCKET_NAME)) {
if (client.doesBucketExistV2(BUCKET_NAME)) {
fail("Incomplete teardown, subsequent tests might fail");
}
}
protected void putTestFile(String key, File file) throws AmazonS3Exception {
PutObjectRequest putRequest = new PutObjectRequest(BUCKET_NAME, key, file);
client.putObject(putRequest);
}
@ -171,10 +192,27 @@ public abstract class AbstractS3IT {
client.putObject(putRequest);
}
protected void waitForFilesAvailable() {
for (final String key : addedKeys) {
final long maxWaitTimestamp = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10L);
while (System.currentTimeMillis() < maxWaitTimestamp) {
try {
client.getObject(BUCKET_NAME, key);
} catch (final Exception e) {
try {
Thread.sleep(100L);
} catch (final InterruptedException ie) {
throw new AssertionError("Interrupted while waiting for files to become available", e);
}
}
}
}
}
protected void putFileWithObjectTag(String key, File file, List<Tag> objectTags) {
PutObjectRequest putRequest = new PutObjectRequest(BUCKET_NAME, key, file);
putRequest.setTagging(new ObjectTagging(objectTags));
PutObjectResult result = client.putObject(putRequest);
client.putObject(putRequest);
}
protected Path getResourcePath(String resourceName) {
@ -210,8 +248,21 @@ public abstract class AbstractS3IT {
return dekResult.getKeyId();
}
protected static void deleteKMSKey(String keyId) {
ScheduleKeyDeletionRequest req = new ScheduleKeyDeletionRequest().withKeyId(keyId).withPendingWindowInDays(7);
kmsClient.scheduleKeyDeletion(req);
protected TestRunner initRunner(final Class<? extends AbstractS3Processor> processorClass) {
TestRunner runner = TestRunners.newTestRunner(processorClass);
try {
setSecureProperties(runner, AbstractS3Processor.AWS_CREDENTIALS_PROVIDER_SERVICE);
} catch (InitializationException e) {
Assertions.fail("Could not set security properties");
}
runner.setProperty(AbstractS3Processor.S3_REGION, getRegion());
runner.setProperty(AbstractS3Processor.ENDPOINT_OVERRIDE, getEndpointOverride());
runner.setProperty(AbstractS3Processor.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
return runner;
}
}

View File

@ -16,13 +16,9 @@
*/
package org.apache.nifi.processors.aws.s3;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
@ -33,15 +29,11 @@ import java.util.Map;
public class ITDeleteS3Object extends AbstractS3IT {
@Test
public void testSimpleDelete() throws IOException {
public void testSimpleDelete() {
// Prepares for this test
putTestFile("delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(DeleteS3Object.S3_REGION, REGION);
runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(DeleteS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "delete-me");
@ -53,16 +45,11 @@ public class ITDeleteS3Object extends AbstractS3IT {
}
@Test
public void testDeleteFolder() throws IOException {
public void testDeleteFolder() {
// Prepares for this test
putTestFile("folder/delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(DeleteS3Object.S3_REGION, REGION);
runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(DeleteS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "folder/delete-me");
runner.enqueue(new byte[0], attrs);
@ -73,24 +60,11 @@ public class ITDeleteS3Object extends AbstractS3IT {
}
@Test
public void testDeleteFolderUsingCredentialsProviderService() throws Throwable {
public void testDeleteFolderUsingCredentialsProviderService() {
// Prepares for this test
putTestFile("folder/delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE, System.getProperty("user.home") + "/aws-credentials.properties");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
runner.setProperty(DeleteS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.setProperty(DeleteS3Object.S3_REGION, REGION);
runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(DeleteS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "folder/delete-me");
runner.enqueue(new byte[0], attrs);
@ -101,15 +75,11 @@ public class ITDeleteS3Object extends AbstractS3IT {
}
@Test
public void testDeleteFolderNoExpressionLanguage() throws IOException {
public void testDeleteFolderNoExpressionLanguage() {
// Prepares for this test
putTestFile("folder/delete-me", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(DeleteS3Object.S3_REGION, REGION);
runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(DeleteS3Object.class);
runner.setProperty(DeleteS3Object.KEY, "folder/delete-me");
final Map<String, String> attrs = new HashMap<>();
@ -122,13 +92,8 @@ public class ITDeleteS3Object extends AbstractS3IT {
}
@Test
public void testTryToDeleteNotExistingFile() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new DeleteS3Object());
runner.setProperty(DeleteS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(DeleteS3Object.S3_REGION, REGION);
runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
public void testTryToDeleteNotExistingFile() {
final TestRunner runner = initRunner(DeleteS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "no-such-a-file");
runner.enqueue(new byte[0], attrs);

View File

@ -17,8 +17,7 @@
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.services.s3.model.ObjectMetadata;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
@ -36,14 +35,10 @@ import java.util.Map;
*/
public class ITFetchS3Object extends AbstractS3IT {
@Test
public void testSimpleGet() throws IOException {
public void testSimpleGet() throws IOException, InitializationException {
putTestFile("test-file", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(FetchS3Object.S3_REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(FetchS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test-file");
@ -59,15 +54,10 @@ public class ITFetchS3Object extends AbstractS3IT {
}
@Test
public void testSimpleGetEncrypted() throws IOException {
public void testSimpleGetEncrypted() throws IOException, InitializationException {
putTestFileEncrypted("test-file", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(FetchS3Object.S3_REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(FetchS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test-file");
runner.enqueue(new byte[0], attrs);
@ -85,20 +75,7 @@ public class ITFetchS3Object extends AbstractS3IT {
public void testFetchS3ObjectUsingCredentialsProviderService() throws Throwable {
putTestFile("test-file", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE, System.getProperty("user.home") + "/aws-credentials.properties");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
runner.setProperty(FetchS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.setProperty(FetchS3Object.S3_REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(FetchS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test-file");
runner.enqueue(new byte[0], attrs);
@ -110,12 +87,12 @@ public class ITFetchS3Object extends AbstractS3IT {
}
@Test
public void testTryToFetchNotExistingFile() {
public void testTryToFetchNotExistingFile() throws InitializationException {
final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(FetchS3Object.S3_REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
setSecureProperties(runner, PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE);
runner.setProperty(FetchS3Object.S3_REGION, getRegion());
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "no-such-a-file");
@ -127,16 +104,11 @@ public class ITFetchS3Object extends AbstractS3IT {
}
@Test
public void testContentsOfFileRetrieved() throws IOException {
public void testContentsOfFileRetrieved() throws IOException, InitializationException {
String key = "folder/1.txt";
putTestFile(key, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(FetchS3Object.S3_REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(FetchS3Object.class);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", key);
runner.enqueue(new byte[0], attrs);

View File

@ -17,11 +17,9 @@
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.services.s3.model.Tag;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Test;
import java.io.FileNotFoundException;
@ -37,18 +35,13 @@ import java.util.Map;
*/
public class ITListS3 extends AbstractS3IT {
@Test
public void testSimpleList() throws IOException {
public void testSimpleList() throws IOException, InitializationException {
putTestFile("a", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
putTestFile("b/c", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
waitForFilesAvailable();
final TestRunner runner = TestRunners.newTestRunner(new ListS3());
runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(ListS3.REGION, REGION);
runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(ListS3.class);
runner.run();
runner.assertAllFlowFilesTransferred(ListS3.REL_SUCCESS, 3);
@ -65,20 +58,7 @@ public class ITListS3 extends AbstractS3IT {
putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
waitForFilesAvailable();
final TestRunner runner = TestRunners.newTestRunner(new ListS3());
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
runner.setProperty(ListS3.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.setProperty(ListS3.REGION, REGION);
runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(ListS3.class);
runner.run();
runner.assertAllFlowFilesTransferred(ListS3.REL_SUCCESS, 3);
@ -95,11 +75,7 @@ public class ITListS3 extends AbstractS3IT {
putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
waitForFilesAvailable();
final TestRunner runner = TestRunners.newTestRunner(new ListS3());
runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(ListS3.REGION, REGION);
runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(ListS3.class);
runner.setProperty(ListS3.DELIMITER, "/");
runner.run();
@ -116,11 +92,7 @@ public class ITListS3 extends AbstractS3IT {
putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
waitForFilesAvailable();
final TestRunner runner = TestRunners.newTestRunner(new ListS3());
runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(ListS3.REGION, REGION);
runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(ListS3.class);
runner.setProperty(ListS3.PREFIX, "b/");
runner.run();
@ -137,11 +109,7 @@ public class ITListS3 extends AbstractS3IT {
putTestFile("d/e", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
waitForFilesAvailable();
final TestRunner runner = TestRunners.newTestRunner(new ListS3());
runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(ListS3.REGION, REGION);
runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
final TestRunner runner = initRunner(ListS3.class);
runner.setProperty(ListS3.PREFIX, "b/");
runner.setProperty(ListS3.USE_VERSIONS, "true");
@ -153,7 +121,7 @@ public class ITListS3 extends AbstractS3IT {
}
@Test
public void testObjectTagsWritten() {
public void testObjectTagsWritten() throws InitializationException {
List<Tag> objectTags = new ArrayList<>();
objectTags.add(new Tag("dummytag1", "dummyvalue1"));
objectTags.add(new Tag("dummytag2", "dummyvalue2"));
@ -161,12 +129,9 @@ public class ITListS3 extends AbstractS3IT {
putFileWithObjectTag("t/fileWithTag", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), objectTags);
waitForFilesAvailable();
final TestRunner runner = TestRunners.newTestRunner(new ListS3());
runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
final TestRunner runner = initRunner(ListS3.class);
runner.setProperty(ListS3.PREFIX, "t/");
runner.setProperty(ListS3.REGION, REGION);
runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty(ListS3.WRITE_OBJECT_TAGS, "true");
runner.run();
@ -183,7 +148,7 @@ public class ITListS3 extends AbstractS3IT {
}
@Test
public void testUserMetadataWritten() throws FileNotFoundException {
public void testUserMetadataWritten() throws FileNotFoundException, InitializationException {
Map<String, String> userMetadata = new HashMap<>();
userMetadata.put("dummy.metadata.1", "dummyvalue1");
userMetadata.put("dummy.metadata.2", "dummyvalue2");
@ -191,12 +156,9 @@ public class ITListS3 extends AbstractS3IT {
putFileWithUserMetadata("m/fileWithUserMetadata", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), userMetadata);
waitForFilesAvailable();
final TestRunner runner = TestRunners.newTestRunner(new ListS3());
runner.setProperty(ListS3.CREDENTIALS_FILE, CREDENTIALS_FILE);
final TestRunner runner = initRunner(ListS3.class);
runner.setProperty(ListS3.PREFIX, "m/");
runner.setProperty(ListS3.REGION, REGION);
runner.setProperty(ListS3.BUCKET, BUCKET_NAME);
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty(ListS3.WRITE_USER_METADATA, "true");
runner.run();
@ -212,12 +174,4 @@ public class ITListS3 extends AbstractS3IT {
flowFiles.assertAttributeEquals("s3.user.metadata.dummy.metadata.2", "dummyvalue2");
}
private void waitForFilesAvailable() {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
}

View File

@ -16,43 +16,29 @@
*/
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.AmazonClientException;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.GetObjectTaggingRequest;
import com.amazonaws.services.s3.model.GetObjectTaggingResult;
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
import com.amazonaws.services.s3.model.MultipartUpload;
import com.amazonaws.services.s3.model.MultipartUploadListing;
import com.amazonaws.services.s3.model.ObjectMetadata;
import com.amazonaws.services.s3.model.PartETag;
import com.amazonaws.services.s3.model.Region;
import com.amazonaws.services.s3.model.StorageClass;
import com.amazonaws.services.s3.model.Tag;
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.processors.aws.s3.encryption.StandardS3EncryptionService;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.MockPropertyValue;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.FileInputStream;
import java.io.FileOutputStream;
@ -67,28 +53,25 @@ import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
/**
* Provides integration level testing with actual AWS S3 resources for {@link PutS3Object} and requires additional configuration and resources to work.
* Makes use of Localstack TestContainer in order to test S3 integrations
*/
public class ITPutS3Object extends AbstractS3IT {
private static final Logger logger = LoggerFactory.getLogger(ITPutS3Object.class);
final static String TEST_ENDPOINT = "https://endpoint.com";
// final static String TEST_TRANSIT_URI = "https://" + BUCKET_NAME + ".endpoint.com";
final static String TEST_PARTSIZE_STRING = "50 mb";
final static Long TEST_PARTSIZE_LONG = 50L * 1024L * 1024L;
final static Long S3_MINIMUM_PART_SIZE = 50L * 1024L * 1024L;
final static Long S3_MAXIMUM_OBJECT_SIZE = 5L * 1024L * 1024L * 1024L;
final static Pattern reS3ETag = Pattern.compile("[0-9a-fA-f]{32,32}(-[0-9]+)?");
final static Pattern reS3ETag = Pattern.compile("[0-9a-fA-f]{32}(-[0-9]+)?");
private static String kmsKeyId = "";
@ -105,12 +88,6 @@ public class ITPutS3Object extends AbstractS3IT {
kmsKeyId = getKMSKey();
}
@AfterAll
public static void teardownClass() {
if (StringUtils.isNotEmpty(kmsKeyId)) {
deleteKMSKey(kmsKeyId);
}
}
@Test
public void testSimplePut() throws IOException {
@ -163,7 +140,7 @@ public class ITPutS3Object extends AbstractS3IT {
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
}
private void testPutThenFetch(String sseAlgorithm) throws IOException {
private void testPutThenFetch(String sseAlgorithm) throws IOException, InitializationException {
// Put
TestRunner runner = initTestRunner();
@ -186,14 +163,8 @@ public class ITPutS3Object extends AbstractS3IT {
}
// Fetch
runner = TestRunners.newTestRunner(new FetchS3Object());
runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(FetchS3Object.S3_REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
runner = initFetchRunner();
runner.enqueue(new byte[0], attrs);
runner.run(1);
runner.assertAllFlowFilesTransferred(FetchS3Object.REL_SUCCESS, 1);
@ -206,37 +177,22 @@ public class ITPutS3Object extends AbstractS3IT {
} else {
ff.assertAttributeNotExists(PutS3Object.S3_SSE_ALGORITHM);
}
}
@Test
public void testPutThenFetchWithoutSSE() throws IOException {
public void testPutThenFetchWithoutSSE() throws IOException, InitializationException {
testPutThenFetch(PutS3Object.NO_SERVER_SIDE_ENCRYPTION);
}
@Test
public void testPutThenFetchWithSSE() throws IOException {
public void testPutThenFetchWithSSE() throws IOException, InitializationException {
testPutThenFetch(ObjectMetadata.AES_256_SERVER_SIDE_ENCRYPTION);
}
@Test
public void testPutS3ObjectUsingCredentialsProviderService() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(new PutS3Object());
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSCredentialsProviderProcessor.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
runner.setProperty(PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
final TestRunner runner = initTestRunner();
assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
for (int i = 0; i < 3; i++) {
@ -247,21 +203,15 @@ public class ITPutS3Object extends AbstractS3IT {
runner.run(3);
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 3);
}
@Test
public void testMetaData() throws IOException {
PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
public void testMetaData() throws IOException, InitializationException {
final TestRunner runner = initTestRunner();
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
PropertyDescriptor prop1 = processor.getSupportedDynamicPropertyDescriptor("TEST-PROP-1");
runner.setProperty(prop1, "TESTING-1-2-3");
PropertyDescriptor prop2 = processor.getSupportedDynamicPropertyDescriptor("TEST-PROP-2");
runner.setProperty(prop2, "TESTING-4-5-6");
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty("TEST-PROP-1", "TESTING-1-2-3");
runner.setProperty("TEST-PROP-2", "TESTING-4-5-6");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "meta.txt");
@ -308,7 +258,7 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
public void testContentDispositionNull() throws IOException {
public void testContentDispositionNull() throws IOException, InitializationException {
// Put
TestRunner runner = initTestRunner();
@ -320,14 +270,8 @@ public class ITPutS3Object extends AbstractS3IT {
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
// Fetch
runner = TestRunners.newTestRunner(new FetchS3Object());
runner.setProperty(FetchS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(FetchS3Object.S3_REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
runner = initFetchRunner();
runner.enqueue(new byte[0], attrs);
runner.run(1);
runner.assertAllFlowFilesTransferred(FetchS3Object.REL_SUCCESS, 1);
@ -339,17 +283,14 @@ public class ITPutS3Object extends AbstractS3IT {
@Test
public void testContentDispositionAttachment() throws IOException {
TestRunner runner = initTestRunner();
final TestRunner runner = initTestRunner();
runner.setProperty(PutS3Object.CONTENT_DISPOSITION, PutS3Object.CONTENT_DISPOSITION_ATTACHMENT);
runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
runner.run();
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
MockFlowFile ff1 = flowFiles.get(0);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
final MockFlowFile ff1 = flowFiles.get(0);
ff1.assertAttributeEquals(PutS3Object.S3_CONTENT_DISPOSITION, "attachment; filename=\"hello.txt\"");
}
@ -384,64 +325,6 @@ public class ITPutS3Object extends AbstractS3IT {
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
}
@Test
public void testStorageClasses() throws IOException {
TestRunner runner = initTestRunner();
assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
for (StorageClass storageClass : StorageClass.values()) {
if (storageClass == StorageClass.Outposts) {
// Outposts storage class cannot be tested on AWS cloud infrastructure
continue;
}
runner.setProperty(PutS3Object.STORAGE_CLASS, storageClass.name());
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "testStorageClasses/small_" + storageClass.name() + ".txt");
runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs);
runner.run();
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
FlowFile file = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS).get(0);
assertEquals(storageClass.toString(), file.getAttribute(PutS3Object.S3_STORAGECLASS_ATTR_KEY));
runner.clearTransferState();
}
}
@Test
public void testStorageClassesMultipart() {
TestRunner runner = initTestRunner();
runner.setProperty(PutS3Object.MULTIPART_THRESHOLD, "50 MB");
runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, "50 MB");
assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
for (StorageClass storageClass : StorageClass.values()) {
if (storageClass == StorageClass.Outposts) {
// Outposts storage class cannot be tested on AWS cloud infrastructure
continue;
}
runner.setProperty(PutS3Object.STORAGE_CLASS, storageClass.name());
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "testStorageClasses/large_" + storageClass.name() + ".dat");
runner.enqueue(new byte[50 * 1024 * 1024 + 1], attrs);
runner.run();
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
FlowFile file = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS).get(0);
assertEquals(storageClass.toString(), file.getAttribute(PutS3Object.S3_STORAGECLASS_ATTR_KEY));
runner.clearTransferState();
}
}
@Test
public void testPermissions() throws IOException {
@ -459,19 +342,15 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
public void testDynamicProperty() {
public void testDynamicProperty() throws InitializationException {
final String DYNAMIC_ATTRIB_KEY = "fs.runTimestamp";
final String DYNAMIC_ATTRIB_VALUE = "${now():toNumber()}";
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final TestRunner runner = initTestRunner();
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
PropertyDescriptor testAttrib = processor.getSupportedDynamicPropertyDescriptor(DYNAMIC_ATTRIB_KEY);
runner.setProperty(testAttrib, DYNAMIC_ATTRIB_VALUE);
runner.setProperty(DYNAMIC_ATTRIB_KEY, DYNAMIC_ATTRIB_VALUE);
final String FILE1_NAME = "file1";
Map<String, String> attribs = new HashMap<>();
@ -479,7 +358,6 @@ public class ITPutS3Object extends AbstractS3IT {
runner.enqueue("123".getBytes(), attribs);
runner.assertValid();
processor.getPropertyDescriptor(DYNAMIC_ATTRIB_KEY);
runner.run();
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
@ -498,15 +376,14 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
public void testProvenance() {
public void testProvenance() throws InitializationException {
final String PROV1_FILE = "provfile1";
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final TestRunner runner = initTestRunner();
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
setSecureProperties(runner, PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE);
runner.setProperty(PutS3Object.S3_REGION, getRegion());
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty(PutS3Object.KEY, "${filename}");
Map<String, String> attribs = new HashMap<>();
@ -523,9 +400,8 @@ public class ITPutS3Object extends AbstractS3IT {
assertEquals(1, provenanceEvents.size());
ProvenanceEventRecord provRec1 = provenanceEvents.get(0);
assertEquals(ProvenanceEventType.SEND, provRec1.getEventType());
assertEquals(processor.getIdentifier(), provRec1.getComponentId());
client.setRegion(Region.fromValue(REGION).toAWSRegion());
String targetUri = client.getUrl(BUCKET_NAME, PROV1_FILE).toString();
assertEquals(runner.getProcessor().getIdentifier(), provRec1.getComponentId());
String targetUri = getClient().getUrl(BUCKET_NAME, PROV1_FILE).toString();
assertEquals(targetUri, provRec1.getTransitUri());
assertEquals(8, provRec1.getUpdatedAttributes().size());
assertEquals(BUCKET_NAME, provRec1.getUpdatedAttributes().get(PutS3Object.S3_BUCKET_KEY));
@ -559,42 +435,16 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
public void testEndpointOverride() {
// remove leading "/" from filename to avoid duplicate separators
final String TESTKEY = AbstractS3IT.SAMPLE_FILE_RESOURCE_NAME.substring(1);
final PutS3Object processor = new TestablePutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
public void testMultipartProperties() throws InitializationException {
final TestRunner runner = initTestRunner();
final ProcessContext context = runner.getProcessContext();
runner.setProperty(PutS3Object.ENDPOINT_OVERRIDE, TEST_ENDPOINT);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(PutS3Object.KEY, TESTKEY);
runner.run();
assertEquals(BUCKET_NAME, context.getProperty(PutS3Object.BUCKET).toString());
assertEquals(TESTKEY, context.getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).toString());
assertEquals(TEST_ENDPOINT, context.getProperty(PutS3Object.ENDPOINT_OVERRIDE).toString());
String s3url = ((TestablePutS3Object)processor).testable_getClient(context).getResourceUrl(BUCKET_NAME, TESTKEY);
assertEquals(TEST_ENDPOINT + "/" + BUCKET_NAME + "/" + TESTKEY, s3url);
}
@Test
public void testMultipartProperties() {
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final ProcessContext context = runner.getProcessContext();
runner.setProperty(PutS3Object.FULL_CONTROL_USER_LIST,
"28545acd76c35c7e91f8409b95fd1aa0c0914bfa1ac60975d9f48bc3c5e090b5");
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.FULL_CONTROL_USER_LIST, "28545acd76c35c7e91f8409b95fd1aa0c0914bfa1ac60975d9f48bc3c5e090b5");
runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty(PutS3Object.KEY, AbstractS3IT.SAMPLE_FILE_RESOURCE_NAME);
assertEquals(BUCKET_NAME, context.getProperty(PutS3Object.BUCKET).toString());
assertEquals(BUCKET_NAME, context.getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).toString());
assertEquals(SAMPLE_FILE_RESOURCE_NAME, context.getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).toString());
assertEquals(TEST_PARTSIZE_LONG.longValue(),
context.getProperty(PutS3Object.MULTIPART_PART_SIZE).asDataSize(DataUnit.B).longValue());
@ -602,10 +452,9 @@ public class ITPutS3Object extends AbstractS3IT {
@Test
public void testLocalStatePersistence() throws IOException {
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final TestRunner runner = initTestRunner();
final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET).getValue();
final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).getValue();
final String key = runner.getProcessContext().getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).getValue();
final String cacheKey1 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key;
final String cacheKey2 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-v2";
@ -615,6 +464,7 @@ public class ITPutS3Object extends AbstractS3IT {
* store 3 versions of state
*/
PutS3Object.MultipartState state1orig = new PutS3Object.MultipartState();
final PutS3Object processor = (PutS3Object) runner.getProcessor();
processor.persistLocalState(cacheKey1, state1orig);
PutS3Object.MultipartState state2orig = new PutS3Object.MultipartState();
@ -642,8 +492,8 @@ public class ITPutS3Object extends AbstractS3IT {
uploadList.add(upload3);
final MultipartUploadListing uploadListing = new MultipartUploadListing();
uploadListing.setMultipartUploads(uploadList);
final MockAmazonS3Client mockClient = new MockAmazonS3Client();
mockClient.setListing(uploadListing);
final AmazonS3 mockClient = mock(AmazonS3.class);
when(mockClient.listMultipartUploads(any(ListMultipartUploadsRequest.class))).thenReturn(uploadListing);
/*
* reload and validate stored state
@ -675,10 +525,10 @@ public class ITPutS3Object extends AbstractS3IT {
@Test
public void testStatePersistsETags() throws IOException {
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final TestRunner runner = initTestRunner();
final PutS3Object processor = (PutS3Object) runner.getProcessor();
final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET).getValue();
final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).getValue();
final String key = runner.getProcessContext().getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).getValue();
final String cacheKey1 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-bv1";
final String cacheKey2 = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-bv2";
@ -732,8 +582,8 @@ public class ITPutS3Object extends AbstractS3IT {
uploadList.add(upload2);
final MultipartUploadListing uploadListing = new MultipartUploadListing();
uploadListing.setMultipartUploads(uploadList);
final MockAmazonS3Client mockClient = new MockAmazonS3Client();
mockClient.setListing(uploadListing);
final AmazonS3 mockClient = mock(AmazonS3.class);
when(mockClient.listMultipartUploads(any(ListMultipartUploadsRequest.class))).thenReturn(uploadListing);
/*
* load state and validate that
@ -751,10 +601,10 @@ public class ITPutS3Object extends AbstractS3IT {
@Test
public void testStateRemove() throws IOException {
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final TestRunner runner = initTestRunner();
final PutS3Object processor = (PutS3Object) runner.getProcessor();
final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET).getValue();
final String bucket = runner.getProcessContext().getProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE).getValue();
final String key = runner.getProcessContext().getProperty(PutS3Object.KEY).evaluateAttributeExpressions(Collections.emptyMap()).getValue();
final String cacheKey = runner.getProcessor().getIdentifier() + "/" + bucket + "/" + key + "-sr";
@ -765,8 +615,8 @@ public class ITPutS3Object extends AbstractS3IT {
uploadList.add(upload1);
final MultipartUploadListing uploadListing = new MultipartUploadListing();
uploadListing.setMultipartUploads(uploadList);
final MockAmazonS3Client mockClient = new MockAmazonS3Client();
mockClient.setListing(uploadListing);
final AmazonS3 mockClient = mock(AmazonS3.class);
when(mockClient.listMultipartUploads(any(ListMultipartUploadsRequest.class))).thenReturn(uploadListing);
/*
* store state, retrieve and validate, remove and validate
@ -786,7 +636,7 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
public void testMultipartSmallerThanMinimum() throws IOException {
public void testMultipartSmallerThanMinimum() throws IOException, InitializationException {
final String FILE1_NAME = "file1";
final byte[] megabyte = new byte[1024 * 1024];
@ -804,12 +654,9 @@ public class ITPutS3Object extends AbstractS3IT {
assertTrue(megabyte.length < S3_MINIMUM_PART_SIZE);
assertTrue(TEST_PARTSIZE_LONG >= S3_MINIMUM_PART_SIZE && TEST_PARTSIZE_LONG <= S3_MAXIMUM_OBJECT_SIZE);
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final TestRunner runner = initTestRunner();
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
Map<String, String> attributes = new HashMap<>();
@ -833,7 +680,7 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
public void testMultipartBetweenMinimumAndMaximum() throws IOException {
public void testMultipartBetweenMinimumAndMaximum() throws IOException, InitializationException {
final String FILE1_NAME = "file1";
final byte[] megabyte = new byte[1024 * 1024];
@ -849,12 +696,11 @@ public class ITPutS3Object extends AbstractS3IT {
assertTrue(tempByteCount > S3_MINIMUM_PART_SIZE && tempByteCount < S3_MAXIMUM_OBJECT_SIZE);
assertTrue(tempByteCount > TEST_PARTSIZE_LONG);
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final TestRunner runner = initTestRunner();
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
setSecureProperties(runner, PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE);
runner.setProperty(PutS3Object.S3_REGION, getRegion());
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, BUCKET_NAME);
runner.setProperty(PutS3Object.MULTIPART_THRESHOLD, TEST_PARTSIZE_STRING);
runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
@ -878,93 +724,6 @@ public class ITPutS3Object extends AbstractS3IT {
assertEquals(tempByteCount, ff1.getSize());
}
@Test
public void testMultipartLargerThanObjectMaximum() throws IOException {
final String FILE1_NAME = "file1";
final byte[] megabyte = new byte[1024 * 1024];
final Path tempFile = Files.createTempFile("s3multipart", "tmp");
final FileOutputStream tempOut = new FileOutputStream(tempFile.toFile());
for (int i = 0; i < (S3_MAXIMUM_OBJECT_SIZE / 1024 / 1024 + 1); i++) {
tempOut.write(megabyte);
}
tempOut.close();
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, TEST_PARTSIZE_STRING);
Map<String, String> attributes = new HashMap<>();
attributes.put(CoreAttributes.FILENAME.key(), FILE1_NAME);
runner.enqueue(new FileInputStream(tempFile.toFile()), attributes);
runner.assertValid();
runner.run();
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_SUCCESS);
assertEquals(1, successFiles.size());
final List<MockFlowFile> failureFiles = runner.getFlowFilesForRelationship(PutS3Object.REL_FAILURE);
assertEquals(0, failureFiles.size());
MockFlowFile ff1 = successFiles.get(0);
assertEquals(FILE1_NAME, ff1.getAttribute(CoreAttributes.FILENAME.key()));
assertEquals(BUCKET_NAME, ff1.getAttribute(PutS3Object.S3_BUCKET_KEY));
assertEquals(FILE1_NAME, ff1.getAttribute(PutS3Object.S3_OBJECT_KEY));
assertTrue(reS3ETag.matcher(ff1.getAttribute(PutS3Object.S3_ETAG_ATTR_KEY)).matches());
assertTrue(ff1.getSize() > S3_MAXIMUM_OBJECT_SIZE);
}
@Test
public void testS3MultipartAgeoff() throws InterruptedException, IOException {
final PutS3Object processor = new PutS3Object();
final TestRunner runner = TestRunners.newTestRunner(processor);
final ProcessContext context = runner.getProcessContext();
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
// set check interval and age off to minimum values
runner.setProperty(PutS3Object.MULTIPART_S3_AGEOFF_INTERVAL, "1 milli");
runner.setProperty(PutS3Object.MULTIPART_S3_MAX_AGE, "1 milli");
// create some dummy uploads
for (int i = 0; i < 3; i++) {
final InitiateMultipartUploadRequest initiateRequest = new InitiateMultipartUploadRequest(
BUCKET_NAME, "file" + i + ".txt");
assertDoesNotThrow(() -> client.initiateMultipartUpload(initiateRequest));
}
// Age off is time dependent, so test has some timing constraints. This
// sleep() delays long enough to satisfy interval and age intervals.
Thread.sleep(2000L);
// System millis are used for timing, but it is incremented on each
// call to circumvent what appears to be caching in the AWS library.
// The increments are 1000 millis because AWS returns upload
// initiation times in whole seconds.
long now = System.currentTimeMillis();
MultipartUploadListing uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now, BUCKET_NAME);
assertEquals(3, uploadList.getMultipartUploads().size());
MultipartUpload upload0 = uploadList.getMultipartUploads().get(0);
processor.abortS3MultipartUpload(client, BUCKET_NAME, upload0);
uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+1000, BUCKET_NAME);
assertEquals(2, uploadList.getMultipartUploads().size());
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test-upload.txt");
runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs);
runner.run();
uploadList = processor.getS3AgeoffListAndAgeoffLocalState(context, client, now+2000, BUCKET_NAME);
assertEquals(0, uploadList.getMultipartUploads().size());
}
@Test
public void testObjectTags() throws IOException {
@ -981,7 +740,7 @@ public class ITPutS3Object extends AbstractS3IT {
runner.run();
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
GetObjectTaggingResult result = client.getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, "tag-test.txt"));
GetObjectTaggingResult result = getClient().getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, "tag-test.txt"));
List<Tag> objectTags = result.getTagSet();
for (Tag tag : objectTags) {
@ -1028,12 +787,14 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
@Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
public void testEncryptionServiceWithServerSideKMSEncryptionStrategyUsingSingleUpload() throws IOException, InitializationException {
byte[] smallData = Files.readAllBytes(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
testEncryptionServiceWithServerSideKMSEncryptionStrategy(smallData);
}
@Test
@Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
public void testEncryptionServiceWithServerSideKMSEncryptionStrategyUsingMultipartUpload() throws IOException, InitializationException {
byte[] largeData = new byte[51 * 1024 * 1024];
testEncryptionServiceWithServerSideKMSEncryptionStrategy(largeData);
@ -1062,12 +823,14 @@ public class ITPutS3Object extends AbstractS3IT {
}
@Test
@Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
public void testEncryptionServiceWithServerSideCEncryptionStrategyUsingSingleUpload() throws IOException, InitializationException {
byte[] smallData = Files.readAllBytes(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
testEncryptionServiceWithServerSideCEncryptionStrategy(smallData);
}
@Test
@Disabled("HTTPS must be used when sending customer encryption keys (SSE-C) to S3, in order to protect your encryption keys but doesn't appear to be supported with Localstack TestContainer.")
public void testEncryptionServiceWithServerSideCEncryptionStrategyUsingMultipartUpload() throws IOException, InitializationException {
byte[] largeData = new byte[51 * 1024 * 1024];
testEncryptionServiceWithServerSideCEncryptionStrategy(largeData);
@ -1097,17 +860,6 @@ public class ITPutS3Object extends AbstractS3IT {
flowFile.assertAttributeEquals(PutS3Object.S3_ENCRYPTION_STRATEGY, AmazonS3EncryptionService.STRATEGY_NAME_SSE_C);
}
@Test
public void testEncryptionServiceWithClientSideKMSEncryptionStrategyUsingSingleUpload() throws IOException, InitializationException {
byte[] smallData = Files.readAllBytes(getResourcePath(SAMPLE_FILE_RESOURCE_NAME));
testEncryptionServiceWithClientSideKMSEncryptionStrategy(smallData);
}
@Test
public void testEncryptionServiceWithClientSideKMSEncryptionStrategyUsingMultipartUpload() throws IOException, InitializationException {
byte[] largeData = new byte[51 * 1024 * 1024];
testEncryptionServiceWithClientSideKMSEncryptionStrategy(largeData);
}
private void testEncryptionServiceWithClientSideKMSEncryptionStrategy(byte[] data) throws InitializationException, IOException {
TestRunner runner = createPutEncryptionTestRunner(AmazonS3EncryptionService.STRATEGY_NAME_CSE_KMS, kmsKeyId);
@ -1162,16 +914,10 @@ public class ITPutS3Object extends AbstractS3IT {
MockFlowFile flowFile = fetchEncryptedFlowFile(attrs, AmazonS3EncryptionService.STRATEGY_NAME_CSE_C, randomKeyMaterial);
flowFile.assertAttributeEquals(PutS3Object.S3_ENCRYPTION_STRATEGY, AmazonS3EncryptionService.STRATEGY_NAME_CSE_C);
flowFile.assertContentEquals(data);
flowFile.assertAttributeExists("x-amz-key");
flowFile.assertAttributeNotEquals("x-amz-key", "");
flowFile.assertAttributeExists("x-amz-iv");
flowFile.assertAttributeNotEquals("x-amz-iv", "");
}
private static TestRunner createPutEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
TestRunner runner = createEncryptionTestRunner(new PutS3Object(), strategyName, keyIdOrMaterial);
private TestRunner createPutEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
TestRunner runner = createEncryptionTestRunner(PutS3Object.class, strategyName, keyIdOrMaterial);
runner.setProperty(PutS3Object.MULTIPART_THRESHOLD, "50 MB");
runner.setProperty(PutS3Object.MULTIPART_PART_SIZE, "50 MB");
@ -1179,15 +925,15 @@ public class ITPutS3Object extends AbstractS3IT {
return runner;
}
private static TestRunner createFetchEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
private TestRunner createFetchEncryptionTestRunner(String strategyName, String keyIdOrMaterial) throws InitializationException {
if (strategyName.equals(AmazonS3EncryptionService.STRATEGY_NAME_SSE_S3) || strategyName.equals(AmazonS3EncryptionService.STRATEGY_NAME_SSE_KMS)) {
strategyName = null;
}
return createEncryptionTestRunner(new FetchS3Object(), strategyName, keyIdOrMaterial);
return createEncryptionTestRunner(FetchS3Object.class, strategyName, keyIdOrMaterial);
}
private static MockFlowFile fetchEncryptedFlowFile(Map<String, String> attributes, String strategyName, String keyIdOrMaterial) throws InitializationException {
private MockFlowFile fetchEncryptedFlowFile(Map<String, String> attributes, String strategyName, String keyIdOrMaterial) throws InitializationException {
final TestRunner runner = createFetchEncryptionTestRunner(strategyName, keyIdOrMaterial);
runner.enqueue(new byte[0], attributes);
runner.run(1);
@ -1196,13 +942,8 @@ public class ITPutS3Object extends AbstractS3IT {
return flowFiles.get(0);
}
private static TestRunner createEncryptionTestRunner(Processor processor, String strategyName, String keyIdOrMaterial) throws InitializationException {
final TestRunner runner = TestRunners.newTestRunner(processor);
final ConfigurationContext context = mock(ConfigurationContext.class);
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
private TestRunner createEncryptionTestRunner(final Class<? extends AbstractS3Processor> processorClass, String strategyName, String keyIdOrMaterial) throws InitializationException {
final TestRunner runner = initRunner(processorClass);
if (strategyName != null) {
final StandardS3EncryptionService service = new StandardS3EncryptionService();
@ -1211,37 +952,18 @@ public class ITPutS3Object extends AbstractS3IT {
runner.setProperty(service, StandardS3EncryptionService.ENCRYPTION_STRATEGY, strategyName);
runner.setProperty(service, StandardS3EncryptionService.ENCRYPTION_VALUE, keyIdOrMaterial);
runner.setProperty(service, StandardS3EncryptionService.KMS_REGION, REGION);
runner.setProperty(service, StandardS3EncryptionService.KMS_REGION, getRegion());
when(context.getProperty(StandardS3EncryptionService.ENCRYPTION_STRATEGY)).thenReturn(new MockPropertyValue(strategyName));
when(context.getProperty(StandardS3EncryptionService.ENCRYPTION_VALUE)).thenReturn(new MockPropertyValue(keyIdOrMaterial));
when(context.getProperty(StandardS3EncryptionService.KMS_REGION)).thenReturn(new MockPropertyValue(REGION));
runner.setProperty(service, StandardS3EncryptionService.ENCRYPTION_STRATEGY, strategyName);
runner.setProperty(service, StandardS3EncryptionService.ENCRYPTION_VALUE, keyIdOrMaterial);
runner.setProperty(service, StandardS3EncryptionService.KMS_REGION, getRegion());
service.onConfigured(context);
runner.enableControllerService(service);
}
return runner;
}
private class MockAmazonS3Client extends AmazonS3Client {
MultipartUploadListing listing;
public void setListing(MultipartUploadListing newlisting) {
listing = newlisting;
}
@Override
public MultipartUploadListing listMultipartUploads(ListMultipartUploadsRequest listMultipartUploadsRequest)
throws AmazonClientException {
return listing;
}
}
public class TestablePutS3Object extends PutS3Object {
public AmazonS3Client testable_getClient(ProcessContext context) {
return this.getClient(context);
}
}
@Test
public void testChunkedEncodingDisabled() throws IOException {
@ -1261,14 +983,12 @@ public class ITPutS3Object extends AbstractS3IT {
executeSimplePutTest(runner);
}
private TestRunner initFetchRunner() {
return initRunner(FetchS3Object.class);
}
private TestRunner initTestRunner() {
TestRunner runner = TestRunners.newTestRunner(PutS3Object.class);
runner.setProperty(PutS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(PutS3Object.S3_REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
return runner;
return initRunner(PutS3Object.class);
}
private void executeSimplePutTest(TestRunner runner) throws IOException {

View File

@ -21,11 +21,10 @@ import com.amazonaws.services.s3.model.GetObjectTaggingResult;
import com.amazonaws.services.s3.model.Tag;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertFalse;
@ -37,26 +36,29 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
* configuration and resources to work.
*/
public class ITTagS3Object extends AbstractS3IT {
private static final String TAG_KEY_NIFI = "nifi-key";
private static final String TAG_VALUE_NIFI = "nifi-val";
private static final String TEST_FILE = "test-file";
private static final Tag OLD_TAG = new Tag("oldkey", "oldvalue");
private static final Tag TAG = new Tag(TAG_KEY_NIFI, TAG_VALUE_NIFI);
private TestRunner initRunner() {
final TestRunner runner = initRunner(TagS3Object.class);
runner.setProperty(TagS3Object.TAG_KEY, TAG_KEY_NIFI);
runner.setProperty(TagS3Object.TAG_VALUE, TAG_VALUE_NIFI);
return runner;
}
@Test
public void testSimpleTag() {
String objectKey = "test-file";
String tagKey = "nifi-key";
String tagValue = "nifi-val";
// put file in s3
putTestFile(objectKey, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
putTestFile(TEST_FILE, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
// Set up processor
final TestRunner runner = TestRunners.newTestRunner(new TagS3Object());
runner.setProperty(TagS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(TagS3Object.S3_REGION, REGION);
runner.setProperty(TagS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagValue);
final TestRunner runner = initRunner();
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", objectKey);
attrs.put("filename", TEST_FILE);
runner.enqueue(new byte[0], attrs);
// tag file
@ -64,33 +66,31 @@ public class ITTagS3Object extends AbstractS3IT {
// Verify processor succeeds
runner.assertAllFlowFilesTransferred(TagS3Object.REL_SUCCESS, 1);
assertTagsCorrect(false);
}
private void assertTagsCorrect(final boolean expectOldTag) {
// Verify tag exists on S3 object
GetObjectTaggingResult res = client.getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, objectKey));
assertTrue(res.getTagSet().contains(new Tag(tagKey, tagValue)), "Expected tag not found on S3 object");
GetObjectTaggingResult res = getClient().getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, TEST_FILE));
assertTrue(res.getTagSet().contains(TAG), "Expected tag not found on S3 object");
if (expectOldTag) {
assertTrue(res.getTagSet().contains(OLD_TAG), "Expected existing tag not found on S3 object");
} else {
assertFalse(res.getTagSet().contains(OLD_TAG), "Existing tag found on S3 object");
}
}
@Test
public void testAppendTag() {
String objectKey = "test-file";
String tagKey = "nifi-key";
String tagValue = "nifi-val";
Tag existingTag = new Tag("oldkey", "oldvalue");
// put file in s3
putFileWithObjectTag(objectKey, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), Arrays.asList(existingTag));
putFileWithObjectTag(TEST_FILE, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), List.of(OLD_TAG));
// Set up processor
final TestRunner runner = TestRunners.newTestRunner(new TagS3Object());
runner.setProperty(TagS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(TagS3Object.S3_REGION, REGION);
runner.setProperty(TagS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagValue);
final TestRunner runner = initRunner();
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", objectKey);
attrs.put("filename", TEST_FILE);
runner.enqueue(new byte[0], attrs);
// tag file
@ -98,36 +98,21 @@ public class ITTagS3Object extends AbstractS3IT {
// Verify processor succeeds
runner.assertAllFlowFilesTransferred(TagS3Object.REL_SUCCESS, 1);
// Verify new tag and existing exist on S3 object
GetObjectTaggingResult res = client.getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, objectKey));
assertTrue(res.getTagSet().contains(new Tag(tagKey, tagValue)), "Expected new tag not found on S3 object");
assertTrue(res.getTagSet().contains(existingTag), "Expected existing tag not found on S3 object");
assertTagsCorrect(true);
}
@Test
public void testReplaceTags() {
String objectKey = "test-file";
String tagKey = "nifi-key";
String tagValue = "nifi-val";
Tag existingTag = new Tag("s3.tag.oldkey", "oldvalue");
// put file in s3
putFileWithObjectTag(objectKey, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), Arrays.asList(existingTag));
putFileWithObjectTag(TEST_FILE, getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME), List.of(OLD_TAG));
// Set up processor
final TestRunner runner = TestRunners.newTestRunner(new TagS3Object());
runner.setProperty(TagS3Object.CREDENTIALS_FILE, CREDENTIALS_FILE);
runner.setProperty(TagS3Object.S3_REGION, REGION);
runner.setProperty(TagS3Object.BUCKET, BUCKET_NAME);
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagValue);
final TestRunner runner = initRunner();
runner.setProperty(TagS3Object.APPEND_TAG, "false");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", objectKey);
attrs.put("s3.tag."+existingTag.getKey(), existingTag.getValue());
attrs.put("filename", TEST_FILE);
attrs.put("s3.tag." + OLD_TAG.getKey(), OLD_TAG.getValue());
runner.enqueue(new byte[0], attrs);
// tag file
@ -138,13 +123,10 @@ public class ITTagS3Object extends AbstractS3IT {
// Verify flowfile attributes match s3 tags
MockFlowFile flowFiles = runner.getFlowFilesForRelationship(TagS3Object.REL_SUCCESS).get(0);
flowFiles.assertAttributeNotExists(existingTag.getKey());
flowFiles.assertAttributeEquals("s3.tag."+tagKey, tagValue);
flowFiles.assertAttributeNotExists(OLD_TAG.getKey());
flowFiles.assertAttributeEquals("s3.tag." + TAG_KEY_NIFI, TAG_VALUE_NIFI);
// Verify new tag exists on S3 object and prior tag removed
GetObjectTaggingResult res = client.getObjectTagging(new GetObjectTaggingRequest(BUCKET_NAME, objectKey));
assertTrue(res.getTagSet().contains(new Tag(tagKey, tagValue)), "Expected new tag not found on S3 object");
assertFalse(res.getTagSet().contains(existingTag), "Existing tag not replaced on S3 object");
assertTagsCorrect(false);
}
}

View File

@ -20,9 +20,7 @@ import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.DeleteObjectRequest;
import com.amazonaws.services.s3.model.DeleteVersionRequest;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.proxy.ProxyConfigurationService;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.jupiter.api.BeforeEach;
@ -31,11 +29,9 @@ import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestDeleteS3Object {
@ -59,7 +55,7 @@ public class TestDeleteS3Object {
@Test
public void testDeleteObjectSimple() {
runner.setProperty(DeleteS3Object.S3_REGION, "us-west-2");
runner.setProperty(DeleteS3Object.BUCKET, "test-bucket");
runner.setProperty(DeleteS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "delete-key");
runner.enqueue(new byte[0], attrs);
@ -78,7 +74,7 @@ public class TestDeleteS3Object {
@Test
public void testDeleteObjectSimpleRegionFromFlowFileAttribute() {
runner.setProperty(DeleteS3Object.S3_REGION, "attribute-defined-region");
runner.setProperty(DeleteS3Object.BUCKET, "test-bucket");
runner.setProperty(DeleteS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "delete-key");
attrs.put("s3.region", "us-east-1");
@ -92,7 +88,7 @@ public class TestDeleteS3Object {
@Test
public void testDeleteObjectS3Exception() {
runner.setProperty(DeleteS3Object.S3_REGION, "us-west-2");
runner.setProperty(DeleteS3Object.BUCKET, "test-bucket");
runner.setProperty(DeleteS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "delete-key");
runner.enqueue(new byte[0], attrs);
@ -108,7 +104,7 @@ public class TestDeleteS3Object {
@Test
public void testDeleteVersionSimple() {
runner.setProperty(DeleteS3Object.S3_REGION, "us-west-2");
runner.setProperty(DeleteS3Object.BUCKET, "test-bucket");
runner.setProperty(DeleteS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(DeleteS3Object.VERSION_ID, "test-version");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test-key");
@ -129,7 +125,7 @@ public class TestDeleteS3Object {
@Test
public void testDeleteVersionFromExpressions() {
runner.setProperty(DeleteS3Object.S3_REGION, "us-west-2");
runner.setProperty(DeleteS3Object.BUCKET, "${s3.bucket}");
runner.setProperty(DeleteS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "${s3.bucket}");
runner.setProperty(DeleteS3Object.VERSION_ID, "${s3.version}");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test-key");
@ -149,35 +145,4 @@ public class TestDeleteS3Object {
Mockito.verify(mockS3Client, Mockito.never()).deleteObject(Mockito.any(DeleteObjectRequest.class));
}
@Test
public void testGetPropertyDescriptors() {
DeleteS3Object processor = new DeleteS3Object();
List<PropertyDescriptor> pd = processor.getSupportedPropertyDescriptors();
assertEquals(25, pd.size(), "size should be eq");
assertTrue(pd.contains(processor.ACCESS_KEY));
assertTrue(pd.contains(processor.AWS_CREDENTIALS_PROVIDER_SERVICE));
assertTrue(pd.contains(processor.BUCKET));
assertTrue(pd.contains(processor.CREDENTIALS_FILE));
assertTrue(pd.contains(processor.ENDPOINT_OVERRIDE));
assertTrue(pd.contains(processor.FULL_CONTROL_USER_LIST));
assertTrue(pd.contains(processor.KEY));
assertTrue(pd.contains(processor.OWNER));
assertTrue(pd.contains(processor.READ_ACL_LIST));
assertTrue(pd.contains(processor.READ_USER_LIST));
assertTrue(pd.contains(processor.S3_REGION));
assertTrue(pd.contains(processor.SECRET_KEY));
assertTrue(pd.contains(processor.SIGNER_OVERRIDE));
assertTrue(pd.contains(processor.S3_CUSTOM_SIGNER_CLASS_NAME));
assertTrue(pd.contains(processor.S3_CUSTOM_SIGNER_MODULE_LOCATION));
assertTrue(pd.contains(processor.SSL_CONTEXT_SERVICE));
assertTrue(pd.contains(processor.TIMEOUT));
assertTrue(pd.contains(processor.VERSION_ID));
assertTrue(pd.contains(processor.WRITE_ACL_LIST));
assertTrue(pd.contains(processor.WRITE_USER_LIST));
assertTrue(pd.contains(ProxyConfigurationService.PROXY_CONFIGURATION_SERVICE));
assertTrue(pd.contains(processor.PROXY_HOST));
assertTrue(pd.contains(processor.PROXY_HOST_PORT));
assertTrue(pd.contains(processor.PROXY_USERNAME));
assertTrue(pd.contains(processor.PROXY_PASSWORD));
}
}

View File

@ -19,6 +19,8 @@ package org.apache.nifi.processors.aws.s3;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.SdkClientException;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.GetObjectRequest;
@ -68,7 +70,8 @@ public class TestFetchS3Object {
mockS3Client = mock(AmazonS3Client.class);
mockFetchS3Object = new FetchS3Object() {
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockS3Client;
}
};
@ -78,7 +81,7 @@ public class TestFetchS3Object {
@Test
public void testGetObject() throws IOException {
runner.setProperty(FetchS3Object.S3_REGION, "attribute-defined-region");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
attrs.put("s3.region", "us-west-2");
@ -145,7 +148,7 @@ public class TestFetchS3Object {
@Test
public void testGetObjectWithRequesterPays() throws IOException {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket");
runner.setProperty(FetchS3Object.REQUESTER_PAYS, "true");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
@ -203,7 +206,7 @@ public class TestFetchS3Object {
@Test
public void testGetObjectVersion() throws IOException {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket");
runner.setProperty(FetchS3Object.VERSION_ID, "${s3.version}");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
@ -243,7 +246,7 @@ public class TestFetchS3Object {
@Test
public void testGetObjectExceptionGoesToFailure() {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
runner.enqueue(new byte[0], attrs);
@ -257,7 +260,7 @@ public class TestFetchS3Object {
@Test
public void testFetchObject_FailAdditionalAttributesBucketName() {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket-bad-name");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket-bad-name");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
runner.enqueue(new byte[0], attrs);
@ -286,7 +289,7 @@ public class TestFetchS3Object {
@Test
public void testFetchObject_FailAdditionalAttributesAuthentication() {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket-bad-name");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket-bad-name");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
runner.enqueue(new byte[0], attrs);
@ -310,7 +313,7 @@ public class TestFetchS3Object {
@Test
public void testFetchObject_FailAdditionalAttributesNetworkFailure() {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket-bad-name");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket-bad-name");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
runner.enqueue(new byte[0], attrs);
@ -328,7 +331,7 @@ public class TestFetchS3Object {
@Test
public void testGetObjectReturnsNull() {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
runner.enqueue(new byte[0], attrs);
@ -342,7 +345,7 @@ public class TestFetchS3Object {
@Test
public void testFlowFileAccessExceptionGoesToFailure() {
runner.setProperty(FetchS3Object.S3_REGION, "us-east-1");
runner.setProperty(FetchS3Object.BUCKET, "request-bucket");
runner.setProperty(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "request-bucket");
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "request-key");
runner.enqueue(new byte[0], attrs);
@ -362,7 +365,7 @@ public class TestFetchS3Object {
assertEquals(23, pd.size(), "size should be eq");
assertTrue(pd.contains(FetchS3Object.ACCESS_KEY));
assertTrue(pd.contains(FetchS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE));
assertTrue(pd.contains(FetchS3Object.BUCKET));
assertTrue(pd.contains(FetchS3Object.BUCKET_WITHOUT_DEFAULT_VALUE));
assertTrue(pd.contains(FetchS3Object.CREDENTIALS_FILE));
assertTrue(pd.contains(FetchS3Object.ENDPOINT_OVERRIDE));
assertTrue(pd.contains(FetchS3Object.KEY));

View File

@ -17,7 +17,9 @@
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.GetObjectMetadataRequest;
import com.amazonaws.services.s3.model.GetObjectTaggingRequest;
@ -70,7 +72,8 @@ public class TestListS3 {
mockS3Client = Mockito.mock(AmazonS3Client.class);
final ListS3 mockListS3 = new ListS3() {
@Override
protected AmazonS3Client createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockS3Client;
}
};
@ -81,7 +84,7 @@ public class TestListS3 {
@Test
public void testList() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
Date lastModified = new Date();
ObjectListing objectListing = new ObjectListing();
@ -133,7 +136,7 @@ public class TestListS3 {
@Test
public void testListWithRecords() throws InitializationException {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
final MockRecordWriter recordWriter = new MockRecordWriter(null, false);
runner.addControllerService("record-writer", recordWriter);
@ -184,7 +187,7 @@ public class TestListS3 {
@Test
public void testListWithRequesterPays() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.REQUESTER_PAYS, "true");
Date lastModified = new Date();
@ -230,7 +233,7 @@ public class TestListS3 {
@Test
public void testListWithRequesterPays_invalid() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.USE_VERSIONS, "true"); // requester pays cannot be used with versions
runner.setProperty(ListS3.REQUESTER_PAYS, "true");
@ -240,7 +243,7 @@ public class TestListS3 {
@Test
public void testListVersion2() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.LIST_TYPE, "2");
Date lastModified = new Date();
@ -286,7 +289,7 @@ public class TestListS3 {
@Test
public void testListVersion2WithRequesterPays() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.REQUESTER_PAYS, "true");
runner.setProperty(ListS3.LIST_TYPE, "2");
@ -333,7 +336,7 @@ public class TestListS3 {
@Test
public void testListVersions() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.USE_VERSIONS, "true");
Date lastModified = new Date();
@ -377,7 +380,7 @@ public class TestListS3 {
@Test
public void testListObjectsNothingNew() throws IOException {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
Calendar calendar = Calendar.getInstance();
calendar.set(2017, Calendar.JUNE, 2);
@ -413,7 +416,7 @@ public class TestListS3 {
@Test
public void testListIgnoreByMinAge() throws IOException {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.MIN_AGE, "30 sec");
Date lastModifiedNow = new Date();
@ -464,7 +467,7 @@ public class TestListS3 {
@Test
public void testListIgnoreByMaxAge() throws IOException {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.MAX_AGE, "30 sec");
Date lastModifiedNow = new Date();
Date lastModifiedMinus1Hour = DateUtils.addHours(lastModifiedNow, -1);
@ -512,7 +515,7 @@ public class TestListS3 {
@Test
public void testWriteObjectTags() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.WRITE_OBJECT_TAGS, "true");
Date lastModified = new Date();
@ -539,7 +542,7 @@ public class TestListS3 {
@Test
public void testWriteUserMetadata() {
runner.setProperty(ListS3.REGION, "eu-west-1");
runner.setProperty(ListS3.BUCKET, "test-bucket");
runner.setProperty(ListS3.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(ListS3.WRITE_USER_METADATA, "true");
Date lastModified = new Date();

View File

@ -22,6 +22,9 @@ import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
import com.amazonaws.auth.Signer;
import com.amazonaws.auth.SignerFactory;
import com.amazonaws.auth.SignerParams;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.internal.AWSS3V4Signer;
import com.amazonaws.services.s3.model.AmazonS3Exception;
@ -73,7 +76,8 @@ public class TestPutS3Object {
mockS3Client = mock(AmazonS3Client.class);
putS3Object = new PutS3Object() {
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockS3Client;
}
};
@ -131,7 +135,8 @@ public class TestPutS3Object {
if (!signerType.equals(customSignerValue)) {
runner.setProperty(PutS3Object.SIGNER_OVERRIDE, signerType);
ProcessContext context = runner.getProcessContext();
assertDoesNotThrow(() -> processor.createClient(context, credentialsProvider, config));
assertDoesNotThrow(() -> processor.createClient(context, credentialsProvider,
Region.getRegion(Regions.DEFAULT_REGION), config, null));
}
}
}
@ -204,7 +209,7 @@ public class TestPutS3Object {
private void prepareTest(String filename) {
runner.setProperty(PutS3Object.S3_REGION, "ap-northeast-1");
runner.setProperty(PutS3Object.BUCKET, "test-bucket");
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.assertValid();
Map<String, String> ffAttributes = new HashMap<>();
@ -217,7 +222,7 @@ public class TestPutS3Object {
private void prepareTestWithRegionInAttributes(String filename, String region) {
runner.setProperty(PutS3Object.S3_REGION, "attribute-defined-region");
runner.setProperty(PutS3Object.BUCKET, "test-bucket");
runner.setProperty(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.assertValid();
Map<String, String> ffAttributes = new HashMap<>();
@ -281,7 +286,7 @@ public class TestPutS3Object {
assertEquals(41, pd.size(), "size should be eq");
assertTrue(pd.contains(PutS3Object.ACCESS_KEY));
assertTrue(pd.contains(PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE));
assertTrue(pd.contains(PutS3Object.BUCKET));
assertTrue(pd.contains(PutS3Object.BUCKET_WITHOUT_DEFAULT_VALUE));
assertTrue(pd.contains(PutS3Object.CANNED_ACL));
assertTrue(pd.contains(PutS3Object.CREDENTIALS_FILE));
assertTrue(pd.contains(PutS3Object.ENDPOINT_OVERRIDE));
@ -333,7 +338,7 @@ public class TestPutS3Object {
runner.setProperty(PutS3Object.S3_CUSTOM_SIGNER_CLASS_NAME, CustomS3Signer.class.getName());
ProcessContext context = runner.getProcessContext();
processor.createClient(context, credentialsProvider, config);
processor.createClient(context, credentialsProvider, Region.getRegion(Regions.DEFAULT_REGION), config, null);
final String signerName = config.getSignerOverride();
assertNotNull(signerName);

View File

@ -18,6 +18,8 @@ package org.apache.nifi.processors.aws.s3;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder;
import com.amazonaws.regions.Region;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.GetObjectTaggingResult;
@ -33,6 +35,7 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
@ -55,7 +58,8 @@ public class TestTagS3Object {
mockS3Client = Mockito.mock(AmazonS3Client.class);
TagS3Object mockTagS3Object = new TagS3Object() {
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final Region region, final ClientConfiguration config,
final AwsClientBuilder.EndpointConfiguration endpointConfiguration) {
return mockS3Client;
}
};
@ -67,7 +71,7 @@ public class TestTagS3Object {
final String tagKey = "k";
final String tagVal = "v";
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagVal);
runner.setProperty(TagS3Object.APPEND_TAG, "false");
@ -95,7 +99,7 @@ public class TestTagS3Object {
@Test
public void testTagObjectSimpleRegionFromFlowFileAttribute() {
runner.setProperty(TagS3Object.S3_REGION, "attribute-defined-region");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, "k");
runner.setProperty(TagS3Object.TAG_VALUE, "v");
runner.setProperty(TagS3Object.APPEND_TAG, "false");
@ -115,7 +119,7 @@ public class TestTagS3Object {
final String tagKey = "k";
final String tagVal = "v";
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.VERSION_ID, "test-version");
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagVal);
@ -145,7 +149,7 @@ public class TestTagS3Object {
final String tagKey = "nk";
final String tagVal = "nv";
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagVal);
final Map<String, String> attrs = new HashMap<>();
@ -180,7 +184,7 @@ public class TestTagS3Object {
final String tagKey = "nk";
final String tagVal = "nv";
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagVal);
final Map<String, String> attrs = new HashMap<>();
@ -209,7 +213,7 @@ public class TestTagS3Object {
final String tagKey = "nk";
final String tagVal = "nv";
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagVal);
runner.setProperty(TagS3Object.APPEND_TAG, "false");
@ -244,7 +248,7 @@ public class TestTagS3Object {
final String tagKey = "nk";
final String tagVal = "nv";
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, tagKey);
runner.setProperty(TagS3Object.TAG_VALUE, tagVal);
final Map<String, String> attrs = new HashMap<>();
@ -265,7 +269,7 @@ public class TestTagS3Object {
assertEquals(22, pd.size(), "size should be eq");
assertTrue(pd.contains(TagS3Object.ACCESS_KEY));
assertTrue(pd.contains(TagS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE));
assertTrue(pd.contains(TagS3Object.BUCKET));
assertTrue(pd.contains(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE));
assertTrue(pd.contains(TagS3Object.CREDENTIALS_FILE));
assertTrue(pd.contains(TagS3Object.ENDPOINT_OVERRIDE));
assertTrue(pd.contains(TagS3Object.KEY));
@ -290,7 +294,7 @@ public class TestTagS3Object {
@Test
public void testBucketEvaluatedAsBlank() {
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "${not.existant.attribute}");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "${not.existant.attribute}");
runner.setProperty(TagS3Object.TAG_KEY, "key");
runner.setProperty(TagS3Object.TAG_VALUE, "val");
final Map<String, String> attrs = new HashMap<>();
@ -305,7 +309,7 @@ public class TestTagS3Object {
@Test
public void testTagKeyEvaluatedAsBlank() {
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, "${not.existant.attribute}");
runner.setProperty(TagS3Object.TAG_VALUE, "val");
final Map<String, String> attrs = new HashMap<>();
@ -320,7 +324,7 @@ public class TestTagS3Object {
@Test
public void testTagValEvaluatedAsBlank() {
runner.setProperty(TagS3Object.S3_REGION, "us-west-2");
runner.setProperty(TagS3Object.BUCKET, "test-bucket");
runner.setProperty(TagS3Object.BUCKET_WITHOUT_DEFAULT_VALUE, "test-bucket");
runner.setProperty(TagS3Object.TAG_KEY, "tagKey");
runner.setProperty(TagS3Object.TAG_VALUE, "${not.existant.attribute}");
final Map<String, String> attrs = new HashMap<>();

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.processors.aws.s3.encryption;
import com.amazonaws.regions.Regions;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
import com.amazonaws.services.s3.model.ObjectMetadata;
@ -35,8 +36,6 @@ import static org.junit.jupiter.api.Assertions.assertNull;
public class TestS3EncryptionStrategies {
private String randomKeyMaterial = "";
private String randomKeyId = "mock-key-id";
private String kmsRegion = "us-west-1";
private ObjectMetadata metadata = null;
private PutObjectRequest putObjectRequest = null;
@ -58,32 +57,14 @@ public class TestS3EncryptionStrategies {
uploadPartRequest = new UploadPartRequest();
}
@Test
public void testClientSideKMSEncryptionStrategy() {
S3EncryptionStrategy strategy = new ClientSideKMSEncryptionStrategy();
// This shows that the strategy builds a client:
assertNotNull(strategy.createEncryptionClient(null, null, kmsRegion, randomKeyMaterial));
// This shows that the strategy does not modify the metadata or any of the requests:
assertNull(metadata.getSSEAlgorithm());
assertNull(putObjectRequest.getSSEAwsKeyManagementParams());
assertNull(putObjectRequest.getSSECustomerKey());
assertNull(initUploadRequest.getSSEAwsKeyManagementParams());
assertNull(initUploadRequest.getSSECustomerKey());
assertNull(getObjectRequest.getSSECustomerKey());
assertNull(uploadPartRequest.getSSECustomerKey());
}
@Test
public void testClientSideCEncryptionStrategy() {
S3EncryptionStrategy strategy = new ClientSideCEncryptionStrategy();
// This shows that the strategy builds a client:
assertNotNull(strategy.createEncryptionClient(null, null, null, randomKeyMaterial));
assertNotNull(strategy.createEncryptionClient(builder -> {
builder.withRegion(Regions.DEFAULT_REGION.name());
}, Regions.DEFAULT_REGION.getName(), randomKeyMaterial));
// This shows that the strategy does not modify the metadata or any of the requests:
assertNull(metadata.getSSEAlgorithm());
@ -103,7 +84,7 @@ public class TestS3EncryptionStrategies {
S3EncryptionStrategy strategy = new ServerSideCEncryptionStrategy();
// This shows that the strategy does *not* build a client:
assertNull(strategy.createEncryptionClient(null, null, null, ""));
assertNull(strategy.createEncryptionClient(null, null, ""));
// This shows that the strategy sets the SSE customer key as expected:
strategy.configurePutObjectRequest(putObjectRequest, metadata, randomKeyMaterial);
@ -133,9 +114,10 @@ public class TestS3EncryptionStrategies {
S3EncryptionStrategy strategy = new ServerSideKMSEncryptionStrategy();
// This shows that the strategy does *not* build a client:
assertNull(strategy.createEncryptionClient(null, null, null, null));
assertNull(strategy.createEncryptionClient(null, null, null));
// This shows that the strategy sets the SSE KMS key id as expected:
String randomKeyId = "mock-key-id";
strategy.configurePutObjectRequest(putObjectRequest, metadata, randomKeyId);
assertEquals(randomKeyId, putObjectRequest.getSSEAwsKeyManagementParams().getAwsKmsKeyId());
assertNull(putObjectRequest.getSSECustomerKey());
@ -153,7 +135,7 @@ public class TestS3EncryptionStrategies {
S3EncryptionStrategy strategy = new ServerSideS3EncryptionStrategy();
// This shows that the strategy does *not* build a client:
assertNull(strategy.createEncryptionClient(null, null, null, null));
assertNull(strategy.createEncryptionClient(null, null, null));
// This shows that the strategy sets the SSE algorithm field as expected:
strategy.configurePutObjectRequest(putObjectRequest, metadata, null);
@ -169,7 +151,7 @@ public class TestS3EncryptionStrategies {
S3EncryptionStrategy strategy = new NoOpEncryptionStrategy();
// This shows that the strategy does *not* build a client:
assertNull(strategy.createEncryptionClient(null, null, "", ""));
assertNull(strategy.createEncryptionClient(null, "", ""));
// This shows the request and metadata start with various null objects:
assertNull(metadata.getSSEAlgorithm());

View File

@ -66,7 +66,7 @@ public class TestStandardS3EncryptionService {
@Test
public void testCreateClientReturnsNull() {
assertNull(service.createEncryptionClient(null, null));
assertNull(service.createEncryptionClient(null));
}
@Test

Some files were not shown because too many files have changed in this diff Show More