Merge branch 'master' into NIFI-259

This commit is contained in:
Mark Payne 2016-01-21 09:32:02 -05:00
commit f6ec437bc7
77 changed files with 3529 additions and 893 deletions

View File

@ -211,11 +211,18 @@ public interface FlowFileQueue {
DropFlowFileStatus cancelDropFlowFileRequest(String requestIdentifier);
/**
* <p>
* Initiates a request to obtain a listing of FlowFiles in this queue. This method returns a
* ListFlowFileStatus that can be used to obtain information about the FlowFiles that exist
* within the queue. Additionally, the ListFlowFileStatus provides a request identifier that
* can then be passed to the {@link #getListFlowFileStatus(String)}. The listing of FlowFiles
* will be returned ordered by the position of the FlowFile in the queue.
* </p>
*
* <p>
* Note that if maxResults is larger than the size of the "active queue" (i.e., the un-swapped queued,
* FlowFiles that are swapped out will not be returned.)
* </p>
*
* @param requestIdentifier the identifier of the List FlowFile Request
* @param maxResults the maximum number of FlowFileSummary objects to add to the ListFlowFileStatus
@ -227,45 +234,6 @@ public interface FlowFileQueue {
*/
ListFlowFileStatus listFlowFiles(String requestIdentifier, int maxResults);
/**
* Initiates a request to obtain a listing of FlowFiles in this queue. This method returns a
* ListFlowFileStatus that can be used to obtain information about the FlowFiles that exist
* within the queue. Additionally, the ListFlowFileStatus provides a request identifier that
* can then be passed to the {@link #getListFlowFileStatus(String)}
*
* @param requestIdentifier the identifier of the List FlowFile Request
* @param maxResults the maximum number of FlowFileSummary objects to add to the ListFlowFileStatus
* @param sortColumn specifies which column to sort on
* @param direction specifies which direction to sort the FlowFiles
*
* @return the status for the request
*
* @throws IllegalStateException if either the source or the destination of the connection to which this queue belongs
* is currently running.
*/
ListFlowFileStatus listFlowFiles(String requestIdentifier, int maxResults, SortColumn sortColumn, SortDirection direction);
/**
* Initiates a request to obtain a listing of FlowFiles in this queue. This method returns a
* ListFlowFileStatus that can be used to obtain information about the FlowFiles that exist
* within the queue. Additionally, the ListFlowFileStatus provides a request identifier that
* can then be passed to the {@link #getListFlowFileStatus(String)}
*
* @param requestIdentifier the identifier of the List FlowFile Request
* @param maxResults the maximum number of FlowFileSummary objects to add to the ListFlowFileStatus
* @param query an Expression Language expression that will be evaluated against all FlowFiles. Only FlowFiles that satisfy the expression will
* be included in the results. The expression must be a valid expression and return a Boolean type
* @param sortColumn specifies which column to sort on
* @param direction specifies which direction to sort the FlowFiles
*
* @return the status for the request
*
* @throws IllegalStateException if either the source or the destination of the connection to which this queue belongs
* is currently running.
* @throws IllegalArgumentException if query is not a valid Expression Language expression or does not return a boolean type
*/
ListFlowFileStatus listFlowFiles(String requestIdentifier, int maxResults, String query, SortColumn sortColumn, SortDirection direction);
/**
* Returns the current status of a List FlowFile Request that was initiated via the {@link #listFlowFiles(String)}
* method that has the given identifier

View File

@ -43,16 +43,6 @@ public interface ListFlowFileStatus {
*/
long getLastUpdated();
/**
* @return the column on which the listing is sorted
*/
SortColumn getSortColumn();
/**
* @return the direction in which the FlowFiles are sorted
*/
SortDirection getSortDirection();
/**
* @return the current state of the operation
*/
@ -77,14 +67,4 @@ public interface ListFlowFileStatus {
* @return the percentage (an integer between 0 and 100, inclusive) of how close the request is to being completed
*/
int getCompletionPercentage();
/**
* @return the total number of steps that are required in order to finish the listing
*/
int getTotalStepCount();
/**
* @return the total number of steps that have already been completed. The value returned will be >= 0 and <= the result of calling {@link #getTotalStepCount()}.
*/
int getCompletedStepCount();
}

View File

@ -471,6 +471,38 @@ For details see http://www.abeautifulsite.net/
FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
OTHER DEALINGS IN THE SOFTWARE.
This product bundles 'jsoup' which is available under the MIT License.
For details see http://jsoup.org/
jsoup License
The jsoup code-base (include source and compiled packages) are distributed under the open source MIT license as described below.
The MIT License
Copyright © 2009 - 2013 Jonathan Hedley (jonathan@hedley.net)
Permission is hereby granted, free of charge, to any person
obtaining a copy of this software and associated documentation
files (the "Software"), to deal in the Software without
restriction, including without limitation the rights to use,
copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the
Software is furnished to do so, subject to the following
conditions:
The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
OTHER DEALINGS IN THE SOFTWARE.
This product bundles 'json2.js' which is available in the 'public domain'.
For details see https://github.com/douglascrockford/JSON-js
@ -1052,4 +1084,30 @@ information can be found here: http://www.adobe.com/devnet/xmp/library/eula-xmp-
OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR
OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
THE POSSIBILITY OF SUCH DAMAGE.
THE POSSIBILITY OF SUCH DAMAGE.
This product bundles 'Jsoup' which is available under "The MIT license". More
information can be found here: http://jsoup.org/license
The MIT License
Copyright (c) 2009-2015, Jonathan Hedley <jonathan@hedley.net>
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in
all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.

View File

@ -162,6 +162,11 @@ language governing permissions and limitations under the License. -->
<artifactId>nifi-http-context-map-nar</artifactId>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-html-nar</artifactId>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kite-nar</artifactId>

Binary file not shown.

After

Width:  |  Height:  |  Size: 549 B

View File

@ -1260,19 +1260,23 @@ image:iconNotSecure.png["Not Secure"]
[[Queue_Listing]]
=== Listing FlowFiles in a Queue
[[Queue_Interaction]]
=== Queue Interaction
The FlowFiles enqueued in a Connection can be viewed when necessary. The Queue listing is opened via a menu item in
a Connection's context menu. This option is only available when the source and destination of the Connection have
been stopped and all active threads have completed. The listing will return the top 100 FlowFiles according to
the currently sorted column.
a Connection's context menu. The listing will return the top 100 FlowFiles in the active queue according to the
configured priority. The listing can be performed even if the source and destination are actively running.
Additionally, details for a Flowfile in the listing can be viewed by clicking on the Details icon (
image:iconDetails.png["Details"]
) in the left most column. From here, the FlowFile details and attributes are available as well buttons for
downloading or viewing the content. Viewing the content is only available if the nifi.content.viewer.url has been configured.
If the source or destination of the Connection are actively running, there is a chance that the desired FlowFile will
no longer be in the active queue.
The FlowFiles enqueued in a Connection can also be deleted when necessary. The removal of the FlowFiles is initiated
via a menu item in the Connection's context menu. This action can also be performed if the source and destination
are actively running.
[[Summary_Page]]

View File

@ -63,6 +63,7 @@
<configuration>
<excludes combine.children="append">
<exclude>src/test/resources/hello.txt</exclude>
<exclude>src/test/resources/mock-aws-credentials.properties</exclude>
</excludes>
</configuration>
</plugin>

View File

@ -0,0 +1,102 @@
/*
* 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 org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider;
/**
* Base class for aws processors that uses AWSCredentialsProvider interface for creating aws clients.
*
* @param <ClientType> client type
*
* @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> {
/**
* 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")
.description("The Controller Service that is used to obtain aws credentials provider")
.required(false)
.identifiesControllerService(AWSCredentialsProviderService.class)
.build();
/**
* This method checks if {#link {@link #AWS_CREDENTIALS_PROVIDER_SERVICE} is available and if it
* is, uses the credentials provider, otherwise it invokes the {@link AbstractAWSProcessor#onScheduled(ProcessContext)}
* which uses static AWSCredentials for the aws processors
*/
@OnScheduled
public void onScheduled(ProcessContext context) {
ControllerService service = context.getProperty(AWS_CREDENTIALS_PROVIDER_SERVICE).asControllerService();
if (service != null) {
getLogger().debug("Using aws credentials provider service for creating client");
onScheduledUsingControllerService(context);
} else {
getLogger().debug("Using aws credentials for creating client");
super.onScheduled(context);
}
}
/**
* Create aws client using credentials provider
* @param context the process context
*/
protected void onScheduledUsingControllerService(ProcessContext context) {
final ClientType awsClient = createClient(context, getCredentialsProvider(context), createConfiguration(context));
this.client = awsClient;
super.intializeRegionAndEndpoint(context);
}
/**
* Get credentials provider using the {@link AWSCredentialsProviderService}
* @param context the process context
* @return AWSCredentialsProvider the credential provider
* @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);
return awsCredentialsProviderService.getCredentialsProvider();
}
/**
* Abstract method to create aws client using credetials provider. This is the preferred method
* for creating aws clients
* @param context process context
* @param credentialsProvider aws credentials provider
* @param config aws client configuraiton
* @return ClientType the client
*/
protected abstract ClientType createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config);
}

View File

@ -53,6 +53,14 @@ import com.amazonaws.http.conn.ssl.SdkTLSSocketFactory;
import com.amazonaws.regions.Region;
import com.amazonaws.regions.Regions;
/**
* 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 AbstractProcessor {
public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
@ -71,14 +79,14 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
.build();
public static final PropertyDescriptor ACCESS_KEY = new PropertyDescriptor.Builder()
.name("Access Key")
.expressionLanguageSupported(false)
.expressionLanguageSupported(true)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
.build();
public static final PropertyDescriptor SECRET_KEY = new PropertyDescriptor.Builder()
.name("Secret Key")
.expressionLanguageSupported(false)
.expressionLanguageSupported(true)
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(true)
@ -113,8 +121,8 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
.addValidator(StandardValidators.URL_VALIDATOR)
.build();
private volatile ClientType client;
private volatile Region region;
protected volatile ClientType client;
protected volatile Region region;
// If protocol is changed to be a property, ensure other uses are also changed
protected static final Protocol DEFAULT_PROTOCOL = Protocol.HTTPS;
@ -181,7 +189,10 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
public void onScheduled(final ProcessContext context) {
final ClientType awsClient = createClient(context, getCredentials(context), createConfiguration(context));
this.client = awsClient;
intializeRegionAndEndpoint(context);
}
protected void intializeRegionAndEndpoint(ProcessContext context) {
// if the processor supports REGION, get the configured region.
if (getSupportedPropertyDescriptors().contains(REGION)) {
final String region = context.getProperty(REGION).getValue();
@ -199,8 +210,19 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
if (!urlstr.isEmpty()) {
this.client.setEndpoint(urlstr);
}
}
/**
* 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);
protected ClientType getClient() {
@ -212,8 +234,8 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
}
protected AWSCredentials getCredentials(final ProcessContext context) {
final String accessKey = context.getProperty(ACCESS_KEY).getValue();
final String secretKey = context.getProperty(SECRET_KEY).getValue();
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();
@ -230,10 +252,7 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
}
return new AnonymousAWSCredentials();
}
protected boolean isEmpty(final String value) {
return value == null || value.trim().equals("");
}
}

View File

@ -0,0 +1,193 @@
/*
* 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.credentials.provider.service;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnEnabled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.reporting.InitializationException;
import static org.apache.nifi.processors.aws.AbstractAWSProcessor.ACCESS_KEY;
import static org.apache.nifi.processors.aws.AbstractAWSProcessor.SECRET_KEY;
import static org.apache.nifi.processors.aws.AbstractAWSProcessor.CREDENTIALS_FILE;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.auth.BasicAWSCredentials;
import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
import com.amazonaws.auth.PropertiesFileCredentialsProvider;
import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
import com.amazonaws.internal.StaticCredentialsProvider;
/**
* Implementation of AWSCredentialsProviderService interface
*
* @see AWSCredentialsProviderService
*/
@CapabilityDescription("Defines credentials for Amazon Web Services processors.")
@Tags({ "aws", "credentials","provider" })
public class AWSCredentialsProviderControllerService extends AbstractControllerService implements AWSCredentialsProviderService {
/**
* AWS Role Arn used for cross account access
*
* @see <a href="http://docs.aws.amazon.com/general/latest/gr/aws-arns-and-namespaces.html#genref-arns">AWS ARN</a>
*/
public static final PropertyDescriptor ASSUME_ROLE_ARN = new PropertyDescriptor.Builder().name("Assume Role ARN")
.expressionLanguageSupported(false).required(false).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(false).description("The AWS Role ARN for cross account access. This is used in conjunction with role name and session timeout").build();
/**
* The role name while creating aws role
*/
public static final PropertyDescriptor ASSUME_ROLE_NAME = new PropertyDescriptor.Builder().name("Assume Role Session Name")
.expressionLanguageSupported(false).required(false).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.sensitive(false).description("The aws role name for cross account access. This is used in conjunction with role arn and session time out").build();
/**
* Max session time for role based credentials. The range is between 900 and 3600 seconds.
*/
public static final PropertyDescriptor MAX_SESSION_TIME = new PropertyDescriptor.Builder()
.name("Session Time")
.description("Session time for role based session (between 900 and 3600 seconds). This is used in conjunction with role arn and name")
.defaultValue("3600")
.required(false)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.sensitive(false)
.build();
private static final List<PropertyDescriptor> properties;
static {
final List<PropertyDescriptor> props = new ArrayList<>();
props.add(ACCESS_KEY);
props.add(SECRET_KEY);
props.add(CREDENTIALS_FILE);
props.add(ASSUME_ROLE_ARN);
props.add(ASSUME_ROLE_NAME);
props.add(MAX_SESSION_TIME);
properties = Collections.unmodifiableList(props);
}
private volatile AWSCredentialsProvider credentialsProvider;
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
}
@Override
public AWSCredentialsProvider getCredentialsProvider() throws ProcessException {
return credentialsProvider;
}
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final boolean accessKeySet = validationContext.getProperty(ACCESS_KEY).isSet();
final boolean secretKeySet = validationContext.getProperty(SECRET_KEY).isSet();
final boolean assumeRoleArnIsSet = validationContext.getProperty(ASSUME_ROLE_ARN).isSet();
final boolean assumeRoleNameIsSet = validationContext.getProperty(ASSUME_ROLE_NAME).isSet();
final Integer maxSessionTime = validationContext.getProperty(MAX_SESSION_TIME).asInteger();
final boolean credentialsFileSet = validationContext.getProperty(CREDENTIALS_FILE).isSet();
final Collection<ValidationResult> validationFailureResults = new ArrayList<>();
// both keys are required if present
if ((accessKeySet && !secretKeySet) || (secretKeySet && !accessKeySet)) {
validationFailureResults.add(new ValidationResult.Builder().input("Access Key").valid(false)
.explanation("If setting Secret Key or Access Key, must set both").build());
}
// Either keys or creds file is valid
if ((secretKeySet || accessKeySet) && credentialsFileSet) {
validationFailureResults.add(new ValidationResult.Builder().input("Access Key").valid(false)
.explanation("Cannot set both Credentials File and Secret Key/Access Key").build());
}
// Both role and arn name are req if present
if (assumeRoleArnIsSet ^ assumeRoleNameIsSet ) {
validationFailureResults.add(new ValidationResult.Builder().input("Assume Role Arn and Name")
.valid(false).explanation("Assume role requires both arn and name to be set").build());
}
// Session time only b/w 900 to 3600 sec (see sts session class)
if ( maxSessionTime < 900 || maxSessionTime > 3600 )
validationFailureResults.add(new ValidationResult.Builder().valid(false).input(maxSessionTime + "")
.subject(MAX_SESSION_TIME.getDisplayName() +
" can have value only between 900 and 3600 seconds").build());
return validationFailureResults;
}
@OnEnabled
public void onConfigured(final ConfigurationContext context) throws InitializationException {
final String accessKey = context.getProperty(ACCESS_KEY).evaluateAttributeExpressions().getValue();
final String secretKey = context.getProperty(SECRET_KEY).evaluateAttributeExpressions().getValue();
final String assumeRoleArn = context.getProperty(ASSUME_ROLE_ARN).getValue();
final Integer maxSessionTime = context.getProperty(MAX_SESSION_TIME).asInteger();
final String assumeRoleName = context.getProperty(ASSUME_ROLE_NAME).getValue();
final String credentialsFile = context.getProperty(CREDENTIALS_FILE).getValue();
// Create creds provider from file or keys
if (credentialsFile != null) {
try {
getLogger().debug("Creating properties file credentials provider");
credentialsProvider = new PropertiesFileCredentialsProvider(credentialsFile);
} catch (final Exception ioe) {
throw new ProcessException("Could not read Credentials File", ioe);
}
}
if (credentialsProvider == null && accessKey != null && secretKey != null) {
getLogger().debug("Creating static credentials provider");
credentialsProvider = new StaticCredentialsProvider(new BasicAWSCredentials(accessKey, secretKey));
}
// If no credentials explicitly provided, then create default one
if (credentialsProvider == null) {
getLogger().debug("Creating default credentials provider");
credentialsProvider = new DefaultAWSCredentialsProviderChain();
}
if (credentialsProvider != null && assumeRoleArn != null && assumeRoleName != null) {
getLogger().debug("Creating sts assume role session credentials provider");
credentialsProvider = new STSAssumeRoleSessionCredentialsProvider.Builder(assumeRoleArn, assumeRoleName)
.withLongLivedCredentialsProvider(credentialsProvider)
.withRoleSessionDurationSeconds(maxSessionTime).build();
}
}
@Override
public String toString() {
return "AWSCredentialsProviderService[id=" + getIdentifier() + "]";
}
}

View File

@ -0,0 +1,44 @@
/*
* 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.credentials.provider.service;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.processor.exception.ProcessException;
import com.amazonaws.auth.AWSCredentialsProvider;
/**
* AWSCredentialsProviderService interface to support getting AWSCredentialsProvider used for instantiating
* aws clients
*
* @see <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
*/
@Tags({"aws", "security", "credentials", "provider", "session"})
@CapabilityDescription("Provides AWSCredentialsProvider.")
public interface AWSCredentialsProviderService extends ControllerService {
/**
* Get credentials provider
* @return credentials provider
* @throws ProcessException process exception in case there is problem in getting credentials provider
*
* @see <a href="http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html">AWSCredentialsProvider</a>
*/
public AWSCredentialsProvider getCredentialsProvider() throws ProcessException;
}

View File

