NIFI-2912 Allow custom text in GenerateFlowFile

This closes #1206.

Signed-off-by: Andy LoPresto <alopresto@apache.org>
This commit is contained in:
Pierre Villard 2016-11-11 00:44:06 +01:00 committed by Andy LoPresto
parent 9f417a84b9
commit 06f191ca10
No known key found for this signature in database
GPG Key ID: 3C6EF65B2F7DEF69
2 changed files with 89 additions and 2 deletions

View File

@ -19,6 +19,7 @@ package org.apache.nifi.processors.standard;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
@ -33,6 +34,8 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
@ -58,6 +61,7 @@ public class GenerateFlowFile extends AbstractProcessor {
.name("File Size")
.description("The size of the file that will be used")
.required(true)
.defaultValue("0B")
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
.build();
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
@ -71,7 +75,7 @@ public class GenerateFlowFile extends AbstractProcessor {
.name("Data Format")
.description("Specifies whether the data should be Text or Binary")
.required(true)
.defaultValue(DATA_FORMAT_BINARY)
.defaultValue(DATA_FORMAT_TEXT)
.allowableValues(DATA_FORMAT_BINARY, DATA_FORMAT_TEXT)
.build();
public static final PropertyDescriptor UNIQUE_FLOWFILES = new PropertyDescriptor.Builder()
@ -82,6 +86,16 @@ public class GenerateFlowFile extends AbstractProcessor {
.allowableValues("true", "false")
.defaultValue("false")
.build();
public static final PropertyDescriptor CUSTOM_TEXT = new PropertyDescriptor.Builder()
.displayName("Custom Text")
.name("generate-ff-custom-text")
.description("If Data Format is text and if Unique FlowFiles is false, then this custom text will be used as content of the generated "
+ "FlowFiles and the File Size will be ignored. Finally, if Expression Language is used, evaluation will be performed only once "
+ "per batch of generated FlowFiles")
.required(false)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
public static final Relationship SUCCESS = new Relationship.Builder()
.name("success")
@ -99,6 +113,7 @@ public class GenerateFlowFile extends AbstractProcessor {
descriptors.add(BATCH_SIZE);
descriptors.add(DATA_FORMAT);
descriptors.add(UNIQUE_FLOWFILES);
descriptors.add(CUSTOM_TEXT);
this.descriptors = Collections.unmodifiableList(descriptors);
final Set<Relationship> relationships = new HashSet<>();
@ -120,10 +135,24 @@ public class GenerateFlowFile extends AbstractProcessor {
public void onScheduled(final ProcessContext context) {
if (context.getProperty(UNIQUE_FLOWFILES).asBoolean()) {
this.data.set(null);
} else {
} else if(!context.getProperty(CUSTOM_TEXT).isSet()) {
this.data.set(generateData(context));
}
}
@Override
protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
final List<ValidationResult> results = new ArrayList<>(1);
final boolean isUnique = validationContext.getProperty(UNIQUE_FLOWFILES).asBoolean();
final boolean isText = validationContext.getProperty(DATA_FORMAT).getValue().equals(DATA_FORMAT_TEXT);
final boolean isCustom = validationContext.getProperty(CUSTOM_TEXT).isSet();
if(isCustom && (isUnique || !isText)) {
results.add(new ValidationResult.Builder().subject("Custom Text").valid(false).explanation("If Custom Text is set, then Data Format must be "
+ "text and Unique FlowFiles must be false.").build());
}
return results;
}
private byte[] generateData(final ProcessContext context) {
@ -148,6 +177,8 @@ public class GenerateFlowFile extends AbstractProcessor {
final byte[] data;
if (context.getProperty(UNIQUE_FLOWFILES).asBoolean()) {
data = generateData(context);
} else if(context.getProperty(CUSTOM_TEXT).isSet()) {
data = context.getProperty(CUSTOM_TEXT).evaluateAttributeExpressions().getValue().getBytes();
} else {
data = this.data.get();
}

View File

@ -0,0 +1,56 @@
/*
* 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.standard;
import java.io.IOException;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
/**
* Unit tests for the GenerateFlowFile processor.
*/
public class TestGenerateFlowFile {
@Test
public void testGenerateCustomText() throws IOException {
TestRunner runner = TestRunners.newTestRunner(new GenerateFlowFile());
runner.setProperty(GenerateFlowFile.FILE_SIZE, "100MB");
runner.setProperty(GenerateFlowFile.DATA_FORMAT, GenerateFlowFile.DATA_FORMAT_TEXT);
runner.setProperty(GenerateFlowFile.CUSTOM_TEXT, "This is my custom text!");
runner.run();
runner.assertTransferCount(GenerateFlowFile.SUCCESS, 1);
runner.getFlowFilesForRelationship(GenerateFlowFile.SUCCESS).get(0).assertContentEquals("This is my custom text!");
}
@Test
public void testInvalidCustomText() throws IOException {
TestRunner runner = TestRunners.newTestRunner(new GenerateFlowFile());
runner.setProperty(GenerateFlowFile.FILE_SIZE, "100MB");
runner.setProperty(GenerateFlowFile.DATA_FORMAT, GenerateFlowFile.DATA_FORMAT_BINARY);
runner.setProperty(GenerateFlowFile.CUSTOM_TEXT, "This is my custom text!");
runner.assertNotValid();
runner.setProperty(GenerateFlowFile.DATA_FORMAT, GenerateFlowFile.DATA_FORMAT_TEXT);
runner.setProperty(GenerateFlowFile.UNIQUE_FLOWFILES, "true");
runner.assertNotValid();
}
}