mirror of https://github.com/apache/nifi.git
NIFI-1788 Fixed listed allowable values
NIFI-1788 Added UT NIFI-1788 modified test name This closes #368
This commit is contained in:
parent
3d6e664097
commit
293a3042cd
|
@ -90,7 +90,7 @@ public class CreateHadoopSequenceFile extends AbstractHadoopProcessor {
|
|||
static final PropertyDescriptor COMPRESSION_TYPE = new PropertyDescriptor.Builder()
|
||||
.name("compression type")
|
||||
.description("Type of compression to use when creating Sequence File")
|
||||
.allowableValues(CompressionType.values())
|
||||
.allowableValues(SequenceFile.CompressionType.values())
|
||||
.build();
|
||||
|
||||
// Default Values.
|
||||
|
|
|
@ -94,8 +94,10 @@ public class SequenceFileWriterImpl implements SequenceFileWriter {
|
|||
|
||||
try (final FSDataOutputStream fsDataOutputStream = new FSDataOutputStream(bwos, new Statistics(""));
|
||||
final SequenceFile.Writer writer = SequenceFile.createWriter(configuration,
|
||||
fsDataOutputStream, Text.class, InputStreamWritable.class, compressionType,
|
||||
new DefaultCodec())) {
|
||||
SequenceFile.Writer.stream(fsDataOutputStream),
|
||||
SequenceFile.Writer.keyClass(Text.class),
|
||||
SequenceFile.Writer.valueClass(InputStreamWritable.class),
|
||||
SequenceFile.Writer.compression(compressionType, new DefaultCodec()))) {
|
||||
|
||||
processInputStream(in, flowFile, writer);
|
||||
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.nifi.processors.hadoop;
|
|||
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.hadoop.KerberosProperties;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
|
@ -79,6 +81,15 @@ public class TestCreateHadoopSequenceFile {
|
|||
controller.clearTransferState();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void validateAllowableValuesForCompressionType() {
|
||||
PropertyDescriptor pd = CreateHadoopSequenceFile.COMPRESSION_TYPE;
|
||||
List<AllowableValue> allowableValues = pd.getAllowableValues();
|
||||
assertEquals("NONE", allowableValues.get(0).getValue());
|
||||
assertEquals("RECORD", allowableValues.get(1).getValue());
|
||||
assertEquals("BLOCK", allowableValues.get(2).getValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleCase() throws IOException {
|
||||
for (File inFile : inFiles) {
|
||||
|
|
Loading…
Reference in New Issue