@ -25,10 +25,11 @@ import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentials;
import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.regions.Region;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.S3ClientOptions;
@ -39,7 +40,7 @@ import com.amazonaws.services.s3.model.Grantee;
import com.amazonaws.services.s3.model.Owner;
import com.amazonaws.services.s3.model.Permission;
public abstract class AbstractS3Processor extends AbstractAWSProcessor<AmazonS3Client> {
public abstract class AbstractS3Processor extends AbstractAWSCredentialsProviderProcessor<AmazonS3Client> {
public static final PropertyDescriptor FULL_CONTROL_USER_LIST = new PropertyDescriptor.Builder()
.name("FullControl User List")
@ -103,22 +104,47 @@ public abstract class AbstractS3Processor extends AbstractAWSProcessor<AmazonS3C
.defaultValue("${filename}")
.build();
/**
* Create client using credentials provider. This is the preferred way for creating clients
*/
@Override
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
final AmazonS3Client s3 = new AmazonS3Client(credentials, config);
protected AmazonS3Client createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
getLogger().info("Creating client with credentials provider");
final AmazonS3Client s3 = new AmazonS3Client(credentialsProvider, config);
initalizeEndpointOverride(context, s3);
return s3;
}
private void initalizeEndpointOverride(final ProcessContext context, final AmazonS3Client s3) {
// if ENDPOINT_OVERRIDE is set, use PathStyleAccess
if(StringUtils.trimToEmpty(context.getProperty(ENDPOINT_OVERRIDE).getValue()).isEmpty() == false){
final S3ClientOptions s3Options = new S3ClientOptions();
s3Options.setPathStyleAccess(true);
s3.setS3ClientOptions(s3Options);
}
}
/**
* 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");
final AmazonS3Client s3 = new AmazonS3Client(credentials, config);
initalizeEndpointOverride(context, s3);
return s3;
}
protected Grantee createGrantee(final String value) {
if (isEmpty(value)) {
if (StringUtils.isEmpty(value)) {
return null;
}
@ -130,7 +156,7 @@ public abstract class AbstractS3Processor extends AbstractAWSProcessor<AmazonS3C
}
protected final List<Grantee> createGrantees(final String value) {
if (isEmpty(value)) {
if (StringUtils.isEmpty(value)) {
return Collections.emptyList();
}
@ -161,7 +187,7 @@ public abstract class AbstractS3Processor extends AbstractAWSProcessor<AmazonS3C
final AccessControlList acl = new AccessControlList();
final String ownerId = context.getProperty(OWNER).evaluateAttributeExpressions(flowFile).getValue();
if (!isEmpty(ownerId)) {
if (!StringUtils.isEmpty(ownerId)) {
final Owner owner = new Owner();
owner.setId(ownerId);
acl.setOwner(owner);

View File

@ -56,7 +56,7 @@ public class DeleteS3Object extends AbstractS3Processor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, REGION, TIMEOUT, VERSION_ID,
Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, REGION, TIMEOUT, VERSION_ID,
FULL_CONTROL_USER_LIST, READ_USER_LIST, WRITE_USER_LIST, READ_ACL_LIST, WRITE_ACL_LIST, OWNER,
SSL_CONTEXT_SERVICE, ENDPOINT_OVERRIDE));

View File

@ -73,7 +73,7 @@ public class FetchS3Object extends AbstractS3Processor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(BUCKET, KEY, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, TIMEOUT, VERSION_ID, SSL_CONTEXT_SERVICE, ENDPOINT_OVERRIDE));
Arrays.asList(BUCKET, KEY, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT, VERSION_ID, SSL_CONTEXT_SERVICE, ENDPOINT_OVERRIDE));
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {

View File

@ -90,7 +90,7 @@ public class PutS3Object extends AbstractS3Processor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, STORAGE_CLASS, REGION, TIMEOUT, EXPIRATION_RULE_ID,
Arrays.asList(KEY, BUCKET, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, STORAGE_CLASS, REGION, TIMEOUT, EXPIRATION_RULE_ID,
FULL_CONTROL_USER_LIST, READ_USER_LIST, WRITE_USER_LIST, READ_ACL_LIST, WRITE_ACL_LIST, OWNER, SSL_CONTEXT_SERVICE, ENDPOINT_OVERRIDE));
final static String S3_BUCKET_KEY = "s3.bucket";

View File

@ -20,13 +20,14 @@ import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
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.sns.AmazonSNSClient;
public abstract class AbstractSNSProcessor extends AbstractAWSProcessor<AmazonSNSClient> {
public abstract class AbstractSNSProcessor extends AbstractAWSCredentialsProviderProcessor<AmazonSNSClient> {
protected static final AllowableValue ARN_TYPE_TOPIC
= new AllowableValue("Topic ARN", "Topic ARN", "The ARN is the name of a topic");
@ -50,9 +51,25 @@ public abstract class AbstractSNSProcessor extends AbstractAWSProcessor<AmazonSN
.defaultValue(ARN_TYPE_TOPIC.getValue())
.build();
/**
* Create client using aws credentials provider. This is the preferred way for creating clients
*/
@Override
protected AmazonSNSClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
return new AmazonSNSClient(credentials, config);
protected AmazonSNSClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials provider");
return new AmazonSNSClient(credentialsProvider, config);
}
/**
* Create client using AWSCredentails
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Override
protected AmazonSNSClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials");
return new AmazonSNSClient(credentials, config);
}
}

View File

@ -23,6 +23,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
@ -74,7 +75,7 @@ public class PutSNS extends AbstractSNSProcessor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(ARN, ARN_TYPE, SUBJECT, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, TIMEOUT,
Arrays.asList(ARN, ARN_TYPE, SUBJECT, REGION, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, TIMEOUT,
USE_JSON_STRUCTURE, CHARACTER_ENCODING));
public static final int MAX_SIZE = 256 * 1024;
@ -136,7 +137,7 @@ public class PutSNS extends AbstractSNSProcessor {
}
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
if (entry.getKey().isDynamic() && !isEmpty(entry.getValue())) {
if (entry.getKey().isDynamic() && !StringUtils.isEmpty(entry.getValue())) {
final MessageAttributeValue value = new MessageAttributeValue();
value.setStringValue(context.getProperty(entry.getKey()).evaluateAttributeExpressions(flowFile).getValue());
value.setDataType("String");

View File

@ -19,13 +19,14 @@ package org.apache.nifi.processors.aws.sqs;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
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.sqs.AmazonSQSClient;
public abstract class AbstractSQSProcessor extends AbstractAWSProcessor<AmazonSQSClient> {
public abstract class AbstractSQSProcessor extends AbstractAWSCredentialsProviderProcessor<AmazonSQSClient> {
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")
@ -43,8 +44,25 @@ public abstract class AbstractSQSProcessor extends AbstractAWSProcessor<AmazonSQ
.required(true)
.build();
/**
* Create client using credentials provider. This is the preferred way for creating clients
*/
@Override
protected AmazonSQSClient createClient(final ProcessContext context, final AWSCredentialsProvider credentialsProvider, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials provider ");
return new AmazonSQSClient(credentialsProvider, config);
}
/**
* Create client using AWSCredentails
*
* @deprecated use {@link #createClient(ProcessContext, AWSCredentialsProvider, ClientConfiguration)} instead
*/
@Override
protected AmazonSQSClient createClient(final ProcessContext context, final AWSCredentials credentials, final ClientConfiguration config) {
getLogger().info("Creating client using aws credentials ");
return new AmazonSQSClient(credentials, config);
}

View File

@ -54,7 +54,7 @@ public class DeleteSQS extends AbstractSQSProcessor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(ACCESS_KEY, SECRET_KEY, REGION, QUEUE_URL, TIMEOUT));
Arrays.asList(ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, REGION, QUEUE_URL, TIMEOUT));
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {

View File

@ -114,7 +114,8 @@ public class GetSQS extends AbstractSQSProcessor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(STATIC_QUEUE_URL, AUTO_DELETE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, REGION, BATCH_SIZE, TIMEOUT, CHARSET, VISIBILITY_TIMEOUT, RECEIVE_MSG_WAIT_TIME));
Arrays.asList(STATIC_QUEUE_URL, AUTO_DELETE, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE,
AWS_CREDENTIALS_PROVIDER_SERVICE, REGION, BATCH_SIZE, TIMEOUT, CHARSET, VISIBILITY_TIMEOUT, RECEIVE_MSG_WAIT_TIME));
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {

View File

@ -65,7 +65,7 @@ public class PutSQS extends AbstractSQSProcessor {
.build();
public static final List<PropertyDescriptor> properties = Collections.unmodifiableList(
Arrays.asList(QUEUE_URL, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, REGION, DELAY, TIMEOUT));
Arrays.asList(QUEUE_URL, ACCESS_KEY, SECRET_KEY, CREDENTIALS_FILE, AWS_CREDENTIALS_PROVIDER_SERVICE, REGION, DELAY, TIMEOUT));
private volatile List<PropertyDescriptor> userDefinedProperties = Collections.emptyList();

View File

@ -0,0 +1,15 @@
# 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.
org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService

View File

@ -0,0 +1,312 @@
/*
* 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.credentials.provider.service;
import static org.junit.Assert.assertEquals;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.s3.FetchS3Object;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Assert;
import org.junit.Test;
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;
public class AWSCredentialsProviderControllerServiceTest {
@Test
public void testDefaultAWSCredentialsProviderChain() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
final AWSCredentialsProviderService service = (AWSCredentialsProviderService) runner.getProcessContext()
.getControllerServiceLookup().getControllerService("awsCredentialsProvider");
Assert.assertNotNull(service);
final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
Assert.assertNotNull(credentialsProvider);
assertEquals("credentials provider should be equal", DefaultAWSCredentialsProviderChain.class,
credentialsProvider.getClass());
}
@Test
public void testKeysCredentialsProvider() throws Throwable {
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.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
final AWSCredentialsProviderService service = (AWSCredentialsProviderService) runner.getProcessContext()
.getControllerServiceLookup().getControllerService("awsCredentialsProvider");
Assert.assertNotNull(service);
final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
Assert.assertNotNull(credentialsProvider);
assertEquals("credentials provider should be equal", StaticCredentialsProvider.class,
credentialsProvider.getClass());
}
@Test
public void testKeysCredentialsProviderWithRoleAndName() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
final AWSCredentialsProviderService service = (AWSCredentialsProviderService) runner.getProcessContext()
.getControllerServiceLookup().getControllerService("awsCredentialsProvider");
Assert.assertNotNull(service);
final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
Assert.assertNotNull(credentialsProvider);
assertEquals("credentials provider should be equal", STSAssumeRoleSessionCredentialsProvider.class,
credentialsProvider.getClass());
}
@Test
public void testKeysCredentialsProviderWithRoleAndNameAndSessionTimeoutInRange() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "1000");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
final AWSCredentialsProviderService service = (AWSCredentialsProviderService) runner.getProcessContext()
.getControllerServiceLookup().getControllerService("awsCredentialsProvider");
Assert.assertNotNull(service);
final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
Assert.assertNotNull(credentialsProvider);
assertEquals("credentials provider should be equal", STSAssumeRoleSessionCredentialsProvider.class,
credentialsProvider.getClass());
}
@Test
public void testKeysCredentialsProviderWithRoleAndNameAndSessionTimeout900() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "900");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
}
@Test
public void testKeysCredentialsProviderWithRoleAndNameAndSessionTimeout3600() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "900");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
}
@Test(expected = AssertionError.class)
public void testKeysCredentialsProviderWithRoleAndNameAndSessionTimeoutLessThan900() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "899");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test(expected = AssertionError.class)
public void testKeysCredentialsProviderWithRoleAndNameAndSessionTimeoutGreaterThan3600() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.MAX_SESSION_TIME, "899");
runner.enableControllerService(serviceImpl);
}
@Test
public void testKeysCredentialsProviderWithRoleOnlyInvalid() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test
public void testKeysCredentialsProviderWithRoleNameOnlyInvalid() throws Throwable {
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, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test
public void testFileCredentialsProviderWithRole() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_ARN, "Role");
runner.setProperty(serviceImpl, AWSCredentialsProviderControllerService.ASSUME_ROLE_NAME, "RoleName");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
final AWSCredentialsProviderService service = (AWSCredentialsProviderService) runner.getProcessContext()
.getControllerServiceLookup().getControllerService("awsCredentialsProvider");
Assert.assertNotNull(service);
final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
Assert.assertNotNull(credentialsProvider);
assertEquals("credentials provider should be equal", STSAssumeRoleSessionCredentialsProvider.class,
credentialsProvider.getClass());
}
@Test
public void testFileCredentialsProvider() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
final AWSCredentialsProviderService service = (AWSCredentialsProviderService) runner.getProcessContext()
.getControllerServiceLookup().getControllerService("awsCredentialsProvider");
Assert.assertNotNull(service);
final AWSCredentialsProvider credentialsProvider = service.getCredentialsProvider();
Assert.assertNotNull(credentialsProvider);
assertEquals("credentials provider should be equal", PropertiesFileCredentialsProvider.class,
credentialsProvider.getClass());
}
@Test
public void testFileCredentialsProviderBadFile() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
"src/test/resources/bad-mock-aws-credentials.properties");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test
public void testFileAndAccessSecretKeyInvalid() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test
public void testFileAndAccessKeyInvalid() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test
public void testFileAndSecretKeyInvalid() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.CREDENTIALS_FILE,
"src/test/resources/mock-aws-credentials.properties");
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test
public void testAccessKeyOnlyInvalid() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.ACCESS_KEY, "awsAccessKey");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
@Test
public void testSecretKeyOnlyInvalid() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(FetchS3Object.class);
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSProcessor.SECRET_KEY, "awsSecretKey");
runner.enableControllerService(serviceImpl);
runner.assertNotValid(serviceImpl);
}
}

View File

@ -16,6 +16,19 @@
*/
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.auth.PropertiesCredentials;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CreateBucketRequest;
import com.amazonaws.services.s3.model.DeleteBucketRequest;
import com.amazonaws.services.s3.model.ObjectListing;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import org.apache.nifi.util.file.FileUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
@ -26,26 +39,16 @@ import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Iterator;
import org.apache.nifi.util.file.FileUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import static org.junit.Assert.fail;
import com.amazonaws.auth.PropertiesCredentials;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CreateBucketRequest;
import com.amazonaws.services.s3.model.DeleteBucketRequest;
import com.amazonaws.services.s3.model.ObjectListing;
import com.amazonaws.services.s3.model.PutObjectRequest;
import com.amazonaws.services.s3.model.S3ObjectSummary;
/**
* Base class for S3 tests. Establishes a bucket and helper methods for creating test scenarios
* Base class for S3 Integration Tests. Establishes a bucket and helper methods for creating test scenarios
*
* @see ITDeleteS3Object
* @see ITFetchS3Object
* @see ITPutS3Object
*/
public abstract class AbstractS3Test {
public abstract class AbstractS3IT {
protected final static String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
protected final static String BUCKET_NAME = "test-bucket-00000000-0000-0000-0000-123456789021";
protected final static String SAMPLE_FILE_RESOURCE_NAME = "/hello.txt";

View File

@ -16,17 +16,25 @@
*/
package org.apache.nifi.processors.aws.s3;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.components.PropertyDescriptor;
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.Ignore;
import org.junit.Test;
@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
public class TestDeleteS3Object extends AbstractS3Test {
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* Provides integration level testing with actual AWS S3 resources for {@link DeleteS3Object} and requires additional configuration and resources to work.
*/
public class ITDeleteS3Object extends AbstractS3IT {
@Test
public void testSimpleDelete() throws IOException {
@ -68,6 +76,34 @@ public class TestDeleteS3Object extends AbstractS3Test {
runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_SUCCESS, 1);
}
@Test
public void testDeleteFolderUsingCredentialsProviderService() throws Throwable {
// 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.REGION, REGION);
runner.setProperty(DeleteS3Object.BUCKET, BUCKET_NAME);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "folder/delete-me");
runner.enqueue(new byte[0], attrs);
runner.run(1);
runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_SUCCESS, 1);
}
@Test
public void testDeleteFolderNoExpressionLanguage() throws IOException {
// Prepares for this test
@ -106,4 +142,27 @@ public class TestDeleteS3Object extends AbstractS3Test {
runner.assertAllFlowFilesTransferred(DeleteS3Object.REL_SUCCESS, 1);
}
@Test
public void testGetPropertyDescriptors() throws Exception {
DeleteS3Object processor = new DeleteS3Object();
List<PropertyDescriptor> pd = processor.getSupportedPropertyDescriptors();
assertEquals("size should be eq", 17, pd.size());
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.REGION));
assertTrue(pd.contains(processor.SECRET_KEY));
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));
}
}

View File

@ -16,20 +16,27 @@
*/
package org.apache.nifi.processors.aws.s3;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import java.io.IOException;
import java.nio.file.Files;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Ignore;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
public class TestFetchS3Object extends AbstractS3Test {
/**
* Provides integration level testing with actual AWS S3 resources for {@link FetchS3Object} and requires additional configuration and resources to work.
*/
public class ITFetchS3Object extends AbstractS3IT {
@Test
public void testSimpleGet() throws IOException {
putTestFile("test-file", getFileFromResourceName(SAMPLE_FILE_RESOURCE_NAME));
@ -49,6 +56,34 @@ public class TestFetchS3Object extends AbstractS3Test {
runner.assertAllFlowFilesTransferred(FetchS3Object.REL_SUCCESS, 1);
}
@Test
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.REGION, REGION);
runner.setProperty(FetchS3Object.BUCKET, BUCKET_NAME);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test-file");
runner.enqueue(new byte[0], attrs);
runner.run(1);
runner.assertAllFlowFilesTransferred(FetchS3Object.REL_SUCCESS, 1);
}
@Test
public void testTryToFetchNotExistingFile() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new FetchS3Object());
@ -95,4 +130,23 @@ public class TestFetchS3Object extends AbstractS3Test {
System.out.println(entry.getKey() + " : " + entry.getValue());
}
}
@Test
public void testGetPropertyDescriptors() throws Exception {
FetchS3Object processor = new FetchS3Object();
List<PropertyDescriptor> pd = processor.getSupportedPropertyDescriptors();
assertEquals("size should be eq", 11, pd.size());
assertTrue(pd.contains(FetchS3Object.ACCESS_KEY));
assertTrue(pd.contains(FetchS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE));
assertTrue(pd.contains(FetchS3Object.BUCKET));
assertTrue(pd.contains(FetchS3Object.CREDENTIALS_FILE));
assertTrue(pd.contains(FetchS3Object.ENDPOINT_OVERRIDE));
assertTrue(pd.contains(FetchS3Object.KEY));
assertTrue(pd.contains(FetchS3Object.REGION));
assertTrue(pd.contains(FetchS3Object.SECRET_KEY));
assertTrue(pd.contains(FetchS3Object.SSL_CONTEXT_SERVICE));
assertTrue(pd.contains(FetchS3Object.TIMEOUT));
assertTrue(pd.contains(FetchS3Object.VERSION_ID));
}
}

View File

@ -16,23 +16,28 @@
*/
package org.apache.nifi.processors.aws.s3;
import com.amazonaws.services.s3.model.StorageClass;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.amazonaws.services.s3.model.StorageClass;
@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
public class TestPutS3Object extends AbstractS3Test {
/**
* Provides integration level testing with actual AWS S3 resources for {@link PutS3Object} and requires additional configuration and resources to work.
*/
public class ITPutS3Object extends AbstractS3IT {
@Test
public void testSimplePut() throws IOException {
@ -54,6 +59,36 @@ public class TestPutS3Object extends AbstractS3Test {
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 3);
}
@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, System.getProperty("user.home") + "/aws-credentials.properties");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
runner.setProperty(PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.setProperty(PutS3Object.REGION, REGION);
runner.setProperty(PutS3Object.BUCKET, BUCKET_NAME);
Assert.assertTrue(runner.setProperty("x-custom-prop", "hello").isValid());
for (int i = 0; i < 3; i++) {
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", String.valueOf(i) + ".txt");
runner.enqueue(getResourcePath(SAMPLE_FILE_RESOURCE_NAME), attrs);
}
runner.run(3);
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 3);
}
@Test
public void testMetaData() throws IOException {
PutS3Object processor = new PutS3Object();
@ -137,4 +172,30 @@ public class TestPutS3Object extends AbstractS3Test {
runner.assertAllFlowFilesTransferred(PutS3Object.REL_SUCCESS, 1);
}
@Test
public void testGetPropertyDescriptors() throws Exception {
PutS3Object processor = new PutS3Object();
List<PropertyDescriptor> pd = processor.getSupportedPropertyDescriptors();
assertEquals("size should be eq", 18, pd.size());
assertTrue(pd.contains(PutS3Object.ACCESS_KEY));
assertTrue(pd.contains(PutS3Object.AWS_CREDENTIALS_PROVIDER_SERVICE));
assertTrue(pd.contains(PutS3Object.BUCKET));
assertTrue(pd.contains(PutS3Object.CREDENTIALS_FILE));
assertTrue(pd.contains(PutS3Object.ENDPOINT_OVERRIDE));
assertTrue(pd.contains(PutS3Object.FULL_CONTROL_USER_LIST));
assertTrue(pd.contains(PutS3Object.KEY));
assertTrue(pd.contains(PutS3Object.OWNER));
assertTrue(pd.contains(PutS3Object.READ_ACL_LIST));
assertTrue(pd.contains(PutS3Object.READ_USER_LIST));
assertTrue(pd.contains(PutS3Object.REGION));
assertTrue(pd.contains(PutS3Object.SECRET_KEY));
assertTrue(pd.contains(PutS3Object.SSL_CONTEXT_SERVICE));
assertTrue(pd.contains(PutS3Object.TIMEOUT));
assertTrue(pd.contains(PutS3Object.EXPIRATION_RULE_ID));
assertTrue(pd.contains(PutS3Object.STORAGE_CLASS));
assertTrue(pd.contains(PutS3Object.WRITE_ACL_LIST));
assertTrue(pd.contains(PutS3Object.WRITE_USER_LIST));
}
}

View File

@ -16,20 +16,23 @@
*/
package org.apache.nifi.processors.aws.sns;
import static org.junit.Assert.assertTrue;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.Map;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Ignore;
import org.junit.Test;
import static org.junit.Assert.assertTrue;
import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
@Ignore("For local testing only - interacts with S3 so the credentials file must be configured and all necessary buckets created")
public class TestPutSNS {
/**
* Provides integration level testing with actual AWS S3 resources for {@link PutSNS} and requires additional configuration and resources to work.
*/
public class ITPutSNS {
private final String CREDENTIALS_FILE = System.getProperty("user.home") + "/aws-credentials.properties";
@ -48,4 +51,31 @@ public class TestPutSNS {
runner.assertAllFlowFilesTransferred(PutSNS.REL_SUCCESS, 1);
}
@Test
public void testPublishWithCredentialsProviderService() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(new PutSNS());
String snsArn = "Add Sns arn here";
runner.setValidateExpressionUsage(false);
runner.setProperty(PutSNS.ARN, snsArn);
assertTrue(runner.setProperty("DynamicProperty", "hello!").isValid());
final AWSCredentialsProviderControllerService serviceImpl = new AWSCredentialsProviderControllerService();
runner.addControllerService("awsCredentialsProvider", serviceImpl);
runner.setProperty(serviceImpl, AbstractAWSCredentialsProviderProcessor.CREDENTIALS_FILE, System.getProperty("user.home") + "/aws-credentials.properties");
runner.enableControllerService(serviceImpl);
runner.assertValid(serviceImpl);
runner.setProperty(PutSNS.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.run(1);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "1.txt");
runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs);
runner.run();
runner.assertAllFlowFilesTransferred(PutSNS.REL_SUCCESS, 1);
}
}

View File

@ -18,6 +18,8 @@ package org.apache.nifi.processors.aws.sqs;
import java.util.List;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.processors.aws.sns.PutSNS;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
@ -46,4 +48,31 @@ public class TestGetSQS {
}
}
@Test
public void testSimpleGetUsingCredentailsProviderService() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(new GetSQS());
runner.setProperty(GetSQS.TIMEOUT, "30 secs");
String queueUrl = "Add queue url here";
runner.setProperty(GetSQS.QUEUE_URL, queueUrl);
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(GetSQS.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.run(1);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(GetSQS.REL_SUCCESS);
for (final MockFlowFile mff : flowFiles) {
System.out.println(mff.getAttributes());
System.out.println(new String(mff.toByteArray()));
}
}
}

View File

@ -19,9 +19,13 @@ package org.apache.nifi.processors.aws.sqs;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.processors.aws.AbstractAWSProcessor;
import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
import org.apache.nifi.processors.aws.sns.PutSNS;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Assert;
@ -49,4 +53,34 @@ public class TestPutSQS {
runner.assertAllFlowFilesTransferred(PutSQS.REL_SUCCESS, 1);
}
@Test
public void testSimplePutUsingCredentailsProviderService() throws Throwable {
final TestRunner runner = TestRunners.newTestRunner(new PutSQS());
runner.setProperty(PutSQS.TIMEOUT, "30 secs");
String queueUrl = "Add queue url here";
runner.setProperty(PutSQS.QUEUE_URL, queueUrl);
runner.setValidateExpressionUsage(false);
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);
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "1.txt");
runner.enqueue(Paths.get("src/test/resources/hello.txt"), attrs);
runner.setProperty(PutSQS.AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredentialsProvider");
runner.run(1);
final List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(PutSQS.REL_SUCCESS);
for (final MockFlowFile mff : flowFiles) {
System.out.println(mff.getAttributes());
System.out.println(new String(mff.toByteArray()));
}
}
}

View File

@ -0,0 +1,2 @@
accessKey = awsAccessKey
secretKey = awsSecretKeyId

View File

@ -22,6 +22,7 @@ import java.util.List;
import javax.xml.bind.annotation.XmlType;
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
import org.apache.nifi.web.api.dto.util.TimeAdapter;
import org.apache.nifi.web.api.dto.util.TimestampAdapter;
import com.wordnik.swagger.annotations.ApiModelProperty;
@ -38,11 +39,10 @@ public class ListingRequestDTO {
private Integer percentCompleted;
private Boolean finished;
private String failureReason;
private String sortColumn;
private String sortDirection;
private Integer maxResults;
private Integer totalStepCount;
private Integer completedStepCount;
private Boolean isSourceRunning;
private Boolean isDestinationRunning;
private String state;
private QueueSizeDTO queueSize;
@ -95,7 +95,7 @@ public class ListingRequestDTO {
/**
* @return the time this request was last updated
*/
@XmlJavaTypeAdapter(TimestampAdapter.class)
@XmlJavaTypeAdapter(TimeAdapter.class)
@ApiModelProperty(
value = "The last time this listing request was updated."
)
@ -177,30 +177,6 @@ public class ListingRequestDTO {
this.flowFileSummaries = flowFileSummaries;
}
/**
* @return the column on which the listing is sorted
*/
@ApiModelProperty(value = "The column on which the FlowFiles are sorted.")
public String getSortColumn() {
return sortColumn;
}
public void setSortColumn(String sortColumn) {
this.sortColumn = sortColumn;
}
/**
* @return the direction in which the FlowFiles are sorted
*/
@ApiModelProperty(value = "The direction in which the FlowFiles are sorted. Either ASCENDING or DESCENDING.")
public String getSortDirection() {
return sortDirection;
}
public void setSortDirection(String sortDirection) {
this.sortDirection = sortDirection;
}
/**
* @return the maximum number of FlowFileSummary objects to return
*/
@ -213,31 +189,6 @@ public class ListingRequestDTO {
this.maxResults = maxResults;
}
/**
* @return the total number of steps required to complete the listing
*/
@ApiModelProperty(value = "The total number of steps required to complete the listing")
public Integer getTotalStepCount() {
return totalStepCount;
}
public void setTotalStepCount(Integer totalStepCount) {
this.totalStepCount = totalStepCount;
}
/**
* @return the number of steps that have already been completed. This value will be >= 0 and <= the total step count
*/
@ApiModelProperty(value = "The number of steps that have already been completed. This value will be between 0 and the total step count (inclusive)")
public Integer getCompletedStepCount() {
return completedStepCount;
}
public void setCompletedStepCount(Integer completedStepCount) {
this.completedStepCount = completedStepCount;
}
/**
* @return the size for the queue
*/
@ -249,4 +200,28 @@ public class ListingRequestDTO {
public void setQueueSize(QueueSizeDTO queueSize) {
this.queueSize = queueSize;
}
/**
* @return whether the source is running
*/
@ApiModelProperty(value = "Whether the source of the connection is running")
public Boolean getSourceRunning() {
return isSourceRunning;
}
public void setSourceRunning(Boolean sourceRunning) {
isSourceRunning = sourceRunning;
}
/**
* @return whether the destination is running
*/
@ApiModelProperty(value = "Whether the destination of the connection is running")
public Boolean getDestinationRunning() {
return isDestinationRunning;
}
public void setDestinationRunning(Boolean destinationRunning) {
isDestinationRunning = destinationRunning;
}
}

View File

@ -23,7 +23,7 @@ import javax.xml.bind.annotation.XmlRootElement;
/**
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds a reference to a FlowFileDTO.
*/
@XmlRootElement(name = "listingRequestEntity")
@XmlRootElement(name = "flowFileEntity")
public class FlowFileEntity extends Entity {
private FlowFileDTO flowFile;

View File

@ -130,7 +130,6 @@ import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.state.StateManagerProvider;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.FlowFileSummaries;
import org.apache.nifi.controller.Heartbeater;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ScheduledState;
@ -140,8 +139,6 @@ import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.exception.ComponentLifeCycleException;
import org.apache.nifi.controller.queue.DropFlowFileState;
import org.apache.nifi.controller.queue.ListFlowFileState;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.controller.reporting.ClusteredReportingTaskNode;
import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
import org.apache.nifi.controller.reporting.ReportingTaskProvider;
@ -2513,7 +2510,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
}
private static boolean isListFlowFilesEndpoint(final URI uri, final String method) {
if ("GET".equalsIgnoreCase(method) && LISTING_REQUEST_URI.matcher(uri.getPath()).matches()) {
if (("GET".equalsIgnoreCase(method) || "DELETE".equalsIgnoreCase(method)) && LISTING_REQUEST_URI.matcher(uri.getPath()).matches()) {
return true;
} else if ("POST".equalsIgnoreCase(method) && LISTING_REQUESTS_URI.matcher(uri.getPath()).matches()) {
return true;
@ -2957,8 +2954,28 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
* @param listingRequestMap the mapping of all responses being merged
*/
private void mergeListingRequests(final ListingRequestDTO listingRequest, final Map<NodeIdentifier, ListingRequestDTO> listingRequestMap) {
final Comparator<FlowFileSummaryDTO> comparator = FlowFileSummaries.createDTOComparator(
SortColumn.valueOf(listingRequest.getSortColumn()), SortDirection.valueOf(listingRequest.getSortDirection()));
final Comparator<FlowFileSummaryDTO> comparator = new Comparator<FlowFileSummaryDTO>() {
@Override
public int compare(final FlowFileSummaryDTO dto1, final FlowFileSummaryDTO dto2) {
int positionCompare = dto1.getPosition().compareTo(dto2.getPosition());
if (positionCompare != 0) {
return positionCompare;
}
final String address1 = dto1.getClusterNodeAddress();
final String address2 = dto2.getClusterNodeAddress();
if (address1 == null && address2 == null) {
return 0;
}
if (address1 == null) {
return 1;
}
if (address2 == null) {
return -1;
}
return address1.compareTo(address2);
}
};
final NavigableSet<FlowFileSummaryDTO> flowFileSummaries = new TreeSet<>(comparator);
@ -2974,8 +2991,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
final ListingRequestDTO nodeRequest = entry.getValue();
numStepsCompleted += nodeRequest.getCompletedStepCount();
numStepsTotal += nodeRequest.getTotalStepCount();
numStepsTotal++;
if (Boolean.TRUE.equals(nodeRequest.getFinished())) {
numStepsCompleted++;
}
final QueueSizeDTO nodeQueueSize = nodeRequest.getQueueSize();
objectCount += nodeQueueSize.getObjectCount();
@ -2995,15 +3014,17 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
state = nodeState;
}
for (final FlowFileSummaryDTO summaryDTO : nodeRequest.getFlowFileSummaries()) {
summaryDTO.setClusterNodeId(nodeIdentifier.getId());
summaryDTO.setClusterNodeAddress(nodeAddress);
if (nodeRequest.getFlowFileSummaries() != null) {
for (final FlowFileSummaryDTO summaryDTO : nodeRequest.getFlowFileSummaries()) {
summaryDTO.setClusterNodeId(nodeIdentifier.getId());
summaryDTO.setClusterNodeAddress(nodeAddress);
flowFileSummaries.add(summaryDTO);
flowFileSummaries.add(summaryDTO);
// Keep the set from growing beyond our max
if (flowFileSummaries.size() > listingRequest.getMaxResults()) {
flowFileSummaries.pollLast();
// Keep the set from growing beyond our max
if (flowFileSummaries.size() > listingRequest.getMaxResults()) {
flowFileSummaries.pollLast();
}
}
}

View File

@ -25,24 +25,17 @@ public class ListFlowFileRequest implements ListFlowFileStatus {
private final String requestId;
private final int maxResults;
private final QueueSize queueSize;
private final SortColumn sortColumn;
private final SortDirection sortDirection;
private final long submissionTime = System.currentTimeMillis();
private final List<FlowFileSummary> flowFileSummaries = new ArrayList<>();
private ListFlowFileState state = ListFlowFileState.WAITING_FOR_LOCK;
private String failureReason;
private int numSteps;
private int completedStepCount;
private long lastUpdated = System.currentTimeMillis();
public ListFlowFileRequest(final String requestId, final SortColumn sortColumn, final SortDirection sortDirection, final int maxResults, final QueueSize queueSize, final int numSteps) {
public ListFlowFileRequest(final String requestId, final int maxResults, final QueueSize queueSize) {
this.requestId = requestId;
this.sortColumn = sortColumn;
this.sortDirection = sortDirection;
this.maxResults = maxResults;
this.queueSize = queueSize;
this.numSteps = numSteps;
}
@Override
@ -60,16 +53,6 @@ public class ListFlowFileRequest implements ListFlowFileStatus {
return lastUpdated;
}
@Override
public SortColumn getSortColumn() {
return sortColumn;
}
@Override
public SortDirection getSortDirection() {
return sortDirection;
}
@Override
public synchronized ListFlowFileState getState() {
return state;
@ -118,25 +101,11 @@ public class ListFlowFileRequest implements ListFlowFileStatus {
@Override
public synchronized int getCompletionPercentage() {
return (int) (100F * completedStepCount / numSteps);
}
public synchronized void setCompletedStepCount(final int completedStepCount) {
this.completedStepCount = completedStepCount;
return state == ListFlowFileState.COMPLETE ? 100 : 0;
}
@Override
public int getMaxResults() {
return maxResults;
}
@Override
public int getTotalStepCount() {
return numSteps;
}
@Override
public int getCompletedStepCount() {
return completedStepCount;
}
}

View File

@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.nifi.controller.Heartbeater;
import org.apache.nifi.controller.ProcessScheduler;
import org.apache.nifi.controller.StandardFlowFileQueue;
import org.apache.nifi.controller.queue.FlowFileQueue;
@ -70,7 +71,7 @@ public final class StandardConnection implements Connection {
relationships = new AtomicReference<>(Collections.unmodifiableCollection(builder.relationships));
scheduler = builder.scheduler;
flowFileQueue = new StandardFlowFileQueue(id, this, builder.flowFileRepository, builder.provenanceRepository, builder.resourceClaimManager,
scheduler, builder.swapManager, builder.eventReporter, NiFiProperties.getInstance().getQueueSwapThreshold());
scheduler, builder.swapManager, builder.eventReporter, NiFiProperties.getInstance().getQueueSwapThreshold(), builder.heartbeater);
hashCode = new HashCodeBuilder(7, 67).append(id).toHashCode();
}
@ -269,6 +270,7 @@ public final class StandardConnection implements Connection {
private FlowFileRepository flowFileRepository;
private ProvenanceEventRepository provenanceRepository;
private ResourceClaimManager resourceClaimManager;
private Heartbeater heartbeater;
public Builder(final ProcessScheduler scheduler) {
this.scheduler = scheduler;
@ -304,6 +306,11 @@ public final class StandardConnection implements Connection {
return this;
}
public Builder heartbeater(final Heartbeater heartbeater) {
this.heartbeater = heartbeater;
return this;
}
public Builder bendPoints(final List<Position> bendPoints) {
this.bendPoints.clear();
this.bendPoints.addAll(bendPoints);

View File

@ -826,6 +826,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
.resourceClaimManager(resourceClaimManager)
.flowFileRepository(flowFileRepository)
.provenanceRepository(provenanceEventRepository)
.heartbeater(this)
.build();
}

View File

@ -1,95 +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.controller;
import java.util.Collections;
import java.util.Comparator;
import org.apache.nifi.controller.queue.FlowFileSummary;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
public class FlowFileSummaries {
public static Comparator<FlowFileSummary> createComparator(final SortColumn column, final SortDirection direction) {
final Comparator<FlowFileSummary> comparator = new Comparator<FlowFileSummary>() {
@Override
public int compare(final FlowFileSummary o1, final FlowFileSummary o2) {
switch (column) {
case FILENAME:
return o1.getFilename().compareTo(o2.getFilename());
case FLOWFILE_AGE:
return Long.compare(o1.getLineageStartDate(), o2.getLineageStartDate());
case FLOWFILE_SIZE:
return Long.compare(o1.getSize(), o2.getSize());
case FLOWFILE_UUID:
return o1.getUuid().compareTo(o2.getUuid());
case PENALIZATION:
return Boolean.compare(o1.isPenalized(), o2.isPenalized());
case QUEUE_POSITION:
return Long.compare(o1.getPosition(), o2.getPosition());
case QUEUED_DURATION:
return Long.compare(o1.getLastQueuedTime(), o2.getLastQueuedTime());
}
return 0;
}
};
if (direction == SortDirection.DESCENDING) {
return Collections.reverseOrder(comparator);
} else {
return comparator;
}
}
public static Comparator<FlowFileSummaryDTO> createDTOComparator(final SortColumn column, final SortDirection direction) {
final Comparator<FlowFileSummaryDTO> comparator = new Comparator<FlowFileSummaryDTO>() {
@Override
public int compare(final FlowFileSummaryDTO o1, final FlowFileSummaryDTO o2) {
switch (column) {
case FILENAME:
return o1.getFilename().compareTo(o2.getFilename());
case FLOWFILE_AGE:
return o1.getLineageDuration().compareTo(o2.getLineageDuration());
case FLOWFILE_SIZE:
return Long.compare(o1.getSize(), o2.getSize());
case FLOWFILE_UUID:
return o1.getUuid().compareTo(o2.getUuid());
case PENALIZATION:
return Boolean.compare(o1.getPenalized(), o2.getPenalized());
case QUEUE_POSITION:
return Long.compare(o1.getPosition(), o2.getPosition());
case QUEUED_DURATION:
return o1.getQueuedDuration().compareTo(o2.getQueuedDuration());
}
return 0;
}
};
if (direction == SortDirection.DESCENDING) {
return Collections.reverseOrder(comparator);
} else {
return comparator;
}
}
}

View File

@ -16,9 +16,25 @@
*/
package org.apache.nifi.controller;
import org.apache.nifi.attribute.expression.language.PreparedQuery;
import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.queue.DropFlowFileState;
import org.apache.nifi.controller.queue.DropFlowFileStatus;
@ -28,8 +44,6 @@ import org.apache.nifi.controller.queue.ListFlowFileRequest;
import org.apache.nifi.controller.queue.ListFlowFileState;
import org.apache.nifi.controller.queue.ListFlowFileStatus;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.FlowFileSwapManager;
@ -39,7 +53,6 @@ import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.expression.AttributeExpression.ResultType;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
@ -57,27 +70,6 @@ import org.apache.nifi.util.concurrency.TimedLock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.PriorityQueue;
import java.util.Queue;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantReadWriteLock;
/**
* A FlowFileQueue is used to queue FlowFile objects that are awaiting further
* processing. Must be thread safe.
@ -115,6 +107,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
private final FlowFileRepository flowFileRepository;
private final ProvenanceEventRepository provRepository;
private final ResourceClaimManager resourceClaimManager;
private final Heartbeater heartbeater;
private final ConcurrentMap<String, DropFlowFileRequest> dropRequestMap = new ConcurrentHashMap<>();
private final ConcurrentMap<String, ListFlowFileRequest> listRequestMap = new ConcurrentHashMap<>();
@ -123,7 +116,8 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
private final ProcessScheduler scheduler;
public StandardFlowFileQueue(final String identifier, final Connection connection, final FlowFileRepository flowFileRepo, final ProvenanceEventRepository provRepo,
final ResourceClaimManager resourceClaimManager, final ProcessScheduler scheduler, final FlowFileSwapManager swapManager, final EventReporter eventReporter, final int swapThreshold) {
final ResourceClaimManager resourceClaimManager, final ProcessScheduler scheduler, final FlowFileSwapManager swapManager, final EventReporter eventReporter, final int swapThreshold,
final Heartbeater heartbeater) {
activeQueue = new PriorityQueue<>(20, new Prioritizer(new ArrayList<FlowFilePrioritizer>()));
priorities = new ArrayList<>();
swapQueue = new ArrayList<>();
@ -137,6 +131,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
this.swapThreshold = swapThreshold;
this.scheduler = scheduler;
this.connection = connection;
this.heartbeater = heartbeater;
readLock = new TimedLock(this.lock.readLock(), identifier + " Read Lock", 100);
writeLock = new TimedLock(this.lock.writeLock(), identifier + " Write Lock", 100);
@ -852,33 +847,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
@Override
public ListFlowFileStatus listFlowFiles(final String requestIdentifier, final int maxResults) {
return listFlowFiles(requestIdentifier, maxResults, SortColumn.QUEUE_POSITION, SortDirection.ASCENDING);
}
@Override
public ListFlowFileStatus listFlowFiles(final String requestIdentifier, final int maxResults, final SortColumn sortColumn, final SortDirection direction) {
return listFlowFiles(requestIdentifier, maxResults, null, sortColumn, direction);
}
@Override
public ListFlowFileStatus listFlowFiles(final String requestIdentifier, final int maxResults, final String query, final SortColumn sortColumn, final SortDirection direction) {
final PreparedQuery preparedQuery;
if (query == null) {
preparedQuery = null;
} else {
try {
final ResultType resultType = Query.compile(query).getResultType();
if (resultType != ResultType.BOOLEAN) {
throw new IllegalArgumentException("Invalid expression Language provided to search the listing of FlowFiles. "
+ "The expression must return a 'Boolean' type but returns a " + resultType.name() + " type");
}
preparedQuery = Query.prepare(query);
} catch (final AttributeExpressionLanguageParsingException e) {
throw new IllegalArgumentException("Invalid Expression Language provided to search the listing of FlowFiles: " + query, e);
}
}
// purge any old requests from the map just to keep it clean. But if there are very requests, which is usually the case, then don't bother
// purge any old requests from the map just to keep it clean. But if there are very few requests, which is usually the case, then don't bother
if (listRequestMap.size() > 10) {
final List<String> toDrop = new ArrayList<>();
for (final Map.Entry<String, ListFlowFileRequest> entry : listRequestMap.entrySet()) {
@ -896,101 +865,49 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
}
// numSteps = 1 for each swap location + 1 for active queue + 1 for swap queue.
final int numSteps = 2 + size.get().swapFiles;
final ListFlowFileRequest listRequest = new ListFlowFileRequest(requestIdentifier, sortColumn, direction, maxResults, size(), numSteps);
final ListFlowFileRequest listRequest = new ListFlowFileRequest(requestIdentifier, maxResults, size());
final Thread t = new Thread(new Runnable() {
@Override
public void run() {
int position = 0;
int resultCount = 0;
final Comparator<FlowFileSummary> comparator = FlowFileSummaries.createComparator(sortColumn, direction);
final NavigableSet<FlowFileSummary> summaries = new TreeSet<>(comparator);
int completedStepCount = 0;
final List<FlowFileSummary> summaries = new ArrayList<>();
// we need a write lock while using the Active Queue because we can't iterate over it - we have to poll from it
// continually. This is because the iterator for PriorityQueue does not iterate over the elements in any particular
// order. Since we need the 'position' of the element in the queue, we need to iterate over them in the proper order.
writeLock.lock();
// Create an ArrayList that contains all of the contents of the active queue.
// We do this so that we don't have to hold the lock any longer than absolutely necessary.
// We cannot simply pull the first 'maxResults' records from the queue, however, because the
// Iterator provided by PriorityQueue does not return records in order. So we would have to either
// use a writeLock and 'pop' the first 'maxResults' records off the queue or use a read lock and
// do a shallow copy of the queue. The shallow copy is generally quicker because it doesn't have to do
// the sorting to put the records back. So even though this has an expensive of Java Heap to create the
// extra collection, we are making this trade-off to avoid locking the queue any longer than required.
final List<FlowFileRecord> allFlowFiles;
final Prioritizer prioritizer;
readLock.lock();
try {
logger.debug("{} Acquired lock to perform listing of FlowFiles", StandardFlowFileQueue.this);
listRequest.setState(ListFlowFileState.CALCULATING_LIST);
final List<FlowFileRecord> flowFileRecords = new ArrayList<>(activeQueue.size());
FlowFileRecord flowFile;
try {
while ((flowFile = activeQueue.poll()) != null) {
flowFileRecords.add(flowFile);
position++;
if (preparedQuery == null || "true".equals(preparedQuery.evaluateExpressions(flowFile))) {
summaries.add(summarize(flowFile, position));
if (summaries.size() > maxResults) {
summaries.pollLast();
}
}
}
} finally {
activeQueue.addAll(flowFileRecords);
}
allFlowFiles = new ArrayList<>(activeQueue);
prioritizer = new Prioritizer(StandardFlowFileQueue.this.priorities);
} finally {
writeLock.unlock("List FlowFiles");
readLock.unlock("List FlowFiles");
}
listRequest.setState(ListFlowFileState.CALCULATING_LIST);
// sort the FlowFileRecords so that we have the list in the same order as on the queue.
Collections.sort(allFlowFiles, prioritizer);
for (final FlowFileRecord flowFile : allFlowFiles) {
summaries.add(summarize(flowFile, ++position));
if (summaries.size() >= maxResults) {
break;
}
}
logger.debug("{} Finished listing FlowFiles for active queue with a total of {} results", StandardFlowFileQueue.this, resultCount);
listRequest.setCompletedStepCount(++completedStepCount);
position = activeQueue.size();
try {
// We are now iterating over swap files, and we don't need the write lock for this, just the read lock, since
// we are not modifying anything.
readLock.lock();
try {
for (final String location : swapLocations) {
logger.debug("{} Performing listing of FlowFiles for Swap Location {}", StandardFlowFileQueue.this, location);
final List<FlowFileRecord> flowFiles = swapManager.peek(location, StandardFlowFileQueue.this);
for (final FlowFileRecord flowFile : flowFiles) {
position++;
if (preparedQuery == null || "true".equals(preparedQuery.evaluateExpressions(flowFile))) {
summaries.add(summarize(flowFile, position));
if (summaries.size() > maxResults) {
summaries.pollLast();
}
}
}
listRequest.setCompletedStepCount(++completedStepCount);
}
logger.debug("{} Performing listing of FlowFiles from Swap Queue", StandardFlowFileQueue.this);
for (final FlowFileRecord flowFile : swapQueue) {
position++;
if (preparedQuery == null || "true".equals(preparedQuery.evaluateExpressions(flowFile))) {
summaries.add(summarize(flowFile, position));
if (summaries.size() > maxResults) {
summaries.pollLast();
}
}
}
listRequest.setCompletedStepCount(++completedStepCount);
} finally {
readLock.unlock("List FlowFiles");
}
} catch (final IOException ioe) {
logger.error("Failed to read swapped FlowFiles in order to perform listing of queue " + StandardFlowFileQueue.this, ioe);
listRequest.setFailure("Could not read FlowFiles from queue. Check log files for more details.");
}
// We have now completed the listing successfully. Set the number of completed steps to the total number of steps. We may have
// skipped some steps because we have reached the maximum number of results, so we consider those steps completed.
logger.debug("{} Completed listing of FlowFiles", StandardFlowFileQueue.this);
listRequest.setCompletedStepCount(listRequest.getTotalStepCount());
listRequest.setFlowFileSummaries(summaries);
listRequest.setState(ListFlowFileState.COMPLETE);
listRequest.setFlowFileSummaries(new ArrayList<FlowFileSummary>(summaries));
}
}, "List FlowFiles for Connection " + getIdentifier());
t.setDaemon(true);
@ -1082,24 +999,6 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
return flowFile;
}
}
for (final FlowFileRecord flowFile : swapQueue) {
if (flowFileUuid.equals(flowFile.getAttribute(CoreAttributes.UUID.key()))) {
return flowFile;
}
}
// TODO: consider using a Long flowFileId instead of a UUID, and then having the swap manager
// write out the min and max FlowFile ID's. This would allow us to then have a method: boolean isFlowFilePossiblyContained(long id)
// which can return a boolean value that can be used to determine whether or not to even call peek
for (final String swapLocation : swapLocations) {
final List<FlowFileRecord> flowFiles = swapManager.peek(swapLocation, this);
for (final FlowFileRecord flowFile : flowFiles) {
if (flowFileUuid.equals(flowFile.getAttribute(CoreAttributes.UUID.key()))) {
return flowFile;
}
}
}
} finally {
readLock.unlock("getFlowFile");
}
@ -1110,13 +1009,6 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
@Override
public void verifyCanList() throws IllegalStateException {
if (connection.getSource().isRunning()) {
throw new IllegalStateException("Cannot list the FlowFiles of queue because the connection's source is still running");
}
if (connection.getDestination().isRunning()) {
throw new IllegalStateException("Cannot list the FlowFiles of queue because the connection's destination is still running");
}
}
@Override
@ -1248,6 +1140,9 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
logger.info("Successfully dropped {} FlowFiles ({} bytes) from Connection with ID {} on behalf of {}",
dropRequest.getDroppedSize().getObjectCount(), dropRequest.getDroppedSize().getByteCount(), StandardFlowFileQueue.this.getIdentifier(), requestor);
dropRequest.setState(DropFlowFileState.COMPLETE);
if (heartbeater != null) {
heartbeater.heartbeat();
}
} catch (final Exception e) {
logger.error("Failed to drop FlowFiles from Connection with ID {} due to {}", StandardFlowFileQueue.this.getIdentifier(), e.toString());
logger.error("", e);

View File

@ -17,17 +17,31 @@
package org.apache.nifi.controller;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
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.atomic.AtomicLong;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.queue.DropFlowFileState;
import org.apache.nifi.controller.queue.DropFlowFileStatus;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.FlowFileSummary;
import org.apache.nifi.controller.queue.ListFlowFileState;
import org.apache.nifi.controller.queue.ListFlowFileStatus;
import org.apache.nifi.controller.queue.QueueSize;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.controller.repository.FlowFileRepository;
import org.apache.nifi.controller.repository.FlowFileSwapManager;
@ -49,23 +63,6 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.util.ArrayList;
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.atomic.AtomicLong;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
public class TestStandardFlowFileQueue {
private TestSwapManager swapManager = null;
private StandardFlowFileQueue queue = null;
@ -105,7 +102,7 @@ public class TestStandardFlowFileQueue {
}
}).when(provRepo).registerEvents(Mockito.any(Iterable.class));
queue = new StandardFlowFileQueue("id", connection, flowFileRepo, provRepo, claimManager, scheduler, swapManager, null, 10000);
queue = new StandardFlowFileQueue("id", connection, flowFileRepo, provRepo, claimManager, scheduler, swapManager, null, 10000, null);
TestFlowFile.idGenerator.set(0L);
}
@ -417,72 +414,8 @@ public class TestStandardFlowFileQueue {
assertEquals(9999, status.getFlowFileSummaries().size());
assertEquals(100, status.getCompletionPercentage());
assertNull(status.getFailureReason());
assertEquals(2, status.getTotalStepCount());
assertEquals(2, status.getCompletedStepCount());
}
@Test(timeout = 5000)
public void testListFlowFilesActiveQueueAndSwapQueue() throws InterruptedException {
for (int i = 0; i < 11000; i++) {
queue.put(new TestFlowFile());
}
final ListFlowFileStatus status = queue.listFlowFiles(UUID.randomUUID().toString(), 11000);
assertNotNull(status);
assertEquals(11000, status.getQueueSize().getObjectCount());
while (status.getState() != ListFlowFileState.COMPLETE) {
Thread.sleep(100);
}
assertEquals(11000, status.getFlowFileSummaries().size());
assertEquals(100, status.getCompletionPercentage());
assertNull(status.getFailureReason());
assertEquals(2, status.getTotalStepCount());
assertEquals(2, status.getCompletedStepCount());
}
@Test(timeout = 5000)
public void testListFlowFilesActiveQueueAndSwapFile() throws InterruptedException {
for (int i = 0; i < 20000; i++) {
queue.put(new TestFlowFile());
}
final ListFlowFileStatus status = queue.listFlowFiles(UUID.randomUUID().toString(), 20000);
assertNotNull(status);
assertEquals(20000, status.getQueueSize().getObjectCount());
while (status.getState() != ListFlowFileState.COMPLETE) {
Thread.sleep(100);
}
assertEquals(20000, status.getFlowFileSummaries().size());
assertEquals(100, status.getCompletionPercentage());
assertNull(status.getFailureReason());
assertEquals(3, status.getTotalStepCount());
assertEquals(3, status.getCompletedStepCount());
}
@Test(timeout = 5000)
public void testListFlowFilesActiveQueueAndSwapFilesAndSwapQueue() throws InterruptedException {
for (int i = 0; i < 30050; i++) {
queue.put(new TestFlowFile());
}
final ListFlowFileStatus status = queue.listFlowFiles(UUID.randomUUID().toString(), 30050);
assertNotNull(status);
assertEquals(30050, status.getQueueSize().getObjectCount());
while (status.getState() != ListFlowFileState.COMPLETE) {
Thread.sleep(100);
}
assertEquals(30050, status.getFlowFileSummaries().size());
assertEquals(100, status.getCompletionPercentage());
assertNull(status.getFailureReason());
assertEquals(4, status.getTotalStepCount());
assertEquals(4, status.getCompletedStepCount());
}
@Test(timeout = 5000)
public void testListFlowFilesResultsLimited() throws InterruptedException {
@ -501,62 +434,6 @@ public class TestStandardFlowFileQueue {
assertEquals(100, status.getFlowFileSummaries().size());
assertEquals(100, status.getCompletionPercentage());
assertNull(status.getFailureReason());
assertEquals(4, status.getTotalStepCount());
assertEquals(4, status.getCompletedStepCount());
}
@Test
public void testListFlowFilesSortedAscending() throws InterruptedException {
for (int i = 0; i < 30050; i++) {
queue.put(new TestFlowFile(i));
}
final ListFlowFileStatus status = queue.listFlowFiles(UUID.randomUUID().toString(), 100, SortColumn.FLOWFILE_SIZE, SortDirection.ASCENDING);
assertNotNull(status);
assertEquals(30050, status.getQueueSize().getObjectCount());
while (status.getState() != ListFlowFileState.COMPLETE) {
Thread.sleep(100);
}
assertEquals(100, status.getFlowFileSummaries().size());
assertEquals(100, status.getCompletionPercentage());
assertNull(status.getFailureReason());
assertEquals(4, status.getTotalStepCount());
assertEquals(4, status.getCompletedStepCount());
int counter = 0;
for (final FlowFileSummary summary : status.getFlowFileSummaries()) {
assertEquals(counter++, summary.getSize());
}
}
@Test
public void testListFlowFilesSortedDescending() throws InterruptedException {
for (int i = 0; i < 30050; i++) {
queue.put(new TestFlowFile(i));
}
final ListFlowFileStatus status = queue.listFlowFiles(UUID.randomUUID().toString(), 100, SortColumn.FLOWFILE_SIZE, SortDirection.DESCENDING);
assertNotNull(status);
assertEquals(30050, status.getQueueSize().getObjectCount());
while (status.getState() != ListFlowFileState.COMPLETE) {
Thread.sleep(100);
}
assertEquals(100, status.getFlowFileSummaries().size());
assertEquals(100, status.getCompletionPercentage());
assertNull(status.getFailureReason());
assertEquals(4, status.getTotalStepCount());
assertEquals(4, status.getCompletedStepCount());
int counter = 0;
for (final FlowFileSummary summary : status.getFlowFileSummaries()) {
assertEquals((30050 - 1 - counter++), summary.getSize());
}
}

View File

@ -139,7 +139,7 @@ public class TestStandardProcessSession {
final ProcessScheduler processScheduler = Mockito.mock(ProcessScheduler.class);
final FlowFileSwapManager swapManager = Mockito.mock(FlowFileSwapManager.class);
flowFileQueue = new StandardFlowFileQueue("1", connection, flowFileRepo, provenanceRepo, null, processScheduler, swapManager, null, 10000);
flowFileQueue = new StandardFlowFileQueue("1", connection, flowFileRepo, provenanceRepo, null, processScheduler, swapManager, null, 10000, null);
when(connection.getFlowFileQueue()).thenReturn(flowFileQueue);
Mockito.doAnswer(new Answer<Object>() {

View File

@ -16,26 +16,23 @@
*/
package org.apache.nifi.web;
import java.util.Collection;
import java.util.Date;
import java.util.Set;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.controller.repository.claim.ContentDirection;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
import org.apache.nifi.web.api.dto.ClusterDTO;
import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ComponentHistoryDTO;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.ControllerConfigurationDTO;
import org.apache.nifi.web.api.dto.ControllerDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
import org.apache.nifi.web.api.dto.CounterDTO;
import org.apache.nifi.web.api.dto.CountersDTO;
import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
import org.apache.nifi.web.api.dto.DropRequestDTO;
import org.apache.nifi.web.api.dto.FlowFileDTO;
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
import org.apache.nifi.web.api.dto.FunnelDTO;
@ -46,9 +43,6 @@ import org.apache.nifi.web.api.dto.NodeSystemDiagnosticsDTO;
import org.apache.nifi.web.api.dto.PortDTO;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.ComponentHistoryDTO;
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
import org.apache.nifi.web.api.dto.DropRequestDTO;
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
@ -62,8 +56,8 @@ import org.apache.nifi.web.api.dto.UserGroupDTO;
import org.apache.nifi.web.api.dto.action.ActionDTO;
import org.apache.nifi.web.api.dto.action.HistoryDTO;
import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO;
import org.apache.nifi.web.api.dto.provenance.lineage.LineageDTO;
import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
@ -79,6 +73,10 @@ import org.apache.nifi.web.api.dto.status.NodeStatusDTO;
import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import java.util.Collection;
import java.util.Date;
import java.util.Set;
/**
* Defines the NiFiServiceFacade interface.
*/
@ -586,11 +584,9 @@ public interface NiFiServiceFacade {
* @param groupId group
* @param connectionId The ID of the connection
* @param listingRequestId The ID of the listing request
* @param column sort column
* @param direction sort direction
* @return The ListingRequest
*/
ListingRequestDTO createFlowFileListingRequest(String groupId, String connectionId, String listingRequestId, SortColumn column, SortDirection direction);
ListingRequestDTO createFlowFileListingRequest(String groupId, String connectionId, String listingRequestId);
/**
* Gets a new flow file listing request.

View File

@ -51,8 +51,6 @@ import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.Snippet;
import org.apache.nifi.controller.Template;
import org.apache.nifi.controller.label.Label;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.controller.repository.claim.ContentDirection;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceReference;
@ -913,7 +911,18 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public ListingRequestDTO deleteFlowFileListingRequest(String groupId, String connectionId, String listingRequestId) {
return dtoFactory.createListingRequestDTO(connectionDAO.deleteFlowFileListingRequest(groupId, connectionId, listingRequestId));
final ListingRequestDTO listRequest = dtoFactory.createListingRequestDTO(connectionDAO.deleteFlowFileListingRequest(groupId, connectionId, listingRequestId));
// include whether the source and destination are running
final Connection connection = connectionDAO.getConnection(groupId, connectionId);
if (connection.getSource() != null) {
listRequest.setSourceRunning(connection.getSource().isRunning());
}
if (connection.getDestination() != null) {
listRequest.setDestinationRunning(connection.getDestination().isRunning());
}
return listRequest;
}
@Override
@ -1173,8 +1182,19 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
public ListingRequestDTO createFlowFileListingRequest(String groupId, String connectionId, String listingRequestId, SortColumn column, SortDirection direction) {
return dtoFactory.createListingRequestDTO(connectionDAO.createFlowFileListingRequest(groupId, connectionId, listingRequestId, column, direction));
public ListingRequestDTO createFlowFileListingRequest(String groupId, String connectionId, String listingRequestId) {
final ListingRequestDTO listRequest = dtoFactory.createListingRequestDTO(connectionDAO.createFlowFileListingRequest(groupId, connectionId, listingRequestId));
// include whether the source and destination are running
final Connection connection = connectionDAO.getConnection(groupId, connectionId);
if (connection.getSource() != null) {
listRequest.setSourceRunning(connection.getSource().isRunning());
}
if (connection.getDestination() != null) {
listRequest.setDestinationRunning(connection.getDestination().isRunning());
}
return listRequest;
}
@Override
@ -2268,7 +2288,18 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
@Override
public ListingRequestDTO getFlowFileListingRequest(String groupId, String connectionId, String listingRequestId) {
return dtoFactory.createListingRequestDTO(connectionDAO.getFlowFileListingRequest(groupId, connectionId, listingRequestId));
final ListingRequestDTO listRequest = dtoFactory.createListingRequestDTO(connectionDAO.getFlowFileListingRequest(groupId, connectionId, listingRequestId));
// include whether the source and destination are running
final Connection connection = connectionDAO.getConnection(groupId, connectionId);
if (connection.getSource() != null) {
listRequest.setSourceRunning(connection.getSource().isRunning());
}
if (connection.getDestination() != null) {
listRequest.setDestinationRunning(connection.getDestination().isRunning());
}
return listRequest;
}
@Override

View File

@ -29,8 +29,6 @@ import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
import org.apache.nifi.cluster.node.Node;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.ConfigurationSnapshot;
@ -1173,6 +1171,11 @@ public class ConnectionResource extends ApplicationResource {
)
@PathParam("connection-id") String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// defer to the new endpoint that references /drop-requests in the URI
return createDropRequest(httpServletRequest, clientId, id);
}
@ -1186,7 +1189,7 @@ public class ConnectionResource extends ApplicationResource {
* @return A listRequestEntity
*/
@POST
@Consumes(MediaType.WILDCARD)
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
@Path("/{connection-id}/listing-requests")
@PreAuthorize("hasRole('ROLE_DFM')")
@ -1213,47 +1216,16 @@ public class ConnectionResource extends ApplicationResource {
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@ApiParam(
value = "The connection id.",
required = true
)
@PathParam("connection-id") String id,
@ApiParam(
value = "The sort column.",
required = false,
defaultValue = "QUEUE_POSITION",
allowableValues = "QUEUE_POSITION, FLOWFILE_UUID, FILENAME, FLOWFILE_SIZE, QUEUED_DURATION, FLOWFILE_AGE, PENALIZATION"
)
@FormParam("sortColumn") String sortColumn,
@ApiParam(
value = "The sort direction.",
required = false,
defaultValue = "asc",
allowableValues = "asc, desc"
)
@FormParam("sortOrder") @DefaultValue("asc") String sortOrder) {
// parse the sort column
final SortColumn column;
if (sortColumn == null) {
column = SortColumn.QUEUE_POSITION;
} else {
try {
column = SortColumn.valueOf(sortColumn);
} catch (final IllegalArgumentException iae) {
throw new IllegalArgumentException(String.format("Sort Column: Value must be one of [%s]", StringUtils.join(SortColumn.values(), ", ")));
}
}
// normalize the sort order
if (!sortOrder.equalsIgnoreCase("asc") && !sortOrder.equalsIgnoreCase("desc")) {
throw new IllegalArgumentException("The sort order must be 'asc' or 'desc'.");
}
@PathParam("connection-id") String id) {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)
@ -1263,13 +1235,6 @@ public class ConnectionResource extends ApplicationResource {
return generateContinueResponse().build();
}
final SortDirection direction;
if (sortOrder.equalsIgnoreCase("asc")) {
direction = SortDirection.ASCENDING;
} else {
direction = SortDirection.DESCENDING;
}
// ensure the id is the same across the cluster
final String listingRequestId;
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
@ -1280,7 +1245,7 @@ public class ConnectionResource extends ApplicationResource {
}
// submit the listing request
final ListingRequestDTO listingRequest = serviceFacade.createFlowFileListingRequest(groupId, id, listingRequestId, column, direction);
final ListingRequestDTO listingRequest = serviceFacade.createFlowFileListingRequest(groupId, id, listingRequestId);
populateRemainingFlowFileListingContent(id, listingRequest);
// create the revision
@ -1452,7 +1417,7 @@ public class ConnectionResource extends ApplicationResource {
* @return A dropRequestEntity
*/
@POST
@Consumes(MediaType.WILDCARD)
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
@Path("/{connection-id}/drop-requests")
@PreAuthorize("hasRole('ROLE_DFM')")
@ -1479,7 +1444,7 @@ public class ConnectionResource extends ApplicationResource {
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
required = false
)
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
@ApiParam(
value = "The connection id.",
required = true
@ -1488,7 +1453,7 @@ public class ConnectionResource extends ApplicationResource {
// replicate if cluster manager
if (properties.isClusterManager()) {
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
}
// handle expects request (usually from the cluster manager)

View File

@ -440,10 +440,6 @@ public final class DtoFactory {
dto.setFailureReason(listingRequest.getFailureReason());
dto.setFinished(isListingRequestComplete(listingRequest.getState()));
dto.setMaxResults(listingRequest.getMaxResults());
dto.setSortColumn(listingRequest.getSortColumn().name());
dto.setSortDirection(listingRequest.getSortDirection().name());
dto.setTotalStepCount(listingRequest.getTotalStepCount());
dto.setCompletedStepCount(listingRequest.getCompletedStepCount());
dto.setPercentCompleted(listingRequest.getCompletionPercentage());
dto.setQueueSize(createQueueSizeDTO(listingRequest.getQueueSize()));

View File

@ -16,16 +16,15 @@
*/
package org.apache.nifi.web.dao;
import java.util.Set;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.queue.DropFlowFileStatus;
import org.apache.nifi.controller.queue.ListFlowFileStatus;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.web.DownloadableContent;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import java.util.Set;
public interface ConnectionDAO {
/**
@ -118,11 +117,9 @@ public interface ConnectionDAO {
* @param groupId group id
* @param id connection id
* @param listingRequestId listing request id
* @param column sort column
* @param direction sort direction
* @return The listing request status
*/
ListFlowFileStatus createFlowFileListingRequest(String groupId, String id, String listingRequestId, SortColumn column, SortDirection direction);
ListFlowFileStatus createFlowFileListingRequest(String groupId, String id, String listingRequestId);
/**
* Verifies the listing can be processed.

View File

@ -16,18 +16,6 @@
*/
package org.apache.nifi.web.dao.impl;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import javax.ws.rs.WebApplicationException;
import org.apache.nifi.admin.service.UserService;
import org.apache.nifi.authorization.DownloadAuthorization;
import org.apache.nifi.connectable.Connectable;
@ -40,14 +28,12 @@ import org.apache.nifi.controller.exception.ValidationException;
import org.apache.nifi.controller.queue.DropFlowFileStatus;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.ListFlowFileStatus;
import org.apache.nifi.controller.queue.SortColumn;
import org.apache.nifi.controller.queue.SortDirection;
import org.apache.nifi.controller.repository.ContentNotFoundException;
import org.apache.nifi.controller.repository.FlowFileRecord;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.user.NiFiUser;
@ -64,6 +50,18 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.security.access.AccessDeniedException;
import javax.ws.rs.WebApplicationException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO {
private static final Logger logger = LoggerFactory.getLogger(StandardConnectionDAO.class);
@ -126,7 +124,7 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
final FlowFileRecord flowFile = queue.getFlowFile(flowFileUuid);
if (flowFile == null) {
throw new ResourceNotFoundException(String.format("Unable to find FlowFile '%s' in Connection '%s'.", flowFileUuid, id));
throw new ResourceNotFoundException(String.format("The FlowFile with UUID %s is no longer in the active queue.", flowFileUuid));
}
return flowFile;
@ -375,14 +373,14 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
}
@Override
public ListFlowFileStatus createFlowFileListingRequest(String groupId, String id, String listingRequestId, SortColumn column, SortDirection direction) {
public ListFlowFileStatus createFlowFileListingRequest(String groupId, String id, String listingRequestId) {
final Connection connection = locateConnection(groupId, id);
final FlowFileQueue queue = connection.getFlowFileQueue();
// ensure we can list
verifyList(queue);
return queue.listFlowFiles(listingRequestId, 100, column, direction);
return queue.listFlowFiles(listingRequestId, 100);
}
@Override
@ -606,7 +604,7 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
final FlowFileRecord flowFile = queue.getFlowFile(flowFileUuid);
if (flowFile == null) {
throw new ResourceNotFoundException(String.format("Unable to find FlowFile '%s' in Connection '%s'.", flowFileUuid, id));
throw new ResourceNotFoundException(String.format("The FlowFile with UUID %s is no longer in the active queue.", flowFileUuid));
}
// calculate the dn chain

View File

@ -20,10 +20,16 @@
<div id="queue-listing-header-text"></div>
</div>
<div id="queue-listing-stats-container">
<div id="queue-listing-refresh-button" class="pointer" title="Refresh"></div>
<div id="queue-listing-last-refreshed-container">
Last updated:&nbsp;<span id="queue-listing-last-refreshed"></span>
</div>
<div id="queue-listing-loading-container" class="loading-container"></div>
<div id="queue-listing-stats">
Displaying&nbsp;<span id="displayed-flowfiles"></span>&nbsp;of&nbsp;<span id="total-flowfiles-count"></span>&nbsp;(<span id="total-flowfiles-size"></span>)
</div>
<div id="queue-listing-loading-container" class="loading-container"></div>
<div class="clear"></div>
</div>
<div id="queue-listing-table"></div>
<div id="queue-listing-message" class="hidden"></div>
</div>

View File

@ -21,10 +21,10 @@
#queue-listing-container {
position: absolute;
top: 0px;
bottom: 0px;
left: 0px;
right: 0px;
top: 0;
bottom: 0;
left: 0;
right: 0;
}
#queue-listing-header-container {
@ -41,19 +41,28 @@
}
#queue-listing-stats-container {
margin-left: 15px;
margin-top: 30px;
margin-left: 20px;
margin-top: 18px;
-webkit-user-select: none;
-moz-user-select: none;
}
#queue-listing-stats {
font-size: 9px;
#queue-listing-refresh-button {
float: left;
height: 24px;
width: 26px;
}
#queue-listing-last-refreshed-container {
float: left;
color: #666;
font-weight: normal;
margin-top: 6px;
margin-left: 3px;
}
#queue-listing-last-refreshed {
font-weight: bold;
color: #9f6000;
clear: left;
line-height: normal;
margin-left: 5px;
}
#queue-listing-loading-container {
@ -65,6 +74,16 @@
margin-left: 3px;
}
#queue-listing-stats {
font-size: 9px;
font-weight: bold;
color: #9f6000;
float: right;
line-height: normal;
margin-right: 20px;
margin-top: 6px;
}
/* queue listing table */
#queue-listing-table {
@ -77,6 +96,16 @@
overflow: hidden;
}
/* queue listing table */
#queue-listing-message {
position: absolute;
left: 20px;
bottom: 20px;
color: #f00;
font-size: 10px;
}
/* flowfile details */
#flowfile-details-dialog {

View File

@ -975,7 +975,7 @@ nf.Actions = (function () {
});
} else {
// nothing was removed
nf.Dialog.showYesNoDialog({
nf.Dialog.showOkDialog({
dialogContent: 'No FlowFiles were removed.',
overlayBackground: false
});

View File

@ -331,7 +331,7 @@ nf.ContextMenu = (function () {
* @param {selection} selection
*/
var canListQueue = function (selection) {
return nf.Common.isDFM() && isConnection(selection) && nf.CanvasUtils.supportsModification(selection);
return nf.Common.isDFM() && isConnection(selection);
};
/**

View File

@ -22,9 +22,6 @@
*/
nf.QueueListing = (function () {
var DEFAULT_SORT_COL = 'QUEUE_POSITION';
var DEFAULT_SORT_ASC = true;
/**
* Initializes the listing request status dialog.
*/
@ -78,7 +75,7 @@ nf.QueueListing = (function () {
var dataUri = $('#flowfile-uri').text() + '/content';
// conditionally include the cluster node id
var clusterNodeId = $('#flowfile-cluster-node-id').text();;
var clusterNodeId = $('#flowfile-cluster-node-id').text();
if (!nf.Common.isBlank(clusterNodeId)) {
var parameters = {
'clusterNodeId': clusterNodeId
@ -158,10 +155,8 @@ nf.QueueListing = (function () {
* Performs a listing on the specified connection.
*
* @param connection the connection
* @param sortCol the sort column
* @param sortAsc if sort is asc
*/
var performListing = function (connection, sortCol, sortAsc) {
var performListing = function (connection) {
var MAX_DELAY = 4;
var cancelled = false;
var listingRequest = null;
@ -231,6 +226,26 @@ nf.QueueListing = (function () {
$('#total-flowfiles-count').text(nf.Common.formatInteger(listingRequest.queueSize.objectCount));
$('#total-flowfiles-size').text(nf.Common.formatDataSize(listingRequest.queueSize.byteCount));
// update the last updated time
$('#queue-listing-last-refreshed').text(listingRequest.lastUpdated);
// show a message for the queue listing if necessary
var queueListingTable = $('#queue-listing-table');
var queueListingMessage = $('#queue-listing-message');
if (listingRequest.sourceRunning === true || listingRequest.destinationRunning === true) {
if (listingRequest.souceRunning === true && listingRequest.destinationRunning === true) {
queueListingMessage.text('The source and destination of this queue are currently running. This listing may no longer be accurate.').show();
} else if (listingRequest.sourceRunning === true) {
queueListingMessage.text('The source of this queue is currently running. This listing may no longer be accurate.').show();
} else if (listingRequest.destinationRunning === true) {
queueListingMessage.text('The destination of this queue is currently running. This listing may no longer be accurate.').show();
}
queueListingTable.css('bottom', '35px');
} else {
queueListingMessage.text('').hide();
queueListingTable.css('bottom', '20px');
}
// get the grid to load the data
var queueListingGrid = $('#queue-listing-table').data('gridInstance');
var queueListingData = queueListingGrid.getData();
@ -290,10 +305,6 @@ nf.QueueListing = (function () {
$.ajax({
type: 'POST',
url: connection.component.uri + '/listing-requests',
data: {
sortColumn: sortCol,
sortOrder: sortAsc ? 'asc' : 'desc'
},
dataType: 'json'
}).done(function(response) {
// initialize the progress bar value
@ -332,9 +343,15 @@ nf.QueueListing = (function () {
}
};
var params = {};
if (nf.Common.isDefinedAndNotNull(flowFileSummary.clusterNodeId)) {
params['clusterNodeId'] = flowFileSummary.clusterNodeId;
}
$.ajax({
type: 'GET',
url: flowFileSummary.uri,
data: params,
dataType: 'json'
}).done(function(response) {
var flowFile = response.flowFile;
@ -352,12 +369,12 @@ nf.QueueListing = (function () {
$('#flowfile-penalized').text(flowFile.penalized === true ? 'Yes' : 'No');
// conditionally show the cluster node identifier
if (nf.Common.isDefinedAndNotNull(flowFile.clusterNodeId)) {
if (nf.Common.isDefinedAndNotNull(flowFileSummary.clusterNodeId)) {
// save the cluster node id
$('#flowfile-cluster-node-id').text(flowFile.clusterNodeId);
$('#flowfile-cluster-node-id').text(flowFileSummary.clusterNodeId);
// render the cluster node address
formatFlowFileDetail('Node Address', flowFile.clusterNodeAddress);
formatFlowFileDetail('Node Address', flowFileSummary.clusterNodeAddress);
}
if (nf.Common.isDefinedAndNotNull(flowFile.contentClaimContainer)) {
@ -423,6 +440,12 @@ nf.QueueListing = (function () {
resetTableSize();
});
// define mouse over event for the refresh button
nf.Common.addHoverEffect('#queue-listing-refresh-button', 'button-refresh', 'button-refresh-hover').click(function () {
var connection = $('#queue-listing-table').data('connection');
performListing(connection);
});
// define a custom formatter for showing more processor details
var moreDetailsFormatter = function (row, cell, value, columnDef, dataContext) {
return '<img src="images/iconDetails.png" title="View Details" class="pointer show-flowfile-details" style="margin-top: 5px; float: left;"/>';
@ -452,13 +475,13 @@ nf.QueueListing = (function () {
// initialize the queue listing table
var queueListingColumns = [
{id: 'moreDetails', field: 'moreDetails', name: '&nbsp;', sortable: false, resizable: false, formatter: moreDetailsFormatter, width: 50, maxWidth: 50},
{id: 'QUEUE_POSITION', name: 'Position', field: 'position', sortable: true, resizable: false, width: 75, maxWidth: 75},
{id: 'FLOWFILE_UUID', name: 'UUID', field: 'uuid', sortable: true, resizable: true},
{id: 'FILENAME', name: 'Filename', field: 'filename', sortable: true, resizable: true},
{id: 'FLOWFILE_SIZE', name: 'File Size', field: 'size', sortable: true, resizable: true, defaultSortAsc: false, formatter: dataSizeFormatter},
{id: 'QUEUED_DURATION', name: 'Queued Duration', field: 'queuedDuration', sortable: true, resizable: true, formatter: durationFormatter},
{id: 'FLOWFILE_AGE', name: 'Lineage Duration', field: 'lineageDuration', sortable: true, resizable: true, formatter: durationFormatter},
{id: 'PENALIZATION', name: 'Penalized', field: 'penalized', sortable: true, resizable: false, width: 100, maxWidth: 100, formatter: penalizedFormatter}
{id: 'position', name: 'Position', field: 'position', sortable: false, resizable: false, width: 75, maxWidth: 75},
{id: 'uuid', name: 'UUID', field: 'uuid', sortable: false, resizable: true},
{id: 'filename', name: 'Filename', field: 'filename', sortable: false, resizable: true},
{id: 'size', name: 'File Size', field: 'size', sortable: false, resizable: true, defaultSortAsc: false, formatter: dataSizeFormatter},
{id: 'queuedDuration', name: 'Queued Duration', field: 'queuedDuration', sortable: false, resizable: true, formatter: durationFormatter},
{id: 'lineageDuration', name: 'Lineage Duration', field: 'lineageDuration', sortable: false, resizable: true, formatter: durationFormatter},
{id: 'penalized', name: 'Penalized', field: 'penalized', sortable: false, resizable: false, width: 100, maxWidth: 100, formatter: penalizedFormatter}
];
// conditionally show the cluster node identifier
@ -484,10 +507,6 @@ nf.QueueListing = (function () {
var queueListingGrid = new Slick.Grid('#queue-listing-table', queueListingData, queueListingColumns, queueListingOptions);
queueListingGrid.setSelectionModel(new Slick.RowSelectionModel());
queueListingGrid.registerPlugin(new Slick.AutoTooltips());
queueListingGrid.onSort.subscribe(function (e, args) {
var connection = $('#queue-listing-table').data('connection');
performListing(connection, args.sortCol.id, args.sortAsc);
});
// configure a click listener
queueListingGrid.onClick.subscribe(function (e, args) {
@ -530,11 +549,8 @@ nf.QueueListing = (function () {
* @param {object} The connection
*/
listQueue: function (connection) {
var queueListingGrid = $('#queue-listing-table').data('gridInstance');
queueListingGrid.setSortColumn(DEFAULT_SORT_COL, DEFAULT_SORT_ASC);
// perform the initial listing
performListing(connection, DEFAULT_SORT_COL, DEFAULT_SORT_ASC).done(function () {
performListing(connection).done(function () {
// update the connection name
var connectionName = nf.CanvasUtils.formatConnectionName(connection.component);
if (connectionName === '') {
@ -547,6 +563,7 @@ nf.QueueListing = (function () {
$('#queue-listing-table').removeData('connection');
// clear the table
var queueListingGrid = $('#queue-listing-table').data('gridInstance');
var queueListingData = queueListingGrid.getData();
// clear the flowfiles

View File

@ -0,0 +1,40 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-html-bundle</artifactId>
<version>0.4.2-SNAPSHOT</version>
</parent>
<artifactId>nifi-html-nar</artifactId>
<packaging>nar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-standard-services-api-nar</artifactId>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-html-processors</artifactId>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,240 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.
APACHE NIFI SUBCOMPONENTS:
The Apache NiFi project contains subcomponents with separate copyright
notices and license terms. Your use of the source code for the these
subcomponents is subject to the terms and conditions of the following
licenses.
This product bundles 'jsoup' which is available under the MIT License.
For details see http://jsoup.org/
jsoup License
The jsoup code-base (include source and compiled packages) are distributed under the open source MIT license as described below.
The MIT License
Copyright © 2009 - 2013 Jonathan Hedley (jonathan@hedley.net)
Permission is hereby granted, free of charge, to any person
obtaining a copy of this software and associated documentation
files (the "Software"), to deal in the Software without
restriction, including without limitation the rights to use,
copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the
Software is furnished to do so, subject to the following
conditions:
The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
OTHER DEALINGS IN THE SOFTWARE.

View File

@ -0,0 +1,19 @@
nifi-html-nar
Copyright 2015 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
******************
Apache Software License v2
******************
The following binary components are provided under the Apache Software License v2
(ASLv2) Apache Commons Lang
The following NOTICE information applies:
Apache Commons Lang
Copyright 2001-2014 The Apache Software Foundation
This product includes software from the Spring Framework,
under the Apache License 2.0 (see: StringUtils.containsWhitespace())

View File

@ -0,0 +1,71 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-html-bundle</artifactId>
<version>0.4.2-SNAPSHOT</version>
</parent>
<artifactId>nifi-html-processors</artifactId>
<description>Support for parsing HTML documents</description>
<dependencies>
<dependency>
<groupId>org.jsoup</groupId>
<artifactId>jsoup</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-processor-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-simple</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
<configuration>
<excludes combine.children="append">
<exclude>src/test/resources/Weather.html</exclude>
</excludes>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,130 @@
/*
* 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;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
import org.jsoup.select.Selector;
import java.io.IOException;
import java.io.InputStream;
import java.util.concurrent.atomic.AtomicReference;
public abstract class AbstractHTMLProcessor extends AbstractProcessor {
protected static final String ELEMENT_HTML = "HTML";
protected static final String ELEMENT_TEXT = "Text";
protected static final String ELEMENT_DATA = "Data";
protected static final String ELEMENT_ATTRIBUTE = "Attribute";
protected static final Validator CSS_SELECTOR_VALIDATOR = new Validator() {
@Override
public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value)) {
return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build();
}
String reason = null;
try {
Document doc = Jsoup.parse("<html></html>");
doc.select(value);
} catch (final Selector.SelectorParseException e) {
reason = "\"" + value + "\" is an invalid CSS selector";
}
return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
}
};
public static final PropertyDescriptor URL = new PropertyDescriptor
.Builder().name("URL")
.description("Base URL for the HTML page being parsed.")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final PropertyDescriptor CSS_SELECTOR = new PropertyDescriptor
.Builder().name("CSS Selector")
.description("CSS selector syntax string used to extract the desired HTML element(s).")
.required(true)
.addValidator(CSS_SELECTOR_VALIDATOR)
.expressionLanguageSupported(true)
.build();
public static final PropertyDescriptor HTML_CHARSET = new PropertyDescriptor
.Builder().name("HTML Character Encoding")
.description("Character encoding of the input HTML")
.defaultValue("UTF-8")
.required(true)
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.build();
public static final Relationship REL_ORIGINAL = new Relationship.Builder()
.name("original")
.description("The original HTML input")
.build();
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("Successfully parsed HTML element")
.build();
public static final Relationship REL_INVALID_HTML = new Relationship.Builder()
.name("invalid html")
.description("The input HTML syntax is invalid")
.build();
public static final Relationship REL_NOT_FOUND = new Relationship.Builder()
.name("element not found")
.description("Element could not be found in the HTML document. The original HTML input will remain " +
"in the flowfile content unchanged. Relationship '" + REL_ORIGINAL + "' will not be invoked " +
"in this scenario.")
.build();
/**
* Parses the Jsoup HTML document from the FlowFile input content.
*
* @param inputFlowFile Input FlowFile containing the HTML
* @param context ProcessContext
* @param session ProcessSession
*
* @return Jsoup Document
*/
protected Document parseHTMLDocumentFromFlowfile(final FlowFile inputFlowFile, final ProcessContext context, final ProcessSession session) {
final AtomicReference<Document> doc = new AtomicReference<>();
session.read(inputFlowFile, new InputStreamCallback() {
@Override
public void process(InputStream inputStream) throws IOException {
doc.set(Jsoup.parse(inputStream,
context.getProperty(HTML_CHARSET).getValue(),
context.getProperty(URL).getValue()));
}
});
return doc.get();
}
}

View File

@ -0,0 +1,241 @@
/*
* 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;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
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 org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.StreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.jsoup.nodes.Document;
import org.jsoup.nodes.Element;
import org.jsoup.select.Elements;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.HashSet;
import java.util.Collections;
@SupportsBatching
@Tags({"get", "html", "dom", "css", "element"})
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
@CapabilityDescription("Extracts HTML element values from the incoming flowfile's content using a CSS selector." +
" The incoming HTML is first converted into a HTML Document Object Model so that HTML elements may be selected" +
" in the similar manner that CSS selectors are used to apply styles to HTML. The resulting HTML DOM is then \"queried\"" +
" using the user defined CSS selector string. The result of \"querying\" the HTML DOM may produce 0-N results." +
" If no results are found the flowfile will be transferred to the \"element not found\" relationship to indicate" +
" so to the end user. If N results are found a new flowfile will be created and emitted for each result. The query result will" +
" either be placed in the content of the new flowfile or as an attribute of the new flowfile. By default the result is written to an" +
" attribute. This can be controlled by the \"Destination\" property. Resulting query values may also have data" +
" prepended or appended to them by setting the value of property \"Prepend Element Value\" or \"Append Element Value\"." +
" Prepended and appended values are treated as string values and concatenated to the result retrieved from the" +
" HTML DOM query operation. A more thorough reference for the CSS selector syntax can be found at" +
" \"http://jsoup.org/apidocs/org/jsoup/select/Selector.html\"")
@SeeAlso({ModifyHTMLElement.class, PutHTMLElement.class})
@WritesAttributes({@WritesAttribute(attribute="HTMLElement", description="Flowfile attribute where the element result" +
" parsed from the HTML using the CSS selector syntax are placed if the destination is a flowfile attribute.")})
public class GetHTMLElement
extends AbstractHTMLProcessor {
public static final String HTML_ELEMENT_ATTRIBUTE_NAME = "HTMLElement";
public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
public static final String DESTINATION_CONTENT = "flowfile-content";
public static final PropertyDescriptor PREPEND_ELEMENT_VALUE = new PropertyDescriptor
.Builder().name("Prepend Element Value")
.description("Prepends the specified value to the resulting Element")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.build();
public static final PropertyDescriptor APPEND_ELEMENT_VALUE = new PropertyDescriptor
.Builder().name("Append Element Value")
.description("Appends the specified value to the resulting Element")
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.build();
public static final PropertyDescriptor ATTRIBUTE_KEY = new PropertyDescriptor
.Builder().name("Attribute Name")
.description(("When getting the value of a HTML element attribute this value is used as the key to determine" +
" which attribute on the selected element should be retrieved. This value is used when the \"Output Type\"" +
" is set to \"" + ELEMENT_ATTRIBUTE + "\""))
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.build();
public static final PropertyDescriptor OUTPUT_TYPE = new PropertyDescriptor.Builder()
.name("Output Type")
.description("Controls the type of DOM value that is retrieved from the HTML element.")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.allowableValues(ELEMENT_HTML, ELEMENT_TEXT, ELEMENT_ATTRIBUTE, ELEMENT_DATA)
.defaultValue(ELEMENT_HTML)
.build();
public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
.name("Destination")
.description("Control if element extracted is written as a flowfile attribute or " +
"as flowfile content.")
.required(true)
.allowableValues(DESTINATION_ATTRIBUTE, DESTINATION_CONTENT)
.defaultValue(DESTINATION_ATTRIBUTE)
.build();
private List<PropertyDescriptor> descriptors;
private Set<Relationship> relationships;
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(URL);
descriptors.add(CSS_SELECTOR);
descriptors.add(HTML_CHARSET);
descriptors.add(OUTPUT_TYPE);
descriptors.add(DESTINATION);
descriptors.add(PREPEND_ELEMENT_VALUE);
descriptors.add(APPEND_ELEMENT_VALUE);
descriptors.add(ATTRIBUTE_KEY);
this.descriptors = Collections.unmodifiableList(descriptors);
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_ORIGINAL);
relationships.add(REL_SUCCESS);
relationships.add(REL_INVALID_HTML);
relationships.add(REL_NOT_FOUND);
this.relationships = Collections.unmodifiableSet(relationships);
}
@Override
public Set<Relationship> getRelationships() {
return this.relationships;
}
@Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return descriptors;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final FlowFile flowFile = session.get();
if ( flowFile == null ) {
return;
}
final Document doc;
final Elements eles;
try {
doc = parseHTMLDocumentFromFlowfile(flowFile, context, session);
eles = doc.select(context.getProperty(CSS_SELECTOR).evaluateAttributeExpressions(flowFile).getValue());
} catch (final Exception ex) {
getLogger().error("Failed to extract HTML from {} due to {}; routing to {}", new Object[] {flowFile, ex, REL_INVALID_HTML}, ex);
session.transfer(flowFile, REL_INVALID_HTML);
return;
}
final String prependValue = context.getProperty(PREPEND_ELEMENT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
final String appendValue = context.getProperty(APPEND_ELEMENT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
final String outputType = context.getProperty(OUTPUT_TYPE).getValue();
final String attributeKey = context.getProperty(ATTRIBUTE_KEY).evaluateAttributeExpressions(flowFile).getValue();
if (eles == null || eles.isEmpty()) {
// No element found
session.transfer(flowFile, REL_NOT_FOUND);
} else {
// Create a new FlowFile for each matching element.
for (final Element ele : eles) {
final String extractedElementValue = extractElementValue(prependValue, outputType, appendValue, ele, attributeKey);
final FlowFile ff = session.create(flowFile);
FlowFile updatedFF = ff;
switch (context.getProperty(DESTINATION).getValue()) {
case DESTINATION_ATTRIBUTE:
updatedFF = session.putAttribute(ff, HTML_ELEMENT_ATTRIBUTE_NAME, extractedElementValue);
break;
case DESTINATION_CONTENT:
updatedFF = session.write(ff, new StreamCallback() {
@Override
public void process(final InputStream inputStream, final OutputStream outputStream) throws IOException {
outputStream.write(extractedElementValue.getBytes(StandardCharsets.UTF_8));
}
});
break;
}
session.transfer(updatedFF, REL_SUCCESS);
}
// Transfer the original HTML
session.transfer(flowFile, REL_ORIGINAL);
}
}
/**
* Extracts the HTML value based on the configuration values.
*
* @return value from the parsed HTML element
*/
private String extractElementValue(String prependValue, final String outputType, String appendValue, final Element ele, final String attrKey) {
if (StringUtils.isEmpty(prependValue)) {
prependValue = "";
}
if (StringUtils.isEmpty(appendValue)) {
appendValue = "";
}
switch (outputType) {
case ELEMENT_HTML:
return prependValue + ele.html() + appendValue;
case ELEMENT_TEXT:
return prependValue + ele.text() + appendValue;
case ELEMENT_DATA:
return prependValue + ele.data() + appendValue;
case ELEMENT_ATTRIBUTE:
return prependValue + ele.attr(attrKey) + appendValue;
default:
return prependValue + ele.html() + appendValue;
}
}
}

View File

@ -0,0 +1,179 @@
/*
* 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;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
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 org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.StreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.jsoup.nodes.Document;
import org.jsoup.nodes.Element;
import org.jsoup.select.Elements;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.ArrayList;
import java.util.Set;
import java.util.HashSet;
import java.util.Collections;
@Tags({"modify", "html", "dom", "css", "element"})
@SupportsBatching
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
@CapabilityDescription("Modifies the value of an existing HTML element. The desired element to be modified is located by" +
" using CSS selector syntax. The incoming HTML is first converted into a HTML Document Object Model so that HTML elements may be selected" +
" in the similar manner that CSS selectors are used to apply styles to HTML. The resulting HTML DOM is then \"queried\"" +
" using the user defined CSS selector string to find the element the user desires to modify. If the HTML element is found" +
" the element's value is updated in the DOM using the value specified \"Modified Value\" property. All DOM elements" +
" that match the CSS selector will be updated. Once all of the DOM elements have been updated the DOM is rendered" +
" to HTML and the result replaces the flowfile content with the updated HTML. A more thorough reference for the" +
" CSS selector syntax can be found at" +
" \"http://jsoup.org/apidocs/org/jsoup/select/Selector.html\"")
@SeeAlso({GetHTMLElement.class, PutHTMLElement.class})
@WritesAttributes({@WritesAttribute(attribute="NumElementsModified", description="Total number of HTML " +
"element modifications made")})
public class ModifyHTMLElement extends AbstractHTMLProcessor {
public static final String NUM_ELEMENTS_MODIFIED_ATTR = "NumElementsModified";
public static final PropertyDescriptor OUTPUT_TYPE = new PropertyDescriptor.Builder()
.name("Output Type")
.description("Controls whether the HTML element is output as " +
ELEMENT_HTML + "," + ELEMENT_TEXT + " or " + ELEMENT_DATA)
.required(true)
.allowableValues(ELEMENT_HTML, ELEMENT_TEXT, ELEMENT_ATTRIBUTE)
.defaultValue(ELEMENT_HTML)
.build();
public static final PropertyDescriptor MODIFIED_VALUE = new PropertyDescriptor
.Builder().name("Modified Value")
.description("Value to update the found HTML elements with")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.build();
public static final PropertyDescriptor ATTRIBUTE_KEY = new PropertyDescriptor
.Builder().name("Attribute Name")
.description(("When modifying the value of an element attribute this value is used as the key to determine" +
" which attribute on the selected element will be modified with the new value."))
.required(false)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.build();
private List<PropertyDescriptor> descriptors;
private Set<Relationship> relationships;
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(URL);
descriptors.add(CSS_SELECTOR);
descriptors.add(HTML_CHARSET);
descriptors.add(OUTPUT_TYPE);
descriptors.add(MODIFIED_VALUE);
descriptors.add(ATTRIBUTE_KEY);
this.descriptors = Collections.unmodifiableList(descriptors);
final Set<Relationship> relationships = new HashSet<Relationship>();
relationships.add(REL_ORIGINAL);
relationships.add(REL_SUCCESS);
relationships.add(REL_INVALID_HTML);
relationships.add(REL_NOT_FOUND);
this.relationships = Collections.unmodifiableSet(relationships);
}
@Override
public Set<Relationship> getRelationships() {
return this.relationships;
}
@Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return descriptors;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final FlowFile flowFile = session.get();
if (flowFile == null) {
return;
}
final Document doc;
final Elements eles;
try {
doc = parseHTMLDocumentFromFlowfile(flowFile, context, session);
eles = doc.select(context.getProperty(CSS_SELECTOR).evaluateAttributeExpressions(flowFile).getValue());
} catch (Exception ex) {
getLogger().error("Failed to extract HTML from {} due to {}; routing to {}", new Object[] {flowFile, ex.toString(), REL_INVALID_HTML.getName()}, ex);
session.transfer(flowFile, REL_INVALID_HTML);
return;
}
final String modifiedValue = context.getProperty(MODIFIED_VALUE).evaluateAttributeExpressions(flowFile).getValue();
if (eles == null || eles.size() == 0) {
// No element found
session.transfer(flowFile, REL_NOT_FOUND);
} else {
for (Element ele : eles) {
switch (context.getProperty(OUTPUT_TYPE).getValue()) {
case ELEMENT_HTML:
ele.html(modifiedValue);
break;
case ELEMENT_ATTRIBUTE:
ele.attr(context.getProperty(ATTRIBUTE_KEY).evaluateAttributeExpressions(flowFile).getValue(), modifiedValue);
break;
case ELEMENT_TEXT:
ele.text(modifiedValue);
break;
}
}
FlowFile ff = session.write(session.create(flowFile), new StreamCallback() {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
out.write(doc.html().getBytes(StandardCharsets.UTF_8));
}
});
ff = session.putAttribute(ff, NUM_ELEMENTS_MODIFIED_ATTR, new Integer(eles.size()).toString());
session.transfer(ff, REL_SUCCESS);
// Transfer the original HTML
session.transfer(flowFile, REL_ORIGINAL);
}
}
}

View File

@ -0,0 +1,165 @@
/*
* 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;
import org.apache.nifi.annotation.behavior.InputRequirement;
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.PropertyDescriptor;
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.ProcessorInitializationContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.StreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.jsoup.nodes.Document;
import org.jsoup.nodes.Element;
import org.jsoup.select.Elements;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.HashSet;
import java.util.Collections;
@Tags({"put", "html", "dom", "css", "element"})
@SupportsBatching
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
@CapabilityDescription("Places a new HTML element in the existing HTML DOM. The desired position for the new HTML element is specified by" +
" using CSS selector syntax. The incoming HTML is first converted into a HTML Document Object Model so that HTML DOM location may be located" +
" in a similar manner that CSS selectors are used to apply styles to HTML. The resulting HTML DOM is then \"queried\"" +
" using the user defined CSS selector string to find the position where the user desires to add the new HTML element." +
" Once the new HTML element is added to the DOM it is rendered to HTML and the result replaces the flowfile" +
" content with the updated HTML. A more thorough reference for the CSS selector syntax can be found at" +
" \"http://jsoup.org/apidocs/org/jsoup/select/Selector.html\"")
@SeeAlso({GetHTMLElement.class, ModifyHTMLElement.class})
public class PutHTMLElement extends AbstractHTMLProcessor {
public static final String APPEND_ELEMENT = "append-html";
public static final String PREPEND_ELEMENT = "prepend-html";
public static final PropertyDescriptor PUT_LOCATION_TYPE = new PropertyDescriptor.Builder()
.name("Element Insert Location Type")
.description("Controls whether the new element is prepended or appended to the children of the " +
"Element located by the CSS selector. EX: prepended value '<b>Hi</b>' inside of " +
"Element (using CSS Selector 'p') '<p>There</p>' would result in " +
"'<p><b>Hi</b>There</p>'. Appending the value would result in '<p>There<b>Hi</b></p>'")
.required(true)
.allowableValues(APPEND_ELEMENT, PREPEND_ELEMENT)
.defaultValue(APPEND_ELEMENT)
.build();
public static final PropertyDescriptor PUT_VALUE = new PropertyDescriptor.Builder()
.name("Put Value")
.description("Value used when creating the new Element. Value should be a valid HTML element. " +
"The text should be supplied unencoded: characters like '<', '>', etc will be properly HTML " +
"encoded in the resulting output.")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.build();
private List<PropertyDescriptor> descriptors;
private Set<Relationship> relationships;
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
descriptors.add(URL);
descriptors.add(CSS_SELECTOR);
descriptors.add(HTML_CHARSET);
descriptors.add(PUT_LOCATION_TYPE);
descriptors.add(PUT_VALUE);
this.descriptors = Collections.unmodifiableList(descriptors);
final Set<Relationship> relationships = new HashSet<Relationship>();
relationships.add(REL_ORIGINAL);
relationships.add(REL_SUCCESS);
relationships.add(REL_INVALID_HTML);
relationships.add(REL_NOT_FOUND);
this.relationships = Collections.unmodifiableSet(relationships);
}
@Override
public Set<Relationship> getRelationships() {
return this.relationships;
}
@Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return descriptors;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final FlowFile flowFile = session.get();
if (flowFile == null) {
return;
}
final Document doc;
final Elements eles;
try {
doc = parseHTMLDocumentFromFlowfile(flowFile, context, session);
eles = doc.select(context.getProperty(CSS_SELECTOR).evaluateAttributeExpressions().getValue());
} catch (Exception ex) {
getLogger().error("Failed to extract HTML from {} due to {}; routing to {}", new Object[] {flowFile, ex.toString(), REL_INVALID_HTML.getName()}, ex);
session.transfer(flowFile, REL_INVALID_HTML);
return;
}
if (eles == null || eles.isEmpty()) {
// No element found
session.transfer(flowFile, REL_NOT_FOUND);
} else {
final String putValue = context.getProperty(PUT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
for (final Element ele : eles) {
switch (context.getProperty(PUT_LOCATION_TYPE).getValue()) {
case APPEND_ELEMENT:
ele.append(putValue);
break;
case PREPEND_ELEMENT:
ele.prepend(putValue);
break;
}
}
FlowFile ff = session.write(session.create(flowFile), new StreamCallback() {
@Override
public void process(final InputStream in, final OutputStream out) throws IOException {
out.write(doc.html().getBytes(StandardCharsets.UTF_8));
}
});
session.transfer(ff, REL_SUCCESS);
// Transfer the original HTML
session.transfer(flowFile, REL_ORIGINAL);
}
}
}

View File

@ -0,0 +1,17 @@
# 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.
org.apache.nifi.GetHTMLElement
org.apache.nifi.ModifyHTMLElement
org.apache.nifi.PutHTMLElement

View File

@ -0,0 +1,28 @@
/*
* 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;
public abstract class AbstractHTMLTest {
protected final String ATL_WEATHER_TEXT = "Atlanta Weather";
protected final String GDR_WEATHER_TEXT = "<i>Grand Rapids Weather</i>";
protected final String ATL_WEATHER_LINK = "http://w1.weather.gov/obhistory/KPDK.html";
protected final String GR_WEATHER_LINK = "http://w1.weather.gov/obhistory/KGRR.html";
protected final String AUTHOR_NAME = "Jeremy Dyer";
protected final String ATL_ID = "ATL";
protected final String GDR_ID = "GDR";
}

View File

@ -0,0 +1,275 @@
/*
* 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;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
import org.jsoup.select.Selector;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.lang.Exception;
import java.net.URL;
import java.util.List;
public class TestGetHTMLElement extends AbstractHTMLTest {
private TestRunner testRunner;
@Before
public void init() {
testRunner = TestRunners.newTestRunner(GetHTMLElement.class);
testRunner.setProperty(GetHTMLElement.URL, "http://localhost");
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_HTML);
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.HTML_CHARSET, "UTF-8");
}
@Test
public void testCSSSelectorSyntaxValidator() throws IOException {
Document doc = Jsoup.parse(new URL("http://www.google.com"), 5000);
try {
doc.select("---jeremy");
} catch (Selector.SelectorParseException ex) {
ex.printStackTrace();
}
}
@Test
public void testNoElementFound() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "b"); //Bold element is not present in sample HTML
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 1);
}
@Test
public void testInvalidSelector() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "InvalidCSSSelectorSyntax");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 1);
}
@Test
public void testSingleElementFound() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "head");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
}
@Test
public void testMultipleElementFound() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "a");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 3);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
}
@Test
public void testElementFoundWriteToAttribute() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "#" + ATL_ID);
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_ATTRIBUTE);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_ATTRIBUTE);
testRunner.setProperty(GetHTMLElement.ATTRIBUTE_KEY, "href");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(GetHTMLElement.REL_SUCCESS);
ffs.get(0).assertAttributeEquals(GetHTMLElement.HTML_ELEMENT_ATTRIBUTE_NAME, ATL_WEATHER_LINK);
}
@Test
public void testElementFoundWriteToContent() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "#" + ATL_ID);
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_ATTRIBUTE);
testRunner.setProperty(GetHTMLElement.ATTRIBUTE_KEY, "href");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(GetHTMLElement.REL_SUCCESS);
ffs.get(0).assertContentEquals(ATL_WEATHER_LINK);
}
@Test
public void testValidPrependValueToFoundElement() throws Exception {
final String PREPEND_VALUE = "TestPrepend";
testRunner.setProperty(GetHTMLElement.PREPEND_ELEMENT_VALUE, PREPEND_VALUE);
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "#" + ATL_ID);
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_ATTRIBUTE);
testRunner.setProperty(GetHTMLElement.ATTRIBUTE_KEY, "href");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(GetHTMLElement.REL_SUCCESS);
ffs.get(0).assertContentEquals(PREPEND_VALUE + ATL_WEATHER_LINK);
}
@Test
public void testValidPrependValueToNotFoundElement() throws Exception {
final String PREPEND_VALUE = "TestPrepend";
testRunner.setProperty(GetHTMLElement.PREPEND_ELEMENT_VALUE, PREPEND_VALUE);
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "b");
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_TEXT);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 1);
}
@Test
public void testValidAppendValueToFoundElement() throws Exception {
final String APPEND_VALUE = "TestAppend";
testRunner.setProperty(GetHTMLElement.APPEND_ELEMENT_VALUE, APPEND_VALUE);
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "#" + ATL_ID);
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_ATTRIBUTE);
testRunner.setProperty(GetHTMLElement.ATTRIBUTE_KEY, "href");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(GetHTMLElement.REL_SUCCESS);
ffs.get(0).assertContentEquals(ATL_WEATHER_LINK + APPEND_VALUE);
}
@Test
public void testValidAppendValueToNotFoundElement() throws Exception {
final String APPEND_VALUE = "TestAppend";
testRunner.setProperty(GetHTMLElement.APPEND_ELEMENT_VALUE, APPEND_VALUE);
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "b");
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_TEXT);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 1);
}
@Test
public void testExtractAttributeFromElement() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "meta[name=author]");
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_ATTRIBUTE);
testRunner.setProperty(GetHTMLElement.ATTRIBUTE_KEY, "Content");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(GetHTMLElement.REL_SUCCESS);
ffs.get(0).assertContentEquals(AUTHOR_NAME);
}
@Test
public void testExtractTextFromElement() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "#" + ATL_ID);
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_TEXT);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(GetHTMLElement.REL_SUCCESS);
ffs.get(0).assertContentEquals(ATL_WEATHER_TEXT);
}
@Test
public void testExtractHTMLFromElement() throws Exception {
testRunner.setProperty(GetHTMLElement.CSS_SELECTOR, "#" + GDR_ID);
testRunner.setProperty(GetHTMLElement.DESTINATION, GetHTMLElement.DESTINATION_CONTENT);
testRunner.setProperty(GetHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_HTML);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(GetHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(GetHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(GetHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(GetHTMLElement.REL_SUCCESS);
ffs.get(0).assertContentEquals(GDR_WEATHER_TEXT);
}
}

View File

@ -0,0 +1,204 @@
/*
* 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;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
import org.jsoup.nodes.Element;
import org.jsoup.select.Elements;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.util.List;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
public class TestModifyHTMLElement extends AbstractHTMLTest {
private TestRunner testRunner;
@Before
public void init() {
testRunner = TestRunners.newTestRunner(ModifyHTMLElement.class);
testRunner = TestRunners.newTestRunner(ModifyHTMLElement.class);
testRunner.setProperty(ModifyHTMLElement.URL, "http://localhost");
testRunner.setProperty(ModifyHTMLElement.OUTPUT_TYPE, GetHTMLElement.ELEMENT_HTML);
testRunner.setProperty(ModifyHTMLElement.HTML_CHARSET, "UTF-8");
}
@Test
public void testModifyText() throws Exception {
final String MOD_VALUE = "Newly modified value to replace " + ATL_WEATHER_TEXT;
testRunner.setProperty(ModifyHTMLElement.CSS_SELECTOR, "#" + ATL_ID);
testRunner.setProperty(ModifyHTMLElement.OUTPUT_TYPE, ModifyHTMLElement.ELEMENT_TEXT);
testRunner.setProperty(ModifyHTMLElement.MODIFIED_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(ModifyHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(ModifyHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("#" + ATL_ID);
Element ele = eles.get(0);
assertTrue(StringUtils.equals(MOD_VALUE, ele.text()));
}
@Test
public void testModifyHTMLWithExpressionLanguage() throws Exception {
final String MOD_VALUE = "Newly modified value to replace " + ATL_WEATHER_TEXT;
testRunner.setProperty(ModifyHTMLElement.CSS_SELECTOR, "#" + ATL_ID);
testRunner.setProperty(ModifyHTMLElement.OUTPUT_TYPE, ModifyHTMLElement.ELEMENT_TEXT);
testRunner.setProperty(ModifyHTMLElement.MODIFIED_VALUE, "${\" " + MOD_VALUE + " \":trim()}");
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(ModifyHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(ModifyHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("#" + ATL_ID);
Element ele = eles.get(0);
assertNotNull(ele.text());
}
@Test
public void testModifyHTML() throws Exception {
final String MOD_VALUE = "Newly modified HTML to replace " + GDR_WEATHER_TEXT;
testRunner.setProperty(ModifyHTMLElement.CSS_SELECTOR, "#" + GDR_ID);
testRunner.setProperty(ModifyHTMLElement.OUTPUT_TYPE, ModifyHTMLElement.ELEMENT_HTML);
testRunner.setProperty(ModifyHTMLElement.MODIFIED_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(ModifyHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(ModifyHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("#" + GDR_ID);
Element ele = eles.get(0);
assertTrue(StringUtils.equals(MOD_VALUE, ele.html()));
}
@Test
public void testModifyAttribute() throws Exception {
final String MOD_VALUE = "http://localhost/newlink";
testRunner.setProperty(ModifyHTMLElement.CSS_SELECTOR, "#" + GDR_ID);
testRunner.setProperty(ModifyHTMLElement.OUTPUT_TYPE, ModifyHTMLElement.ELEMENT_ATTRIBUTE);
testRunner.setProperty(ModifyHTMLElement.ATTRIBUTE_KEY, "href");
testRunner.setProperty(ModifyHTMLElement.MODIFIED_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(ModifyHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(ModifyHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("#" + GDR_ID);
Element ele = eles.get(0);
assertTrue(StringUtils.equals(MOD_VALUE, ele.attr("href")));
}
@Test
public void testModifyElementNotFound() throws Exception {
final String MOD_VALUE = "http://localhost/newlink";
testRunner.setProperty(ModifyHTMLElement.CSS_SELECTOR, "b");
testRunner.setProperty(ModifyHTMLElement.OUTPUT_TYPE, ModifyHTMLElement.ELEMENT_HTML);
testRunner.setProperty(ModifyHTMLElement.MODIFIED_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(ModifyHTMLElement.REL_SUCCESS, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_ORIGINAL, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_NOT_FOUND, 1);
}
@Test
public void testModifyValueContainsHTMLCharacters() throws Exception {
final String MOD_VALUE = "Text that contains > and < characters";
testRunner.setProperty(ModifyHTMLElement.CSS_SELECTOR, "#" + GDR_ID);
testRunner.setProperty(ModifyHTMLElement.OUTPUT_TYPE, ModifyHTMLElement.ELEMENT_HTML);
testRunner.setProperty(ModifyHTMLElement.MODIFIED_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(ModifyHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(ModifyHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(ModifyHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(ModifyHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("#" + GDR_ID);
Element ele = eles.get(0);
assertTrue(StringUtils.equals(MOD_VALUE, ele.text()));
assertTrue(StringUtils.equals(MOD_VALUE.replace(">", "&gt;").replace("<", "&lt;"), ele.html()));
}
}

View File

@ -0,0 +1,127 @@
/*
* 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;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
import org.jsoup.nodes.Element;
import org.jsoup.select.Elements;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.util.List;
import static org.junit.Assert.assertTrue;
public class TestPutHTMLElement extends AbstractHTMLTest {
private TestRunner testRunner;
@Before
public void init() {
testRunner = TestRunners.newTestRunner(PutHTMLElement.class);
testRunner.setProperty(PutHTMLElement.URL, "http://localhost");
}
@Test
public void testAddNewElementToRoot() throws Exception {
final String MOD_VALUE = "<p>modified value</p>";
testRunner.setProperty(PutHTMLElement.CSS_SELECTOR, "body");
testRunner.setProperty(PutHTMLElement.PUT_LOCATION_TYPE, PutHTMLElement.PREPEND_ELEMENT);
testRunner.setProperty(PutHTMLElement.PUT_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(PutHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(PutHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(PutHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(PutHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(PutHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("body > p");
Element ele = eles.get(0);
assertTrue(StringUtils.equals(MOD_VALUE.replace("<p>", "").replace("</p>", ""), ele.html()));
}
@Test
public void testPrependPElementToDiv() throws Exception {
final String MOD_VALUE = "<p>modified value</p>";
testRunner.setProperty(PutHTMLElement.CSS_SELECTOR, "#put");
testRunner.setProperty(PutHTMLElement.PUT_LOCATION_TYPE, PutHTMLElement.PREPEND_ELEMENT);
testRunner.setProperty(PutHTMLElement.PUT_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(PutHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(PutHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(PutHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(PutHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(PutHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("#put");
Element ele = eles.get(0);
assertTrue(StringUtils.equals("<p>modified value</p> \n<a href=\"httpd://localhost\"></a>", ele.html()));
}
@Test
public void testAppendPElementToDiv() throws Exception {
final String MOD_VALUE = "<p>modified value</p>";
testRunner.setProperty(PutHTMLElement.CSS_SELECTOR, "#put");
testRunner.setProperty(PutHTMLElement.PUT_LOCATION_TYPE, PutHTMLElement.APPEND_ELEMENT);
testRunner.setProperty(PutHTMLElement.PUT_VALUE, MOD_VALUE);
testRunner.enqueue(new File("src/test/resources/Weather.html").toPath());
testRunner.run();
testRunner.assertTransferCount(PutHTMLElement.REL_SUCCESS, 1);
testRunner.assertTransferCount(PutHTMLElement.REL_INVALID_HTML, 0);
testRunner.assertTransferCount(PutHTMLElement.REL_ORIGINAL, 1);
testRunner.assertTransferCount(PutHTMLElement.REL_NOT_FOUND, 0);
List<MockFlowFile> ffs = testRunner.getFlowFilesForRelationship(PutHTMLElement.REL_SUCCESS);
assertTrue(ffs.size() == 1);
String data = new String(testRunner.getContentAsByteArray(ffs.get(0)));
//Contents will be the entire HTML doc. So lets use Jsoup again just the grab the element we want.
Document doc = Jsoup.parse(data);
Elements eles = doc.select("#put");
Element ele = eles.get(0);
assertTrue(StringUtils.equals("<a href=\"httpd://localhost\"></a> \n" +
"<p>modified value</p>", ele.html()));
}
}

View File

@ -0,0 +1,25 @@
<!doctype html>
<html lang="en">
<head>
<title>NiFi HTML Parsing Demo</title>
<meta charset="utf-8">
<meta name="description" content="NiFi HTML Parsing Demo">
<meta name="author" content="Jeremy Dyer">
<link rel="stylesheet" href="css/styles.css?v=1.0">
<!--[if lt IE 9]>
<script src="http://html5shiv.googlecode.com/svn/trunk/html5.js"></script>
<![endif]-->
</head>
<body>
<script src="js/scripts.js"></script>
<p>Check out this weather!
<a id="ATL" href="http://w1.weather.gov/obhistory/KPDK.html">Atlanta Weather</a>
</p>
<p>I guess it could be colder ...
<a id="GDR" href="http://w1.weather.gov/obhistory/KGRR.html"><i>Grand Rapids Weather</i></a>
</p>
<div id="put"><a href="httpd://localhost" /></div>
</body>
</html>

View File

@ -0,0 +1,43 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-bundles</artifactId>
<version>0.4.2-SNAPSHOT</version>
</parent>
<artifactId>nifi-html-bundle</artifactId>
<packaging>pom</packaging>
<modules>
<module>nifi-html-processors</module>
<module>nifi-html-nar</module>
</modules>
<dependencyManagement>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-html-processors</artifactId>
<version>0.4.2-SNAPSHOT</version>
</dependency>
</dependencies>
</dependencyManagement>
</project>

View File

@ -238,6 +238,7 @@ public class MergeContent extends BinFiles {
.description("If specified, like FlowFiles will be binned together, where 'like FlowFiles' means FlowFiles that have the same value for "
+ "this Attribute. If not specified, FlowFiles are bundled by the order in which they are pulled from the queue.")
.required(false)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
.defaultValue(null)
.build();
@ -376,7 +377,8 @@ public class MergeContent extends BinFiles {
@Override
protected String getGroupId(final ProcessContext context, final FlowFile flowFile) {
final String correlationAttributeName = context.getProperty(CORRELATION_ATTRIBUTE_NAME).getValue();
final String correlationAttributeName = context.getProperty(CORRELATION_ATTRIBUTE_NAME)
.evaluateAttributeExpressions(flowFile).getValue();
String groupId = correlationAttributeName == null ? null : flowFile.getAttribute(correlationAttributeName);
// when MERGE_STRATEGY is Defragment and correlationAttributeName is null then bin by fragment.identifier

View File

@ -141,6 +141,8 @@ public class PostHTTP extends AbstractProcessor {
public static final String LOCATION_URI_INTENT_NAME = "x-location-uri-intent";
public static final String LOCATION_URI_INTENT_VALUE = "flowfile-hold";
public static final String GZIPPED_HEADER = "flowfile-gzipped";
public static final String CONTENT_ENCODING_HEADER = "Content-Encoding";
public static final String CONTENT_ENCODING_GZIP_VALUE = "gzip";
public static final String PROTOCOL_VERSION_HEADER = "x-nifi-transfer-protocol-version";
public static final String TRANSACTION_ID_HEADER = "x-nifi-transaction-id";
@ -534,12 +536,7 @@ public class PostHTTP extends AbstractProcessor {
destinationAccepts = config.getDestinationAccepts();
if (destinationAccepts == null) {
try {
if (sendAsFlowFile) {
destinationAccepts = getDestinationAcceptance(client, url, getLogger(), transactionId);
} else {
destinationAccepts = new DestinationAccepts(false, false, false, false, null);
}
destinationAccepts = getDestinationAcceptance(sendAsFlowFile, client, url, getLogger(), transactionId);
config.setDestinationAccepts(destinationAccepts);
} catch (final IOException e) {
flowFile = session.penalize(flowFile);
@ -552,14 +549,14 @@ public class PostHTTP extends AbstractProcessor {
}
}
// if we are not sending as flowfile, or if the destination doesn't accept V3 or V2 (streaming) format,
// then only use a single FlowFile
if (!sendAsFlowFile || !destinationAccepts.isFlowFileV3Accepted() && !destinationAccepts.isFlowFileV2Accepted()) {
bytesToSend += flowFile.getSize();
if (bytesToSend > maxBatchBytes.longValue()) {
break;
}
bytesToSend += flowFile.getSize();
if (bytesToSend > maxBatchBytes.longValue()) {
// if we are not sending as flowfile, or if the destination doesn't accept V3 or V2 (streaming) format,
// then only use a single FlowFile
if (!sendAsFlowFile || !destinationAccepts.isFlowFileV3Accepted() && !destinationAccepts.isFlowFileV2Accepted()) {
break;
}
}
@ -673,7 +670,11 @@ public class PostHTTP extends AbstractProcessor {
post.setHeader(PROTOCOL_VERSION_HEADER, PROTOCOL_VERSION);
post.setHeader(TRANSACTION_ID_HEADER, transactionId);
if (compressionLevel > 0 && accepts.isGzipAccepted()) {
post.setHeader(GZIPPED_HEADER, "true");
if (sendAsFlowFile) {
post.setHeader(GZIPPED_HEADER, "true");
} else {
post.setHeader(CONTENT_ENCODING_HEADER, CONTENT_ENCODING_GZIP_VALUE);
}
}
// Do the actual POST
@ -841,57 +842,58 @@ public class PostHTTP extends AbstractProcessor {
}
}
private DestinationAccepts getDestinationAcceptance(final HttpClient client, final String uri, final ProcessorLog logger, final String transactionId) throws IOException {
private DestinationAccepts getDestinationAcceptance(final boolean sendAsFlowFile, final HttpClient client, final String uri,
final ProcessorLog logger, final String transactionId) throws IOException {
final HttpHead head = new HttpHead(uri);
head.addHeader(TRANSACTION_ID_HEADER, transactionId);
if (sendAsFlowFile) {
head.addHeader(TRANSACTION_ID_HEADER, transactionId);
}
final HttpResponse response = client.execute(head);
// we assume that the destination can support FlowFile v1 always when the processor is also configured to send as a FlowFile
// otherwise, we do not bother to make any determinations concerning this compatibility
final boolean acceptsFlowFileV1 = sendAsFlowFile;
boolean acceptsFlowFileV2 = false;
boolean acceptsFlowFileV3 = false;
boolean acceptsGzip = false;
Integer protocolVersion = null;
final int statusCode = response.getStatusLine().getStatusCode();
if (statusCode == Status.METHOD_NOT_ALLOWED.getStatusCode()) {
// we assume that the destination can support FlowFile v1 always.
return new DestinationAccepts(false, false, true, false, null);
return new DestinationAccepts(acceptsFlowFileV3, acceptsFlowFileV2, acceptsFlowFileV1, false, null);
} else if (statusCode == Status.OK.getStatusCode()) {
boolean acceptsFlowFileV3 = false;
boolean acceptsFlowFileV2 = false;
boolean acceptsFlowFileV1 = true;
boolean acceptsGzip = false;
Integer protocolVersion = null;
Header[] headers = response.getHeaders(ACCEPT);
if (headers != null) {
for (final Header header : headers) {
for (final String accepted : header.getValue().split(",")) {
final String trimmed = accepted.trim();
if (trimmed.equals(APPLICATION_FLOW_FILE_V3)) {
acceptsFlowFileV3 = true;
} else if (trimmed.equals(APPLICATION_FLOW_FILE_V2)) {
acceptsFlowFileV2 = true;
} else {
// we assume that the destination accepts FlowFile V1 because legacy versions
// of NiFi that accepted V1 did not use an Accept header to indicate it... or
// any other header. So the bets thing we can do is just assume that V1 is
// accepted, if we're going to send as FlowFile.
acceptsFlowFileV1 = true;
// If configured to send as a flowfile, determine the capabilities of the endpoint
if (sendAsFlowFile) {
if (headers != null) {
for (final Header header : headers) {
for (final String accepted : header.getValue().split(",")) {
final String trimmed = accepted.trim();
if (trimmed.equals(APPLICATION_FLOW_FILE_V3)) {
acceptsFlowFileV3 = true;
} else if (trimmed.equals(APPLICATION_FLOW_FILE_V2)) {
acceptsFlowFileV2 = true;
}
}
}
}
}
final Header destinationVersion = response.getFirstHeader(PROTOCOL_VERSION_HEADER);
if (destinationVersion != null) {
try {
protocolVersion = Integer.valueOf(destinationVersion.getValue());
} catch (final NumberFormatException e) {
// nothing to do here really.... it's an invalid value, so treat the same as if not specified
final Header destinationVersion = response.getFirstHeader(PROTOCOL_VERSION_HEADER);
if (destinationVersion != null) {
try {
protocolVersion = Integer.valueOf(destinationVersion.getValue());
} catch (final NumberFormatException e) {
// nothing to do here really.... it's an invalid value, so treat the same as if not specified
}
}
}
if (acceptsFlowFileV3) {
logger.debug("Connection to URI " + uri + " will be using Content Type " + APPLICATION_FLOW_FILE_V3 + " if sending data as FlowFile");
} else if (acceptsFlowFileV2) {
logger.debug("Connection to URI " + uri + " will be using Content Type " + APPLICATION_FLOW_FILE_V2 + " if sending data as FlowFile");
} else if (acceptsFlowFileV1) {
logger.debug("Connection to URI " + uri + " will be using Content Type " + APPLICATION_FLOW_FILE_V1 + " if sending data as FlowFile");
if (acceptsFlowFileV3) {
logger.debug("Connection to URI " + uri + " will be using Content Type " + APPLICATION_FLOW_FILE_V3 + " if sending data as FlowFile");
} else if (acceptsFlowFileV2) {
logger.debug("Connection to URI " + uri + " will be using Content Type " + APPLICATION_FLOW_FILE_V2 + " if sending data as FlowFile");
} else if (acceptsFlowFileV1) {
logger.debug("Connection to URI " + uri + " will be using Content Type " + APPLICATION_FLOW_FILE_V1 + " if sending data as FlowFile");
}
}
headers = response.getHeaders(ACCEPT_ENCODING);

View File

@ -71,6 +71,9 @@ public class CaptureServlet extends HttpServlet {
protected void doHead(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
response.setHeader("Accept", "application/flowfile-v3,application/flowfile-v2");
response.setHeader("x-nifi-transfer-protocol-version", "1");
response.setHeader("Accept-Encoding", "gzip");
// Unless an acceptGzip parameter is explicitly set to false, respond that this server accepts gzip
if (!Boolean.toString(false).equalsIgnoreCase(request.getParameter("acceptGzip"))) {
response.setHeader("Accept-Encoding", "gzip");
}
}
}

View File

@ -24,6 +24,7 @@ import java.io.ByteArrayOutputStream;
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.ssl.SSLContextService;
import org.apache.nifi.ssl.StandardSSLContextService;
@ -299,4 +300,74 @@ public class TestPostHTTP {
Map<String, String> lastPostHeaders = servlet.getLastPostHeaders();
Assert.assertEquals(suppliedMimeType, lastPostHeaders.get(PostHTTP.CONTENT_TYPE_HEADER));
}
@Test
public void testSendWithCompressionServerAcceptGzip() throws Exception {
setup(null);
final String suppliedMimeType = "text/plain";
runner.setProperty(PostHTTP.URL, server.getUrl());
runner.setProperty(PostHTTP.CONTENT_TYPE, suppliedMimeType);
runner.setProperty(PostHTTP.COMPRESSION_LEVEL, "9");
final Map<String, String> attrs = new HashMap<>();
attrs.put(CoreAttributes.MIME_TYPE.key(), "text/plain");
runner.enqueue(StringUtils.repeat("This is the song that never ends. It goes on and on my friend.", 100).getBytes(), attrs);
runner.run(1);
runner.assertAllFlowFilesTransferred(PostHTTP.REL_SUCCESS);
Map<String, String> lastPostHeaders = servlet.getLastPostHeaders();
Assert.assertEquals(suppliedMimeType, lastPostHeaders.get(PostHTTP.CONTENT_TYPE_HEADER));
// Ensure that a 'Content-Encoding' header was set with a 'gzip' value
Assert.assertEquals(PostHTTP.CONTENT_ENCODING_GZIP_VALUE, lastPostHeaders.get(PostHTTP.CONTENT_ENCODING_HEADER));
}
@Test
public void testSendWithoutCompressionServerAcceptGzip() throws Exception {
setup(null);
final String suppliedMimeType = "text/plain";
runner.setProperty(PostHTTP.URL, server.getUrl());
runner.setProperty(PostHTTP.CONTENT_TYPE, suppliedMimeType);
runner.setProperty(PostHTTP.COMPRESSION_LEVEL, "0");
final Map<String, String> attrs = new HashMap<>();
attrs.put(CoreAttributes.MIME_TYPE.key(), "text/plain");
runner.enqueue(StringUtils.repeat("This is the song that never ends. It goes on and on my friend.", 100).getBytes(), attrs);
runner.run(1);
runner.assertAllFlowFilesTransferred(PostHTTP.REL_SUCCESS);
Map<String, String> lastPostHeaders = servlet.getLastPostHeaders();
Assert.assertEquals(suppliedMimeType, lastPostHeaders.get(PostHTTP.CONTENT_TYPE_HEADER));
// Ensure that the request was not sent with a 'Content-Encoding' header
Assert.assertNull(lastPostHeaders.get(PostHTTP.CONTENT_ENCODING_HEADER));
}
@Test
public void testSendWithCompressionServerNotAcceptGzip() throws Exception {
setup(null);
final String suppliedMimeType = "text/plain";
// Specify a property to the URL to have the CaptureServlet specify it doesn't accept gzip
runner.setProperty(PostHTTP.URL, server.getUrl()+"?acceptGzip=false");
runner.setProperty(PostHTTP.CONTENT_TYPE, suppliedMimeType);
runner.setProperty(PostHTTP.COMPRESSION_LEVEL, "9");
final Map<String, String> attrs = new HashMap<>();
attrs.put(CoreAttributes.MIME_TYPE.key(), "text/plain");
runner.enqueue(StringUtils.repeat("This is the song that never ends. It goes on and on my friend.", 100).getBytes(), attrs);
runner.run(1);
runner.assertAllFlowFilesTransferred(PostHTTP.REL_SUCCESS);
Map<String, String> lastPostHeaders = servlet.getLastPostHeaders();
Assert.assertEquals(suppliedMimeType, lastPostHeaders.get(PostHTTP.CONTENT_TYPE_HEADER));
// Ensure that the request was not sent with a 'Content-Encoding' header
Assert.assertNull(lastPostHeaders.get(PostHTTP.CONTENT_ENCODING_HEADER));
}
}

View File

@ -21,7 +21,6 @@ import static org.junit.Assert.assertNotNull;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.Field;
@ -44,14 +43,17 @@ import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestJdbcCommon {
final static String DB_LOCATION = "target/db";
@Rule
public TemporaryFolder folder = new TemporaryFolder();
@BeforeClass
public static void setup() {
@ -65,10 +67,9 @@ public class TestJdbcCommon {
public void testCreateSchema() throws ClassNotFoundException, SQLException {
// remove previous test database, if any
final File dbLocation = new File(DB_LOCATION);
dbLocation.delete();
folder.delete();
final Connection con = createConnection();
final Connection con = createConnection(folder.getRoot().getAbsolutePath());
final Statement st = con.createStatement();
try {
@ -102,10 +103,9 @@ public class TestJdbcCommon {
@Test
public void testConvertToBytes() throws ClassNotFoundException, SQLException, IOException {
// remove previous test database, if any
final File dbLocation = new File(DB_LOCATION);
dbLocation.delete();
folder.delete();
final Connection con = createConnection();
final Connection con = createConnection(folder.getRoot().getAbsolutePath());
final Statement st = con.createStatement();
try {
@ -136,8 +136,8 @@ public class TestJdbcCommon {
final InputStream instream = new ByteArrayInputStream(serializedBytes);
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(instream, datumReader)) {
GenericRecord record = null;
while (dataFileReader.hasNext()) {
// Reuse record object by passing it to next(). This saves us from
@ -282,8 +282,8 @@ public class TestJdbcCommon {
final InputStream instream = new ByteArrayInputStream(serializedBytes);
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(instream, datumReader)) {
GenericRecord record = null;
while (dataFileReader.hasNext()) {
record = dataFileReader.next(record);
@ -300,11 +300,9 @@ public class TestJdbcCommon {
assertNotNull(clazz);
}
private Connection createConnection() throws ClassNotFoundException, SQLException {
private Connection createConnection(String location) throws ClassNotFoundException, SQLException {
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
final Connection con = DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
return con;
return DriverManager.getConnection("jdbc:derby:" + location + ";create=true");
}
}

View File

@ -18,7 +18,6 @@ package org.apache.nifi.processors.standard.util;
import static org.junit.Assert.assertEquals;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
@ -36,7 +35,9 @@ import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
/**
* Test streaming using large number of result set rows. 1. Read data from
@ -52,7 +53,8 @@ import org.junit.Test;
*/
public class TestJdbcHugeStream {
final static String DB_LOCATION = "target/db";
@Rule
public TemporaryFolder folder = new TemporaryFolder();
@BeforeClass
public static void setup() {
@ -63,10 +65,9 @@ public class TestJdbcHugeStream {
public void readSend2StreamHuge_FileBased() throws ClassNotFoundException, SQLException, IOException {
// remove previous test database, if any
final File dbLocation = new File(DB_LOCATION);
dbLocation.delete();
folder.delete();
try (final Connection con = createConnection()) {
try (final Connection con = createConnection(folder.getRoot().getAbsolutePath())) {
loadTestData2Database(con, 100, 100, 100);
try (final Statement st = con.createStatement()) {
@ -127,17 +128,17 @@ public class TestJdbcHugeStream {
// tables may not exist, this is not serious problem.
try {
st.executeUpdate(dropPersons);
} catch (final Exception e) {
} catch (final Exception ignored) {
}
try {
st.executeUpdate(dropProducts);
} catch (final Exception e) {
} catch (final Exception ignored) {
}
try {
st.executeUpdate(dropRelationships);
} catch (final Exception e) {
} catch (final Exception ignored) {
}
st.executeUpdate(createPersons);
@ -186,10 +187,9 @@ public class TestJdbcHugeStream {
return new String(text);
}
private Connection createConnection() throws ClassNotFoundException, SQLException {
private Connection createConnection(String location) throws ClassNotFoundException, SQLException {
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
final Connection con = DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
return con;
return DriverManager.getConnection("jdbc:derby:" + location + ";create=true");
}
}

View File

@ -20,7 +20,6 @@ import static org.junit.Assert.assertNotNull;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.sql.Connection;
@ -35,7 +34,9 @@ import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
/**
* Useless test, Derby is so much different from MySQL
@ -46,7 +47,8 @@ import org.junit.Test;
@Ignore
public class TestJdbcTypesDerby {
final static String DB_LOCATION = "target/db";
@Rule
public TemporaryFolder folder = new TemporaryFolder();
@BeforeClass
public static void setup() {
@ -59,9 +61,9 @@ public class TestJdbcTypesDerby {
+ " password varchar(255) DEFAULT NULL, "
+ " activation_code varchar(255) DEFAULT NULL, "
+ " forgotten_password_code varchar(255) DEFAULT NULL, "
+ " forgotten_password_time datetime DEFAULT NULL, "
+ " created datetime NOT NULL, "
+ " active tinyint NOT NULL DEFAULT 0, "
+ " forgotten_password_time DATE DEFAULT NULL, "
+ " created DATE NOT NULL, "
+ " active CHAR NOT NULL DEFAULT 'N', "
+ " home_module_id int DEFAULT NULL, "
+ " PRIMARY KEY (id) ) " ;
@ -70,10 +72,9 @@ public class TestJdbcTypesDerby {
@Test
public void testSQLTypesMapping() throws ClassNotFoundException, SQLException, IOException {
// remove previous test database, if any
final File dbLocation = new File(DB_LOCATION);
dbLocation.delete();
folder.delete();
final Connection con = createConnection();
final Connection con = createConnection(folder.getRoot().getAbsolutePath());
final Statement st = con.createStatement();
try {
@ -103,8 +104,8 @@ public class TestJdbcTypesDerby {
final InputStream instream = new ByteArrayInputStream(serializedBytes);
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(instream, datumReader)) {
GenericRecord record = null;
while (dataFileReader.hasNext()) {
// Reuse record object by passing it to next(). This saves us from
@ -123,11 +124,10 @@ public class TestJdbcTypesDerby {
assertNotNull(clazz);
}
private Connection createConnection() throws ClassNotFoundException, SQLException {
private Connection createConnection(String location) throws ClassNotFoundException, SQLException {
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
final Connection con = DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
return con;
return DriverManager.getConnection("jdbc:derby:" + location + ";create=true");
}
}

View File

@ -20,7 +20,6 @@ import static org.junit.Assert.assertNotNull;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.sql.Connection;
@ -34,11 +33,14 @@ import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestJdbcTypesH2 {
final static String DB_LOCATION = "~/var/test/h2";
@Rule
public TemporaryFolder folder = new TemporaryFolder();
@BeforeClass
public static void setup() {
@ -75,11 +77,7 @@ public class TestJdbcTypesH2 {
@Test
public void testSQLTypesMapping() throws ClassNotFoundException, SQLException, IOException {
// remove previous test database, if any
final File dbLocation = new File(DB_LOCATION);
dbLocation.delete();
final Connection con = createConnection();
final Connection con = createConnection(folder.getRoot().getAbsolutePath());
final Statement st = con.createStatement();
try {
@ -114,8 +112,8 @@ public class TestJdbcTypesH2 {
final InputStream instream = new ByteArrayInputStream(serializedBytes);
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<>();
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<>(instream, datumReader)) {
GenericRecord record = null;
while (dataFileReader.hasNext()) {
// Reuse record object by passing it to next(). This saves us from
@ -132,18 +130,17 @@ public class TestJdbcTypesH2 {
public void testDriverLoad() throws ClassNotFoundException, SQLException {
// final Class<?> clazz = Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
Connection con = createConnection();
Connection con = createConnection(folder.getRoot().getAbsolutePath());
assertNotNull(con);
con.close();
}
private Connection createConnection() throws ClassNotFoundException, SQLException {
private Connection createConnection(String location) throws ClassNotFoundException, SQLException {
// Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
String connectionString = "jdbc:h2:file:" + DB_LOCATION + "/testdb7";
final Connection con = DriverManager.getConnection(connectionString, "SA", "");
return con;
String connectionString = "jdbc:h2:file:" + location + "/testdb7";
return DriverManager.getConnection(connectionString, "SA", "");
}
}

View File

@ -50,6 +50,7 @@
<module>nifi-azure-bundle</module>
<module>nifi-ldap-iaa-providers-bundle</module>
<module>nifi-riemann-bundle</module>
<module>nifi-html-bundle</module>
</modules>
<dependencyManagement>
<dependencies>

41
pom.xml
View File

@ -773,7 +773,11 @@ language governing permissions and limitations under the License. -->
<version>6.8.8</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.jsoup</groupId>
<artifactId>jsoup</artifactId>
<version>1.8.3</version>
</dependency>
<!-- NiFi modules -->
<dependency>
<groupId>org.apache.nifi</groupId>
@ -911,6 +915,12 @@ language governing permissions and limitations under the License. -->
<version>0.4.2-SNAPSHOT</version>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-html-nar</artifactId>
<version>0.4.2-SNAPSHOT</version>
<type>nar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-kite-nar</artifactId>
@ -1160,6 +1170,11 @@ language governing permissions and limitations under the License. -->
<argLine combine.children="append">-Xmx1G -Djava.net.preferIPv4Stack=true</argLine>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-failsafe-plugin</artifactId>
<version>2.18</version>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-assembly-plugin</artifactId>
@ -1442,6 +1457,28 @@ language governing permissions and limitations under the License. -->
</plugins>
</build>
<profiles>
<profile>
<!-- Performs execution of Integration Tests using the Maven FailSafe Plugin. The view of integration tests in this context
are those tests interfacing with external sources and services requiring additional resources or credentials that cannot
be explicitly provided. -->
<id>integration-tests</id>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-failsafe-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>integration-test</goal>
<goal>verify</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
<profile>
<!-- Checks style and licensing requirements. This is a good idea to run
for contributions and for the release process. While it would be nice to
@ -1505,4 +1542,4 @@ language governing permissions and limitations under the License. -->
</build>
</profile>
</profiles>
</project>
</project>