NIFI-615 - Create a processor to extract WAV file characteristics.

* Create new ExtractMediaMetadata processor using Apache Tika Detector and Parser.
* Refactored nifi-image-bundle, nifi-image-nar, and nifi-image-processors to nifi-media-bundle, nifi-media-nar, and nifi-media-processors to reflect broader media related purpose.
* Preserved existing ExtractImageMetadata and ResizeImage processors as well as existing ImageViewerController components to prevent impact on existing uses.
* Resolved collision between ExtractImage and ExtractMedia processors due to common dependency on Noakes' Metadata Extractor project.
  - Updated bundle's Tika dependency from 1.7 to 1.8 and Drew Noakes' Metadata Extractor from 2.7.2 to 2.8.0.
  - Adjusted ExtractImageMetadata tests for enhanced attribute names in new Noakes' Metadata Extractor version.
* Fix assembly POM to remove duplicate reference to site-to-site nar and change nifi-image-nar reference to nifi-media-nar.
* Note the potential attribute changes on upgrade due to underlying libraries.

This closes #556.
This commit is contained in:
Joe Skora 2016-02-26 15:33:40 -05:00 committed by Joe Skora
parent 0e085bdddd
commit 7e7d79fc6a
33 changed files with 2821 additions and 43 deletions

View File

@ -286,8 +286,23 @@ The following binary components are provided under the Apache Software License v
(ASLv2) Apache Tika
The following NOTICE information applies:
Apache Tika Core
Copyright 2007-2015 The Apache Software Foundation
Apache Tika
Copyright 2015 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
Copyright 1993-2010 University Corporation for Atmospheric Research/Unidata
This software contains code derived from UCAR/Unidata's NetCDF library.
Tika-server component uses CDDL-licensed dependencies: jersey (http://jersey.java.net/) and
Grizzly (http://grizzly.java.net/)
Tika-parsers component uses CDDL/LGPL dual-licensed dependency: jhighlight (https://github.com/codelibs/jhighlight)
OpenCSV: Copyright 2005 Bytecode Pty Ltd. Licensed under the Apache License, Version 2.0
IPTC Photo Metadata descriptions Copyright 2010 International Press Telecommunications Council.
(ASLv2) Apache Jakarta Commons Digester
The following NOTICE information applies:

View File

@ -244,7 +244,7 @@ language governing permissions and limitations under the License. -->
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-image-nar</artifactId>
<artifactId>nifi-media-nar</artifactId>
<type>nar</type>
</dependency>
<dependency>

View File

@ -1,16 +0,0 @@
nifi-image-nar
Copyright 2015-2016 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) Metadata-Extractor
The following NOTICE information applies:
Metadata-Extractor
Copyright 2002-2015 Drew Noakes

View File

@ -17,7 +17,7 @@
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-image-bundle</artifactId>
<artifactId>nifi-media-bundle</artifactId>
<version>1.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-image-viewer</artifactId>

View File

@ -18,12 +18,12 @@
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-image-bundle</artifactId>
<artifactId>nifi-media-bundle</artifactId>
<version>1.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-image-nar</artifactId>
<version>1.0.0-SNAPSHOT</version>
<artifactId>nifi-media-nar</artifactId>
<version>0.7.0-SNAPSHOT</version>
<packaging>nar</packaging>
<properties>
<maven.javadoc.skip>true</maven.javadoc.skip>
@ -33,7 +33,7 @@
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-image-processors</artifactId>
<artifactId>nifi-media-processors</artifactId>
<version>1.0.0-SNAPSHOT</version>
</dependency>
<dependency>

View File

@ -0,0 +1,35 @@
nifi-media-nar
Copyright 2015-2016 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) Metadata-Extractor
The following NOTICE information applies:
Metadata-Extractor
Copyright 2002-2015 Drew Noakes
(ASLv2) Apache Tika
Apache Tika
Copyright 2015 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
Copyright 1993-2010 University Corporation for Atmospheric Research/Unidata
This software contains code derived from UCAR/Unidata's NetCDF library.
Tika-server component uses CDDL-licensed dependencies: jersey (http://jersey.java.net/) and
Grizzly (http://grizzly.java.net/)
Tika-parsers component uses CDDL/LGPL dual-licensed dependency: jhighlight (https://github.com/codelibs/jhighlight)
OpenCSV: Copyright 2005 Bytecode Pty Ltd. Licensed under the Apache License, Version 2.0
IPTC Photo Metadata descriptions Copyright 2010 International Press Telecommunications Council.

View File

@ -18,11 +18,11 @@
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-image-bundle</artifactId>
<artifactId>nifi-media-bundle</artifactId>
<version>1.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-image-processors</artifactId>
<artifactId>nifi-media-processors</artifactId>
<packaging>jar</packaging>
<dependencies>
@ -42,7 +42,17 @@
<dependency>
<groupId>com.drewnoakes</groupId>
<artifactId>metadata-extractor</artifactId>
<version>2.7.2</version>
<version>2.8.0</version>
</dependency>
<dependency>
<groupId>org.apache.tika</groupId>
<artifactId>tika-core</artifactId>
<version>1.8</version>
</dependency>
<dependency>
<groupId>org.apache.tika</groupId>
<artifactId>tika-parsers</artifactId>
<version>1.8</version>
</dependency>
</dependencies>
@ -54,6 +64,8 @@
<configuration>
<excludes combine.children="append">
<exclude>src/test/resources/notImage.txt</exclude>
<exclude>src/test/resources/textFile.txt</exclude>
<exclude>src/test/resources/textFileBig.txt</exclude>
</excludes>
</configuration>
</plugin>

View File

@ -0,0 +1,251 @@
/*
* 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.media;
import java.io.IOException;
import java.io.InputStream;
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.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
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.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.tika.exception.TikaException;
import org.apache.tika.io.TikaInputStream;
import org.apache.tika.metadata.Metadata;
import org.apache.tika.parser.AutoDetectParser;
import org.xml.sax.SAXException;
import org.xml.sax.helpers.DefaultHandler;
@InputRequirement(Requirement.INPUT_REQUIRED)
@Tags({"media", "file", "format", "metadata", "audio", "video", "image", "document", "pdf"})
@CapabilityDescription("Extract the content metadata from flowfiles containing audio, video, image, and other file "
+ "types. This processor relies on the Apache Tika project for file format detection and parsing. It "
+ "extracts a long list of metadata types for media files including audio, video, and print media "
+ "formats."
+ "NOTE: the attribute names and content extracted may vary across upgrades because parsing is performed by "
+ "the external Tika tools which in turn depend on other projects for metadata extraction. For the more "
+ "details and the list of supported file types, visit the library's website at http://tika.apache.org/.")
@WritesAttributes({@WritesAttribute(attribute = "<Metadata Key Prefix><attribute>", description = "The extracted content metadata "
+ "will be inserted with the attribute name \"<Metadata Key Prefix><attribute>\", or \"<attribute>\" if "
+ "\"Metadata Key Prefix\" is not provided.")})
@SupportsBatching
public class ExtractMediaMetadata extends AbstractProcessor {
static final PropertyDescriptor MAX_NUMBER_OF_ATTRIBUTES = new PropertyDescriptor.Builder()
.name("Max Number of Attributes")
.description("Specify the max number of attributes to add to the flowfile. There is no guarantee in what order"
+ " the tags will be processed. By default it will process all of them.")
.required(false)
.defaultValue("100")
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
.build();
private static final PropertyDescriptor MAX_ATTRIBUTE_LENGTH = new PropertyDescriptor.Builder()
.name("Max Attribute Length")
.description("Specifies the maximum length of a single attribute value. When a metadata item has multiple"
+ " values, they will be merged until this length is reached and then \", ...\" will be added as"
+ " an indicator that additional values where dropped. If a single value is longer than this, it"
+ " will be truncated and \"(truncated)\" appended to indicate that truncation occurred.")
.required(true)
.defaultValue("100")
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
.build();
static final PropertyDescriptor METADATA_KEY_FILTER = new PropertyDescriptor.Builder()
.name("Metadata Key Filter")
.description("A regular expression identifying which metadata keys received from the parser should be"
+ " added to the flowfile attributes. If left blank, all metadata keys parsed will be added to the"
+ " flowfile attributes.")
.required(false)
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
.build();
static final PropertyDescriptor METADATA_KEY_PREFIX = new PropertyDescriptor.Builder()
.name("Metadata Key Prefix")
.description("Text to be prefixed to metadata keys as the are added to the flowfile attributes. It is"
+ " recommended to end with with a separator character like '.' or '-', this is not automatically "
+ " added by the processor.")
.required(false)
.addValidator(StandardValidators.ATTRIBUTE_KEY_VALIDATOR)
.expressionLanguageSupported(true)
.build();
static final Relationship SUCCESS = new Relationship.Builder()
.name("success")
.description("Any FlowFile that successfully has media metadata extracted will be routed to success")
.build();
static final Relationship FAILURE = new Relationship.Builder()
.name("failure")
.description("Any FlowFile that fails to have media metadata extracted will be routed to failure")
.build();
private Set<Relationship> relationships;
private List<PropertyDescriptor> properties;
private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
private volatile AutoDetectParser autoDetectParser;
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(MAX_NUMBER_OF_ATTRIBUTES);
properties.add(MAX_ATTRIBUTE_LENGTH);
properties.add(METADATA_KEY_FILTER);
properties.add(METADATA_KEY_PREFIX);
this.properties = Collections.unmodifiableList(properties);
final Set<Relationship> relationships = new HashSet<>();
relationships.add(SUCCESS);
relationships.add(FAILURE);
this.relationships = Collections.unmodifiableSet(relationships);
}
@Override
public Set<Relationship> getRelationships() {
return this.relationships;
}
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return this.properties;
}
@SuppressWarnings("unused")
@OnScheduled
public void onScheduled(ProcessContext context) {
String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
} else {
metadataKeyFilterRef.set(null);
}
autoDetectParser = new AutoDetectParser();
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
FlowFile flowFile = session.get();
if (flowFile == null) {
return;
}
final ComponentLog logger = this.getLogger();
final AtomicReference<Map<String, String>> value = new AtomicReference<>(null);
final Integer maxAttribCount = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
final Integer maxAttribLength = context.getProperty(MAX_ATTRIBUTE_LENGTH).asInteger();
final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions(flowFile).getValue();
try {
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(InputStream in) throws IOException {
try {
Map<String, String> results = tika_parse(in, prefix, maxAttribCount, maxAttribLength);
value.set(results);
} catch (SAXException | TikaException e) {
throw new IOException(e);
}
}
});
// Write the results to attributes
Map<String, String> results = value.get();
if (results != null && !results.isEmpty()) {
flowFile = session.putAllAttributes(flowFile, results);
}
session.transfer(flowFile, SUCCESS);
session.getProvenanceReporter().modifyAttributes(flowFile, "media attributes extracted");
} catch (ProcessException e) {
logger.error("Failed to extract media metadata from {} due to {}", new Object[]{flowFile, e});
flowFile = session.penalize(flowFile);
session.transfer(flowFile, FAILURE);
}
}
private Map<String, String> tika_parse(InputStream sourceStream, String prefix, Integer maxAttribs,
Integer maxAttribLen) throws IOException, TikaException, SAXException {
final Metadata metadata = new Metadata();
final TikaInputStream tikaInputStream = TikaInputStream.get(sourceStream);
autoDetectParser.parse(tikaInputStream, new DefaultHandler(), metadata);
final Map<String, String> results = new HashMap<>();
final Pattern metadataKeyFilter = metadataKeyFilterRef.get();
final StringBuilder dataBuilder = new StringBuilder();
for (final String key : metadata.names()) {
if (metadataKeyFilter != null && !metadataKeyFilter.matcher(key).matches()) {
continue;
}
dataBuilder.setLength(0);
if (metadata.isMultiValued(key)) {
for (String val : metadata.getValues(key)) {
if (dataBuilder.length() > 1) {
dataBuilder.append(", ");
}
if (dataBuilder.length() + val.length() < maxAttribLen) {
dataBuilder.append(val);
} else {
dataBuilder.append("...");
break;
}
}
} else {
dataBuilder.append(metadata.get(key));
}
if (prefix == null) {
results.put(key, dataBuilder.toString().trim());
} else {
results.put(prefix + key, dataBuilder.toString().trim());
}
// cutoff at max if provided
if (maxAttribs != null && results.size() >= maxAttribs) {
break;
}
}
return results;
}
}

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.nifi.processors.image.ExtractImageMetadata
org.apache.nifi.processors.image.ResizeImage
org.apache.nifi.processors.media.ExtractMediaMetadata

View File

@ -37,7 +37,7 @@ public class ExtractImageMetadataTest {
private static String BMP_HEADER = "BMP Header.";
private static String JPEG_HEADER = "JPEG.";
private static String GIF_HEADER = "GIF Header.";
private static String PNG_HEADER = "PNG.";
private static String PNG_HEADER = "PNG-";
private TestRunner testRunner;
@ -90,15 +90,15 @@ public class ExtractImageMetadataTest {
MockFlowFile flowFile = verifyTestRunnerFlow("src/test/resources/mspaint-8x10.png", ExtractImageMetadata.SUCCESS, null);
Map<String, String> attributes = flowFile.getAttributes();
assertEquals("8", attributes.get(PNG_HEADER + "Image Width"));
assertEquals("12", attributes.get(PNG_HEADER + "Image Height"));
assertEquals("0.45455", attributes.get(PNG_HEADER + "Image Gamma"));
assertEquals("Deflate", attributes.get(PNG_HEADER + "Compression Type"));
assertEquals("No Interlace", attributes.get(PNG_HEADER + "Interlace Method"));
assertEquals("Perceptual", attributes.get(PNG_HEADER + "sRGB Rendering Intent"));
assertEquals("Adaptive", attributes.get(PNG_HEADER + "Filter Method"));
assertEquals("8", attributes.get(PNG_HEADER + "Bits Per Sample"));
assertEquals("True Color", attributes.get(PNG_HEADER + "Color Type"));
assertEquals("8", attributes.get(PNG_HEADER + "IHDR.Image Width"));
assertEquals("12", attributes.get(PNG_HEADER + "IHDR.Image Height"));
assertEquals("0.45455", attributes.get(PNG_HEADER + "gAMA.Image Gamma"));
assertEquals("Deflate", attributes.get(PNG_HEADER + "IHDR.Compression Type"));
assertEquals("No Interlace", attributes.get(PNG_HEADER + "IHDR.Interlace Method"));
assertEquals("Perceptual", attributes.get(PNG_HEADER + "sRGB.sRGB Rendering Intent"));
assertEquals("Adaptive", attributes.get(PNG_HEADER + "IHDR.Filter Method"));
assertEquals("8", attributes.get(PNG_HEADER + "IHDR.Bits Per Sample"));
assertEquals("True Color", attributes.get(PNG_HEADER + "IHDR.Color Type"));
}
@Test
public void testExtractBMP() throws IOException {

View File

@ -0,0 +1,456 @@
/*
* 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.media;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
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.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class TestExtractMediaMetadata {
@Test
public void testProperties() {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
ProcessContext context = runner.getProcessContext();
Map<PropertyDescriptor, String> propertyValues = context.getProperties();
assertEquals(4, propertyValues.size());
}
@Test
public void testRelationships() {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
ProcessContext context = runner.getProcessContext();
Set<Relationship> relationships = context.getAvailableRelationships();
assertEquals(2, relationships.size());
assertTrue(relationships.contains(ExtractMediaMetadata.SUCCESS));
assertTrue(relationships.contains(ExtractMediaMetadata.FAILURE));
}
@Test
public void testTextBytes() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test1.txt");
runner.enqueue("test1".getBytes(), attrs);
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "test1.txt");
flowFile0.assertAttributeExists("txt.Content-Type");
assertTrue(flowFile0.getAttribute("txt.Content-Type").startsWith("text/plain"));
flowFile0.assertAttributeExists("txt.X-Parsed-By");
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.txt.TXTParser"));
flowFile0.assertAttributeExists("txt.Content-Encoding");
flowFile0.assertAttributeEquals("txt.Content-Encoding", "ISO-8859-1");
flowFile0.assertContentEquals("test1".getBytes("UTF-8"));
}
@Test
public void testProvenance() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "test1.txt");
runner.enqueue("test1".getBytes(), attrs);
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
assertEquals(1, events.size());
final ProvenanceEventRecord event = events.get(0);
assertEquals(ExtractMediaMetadata.class.getSimpleName(), event.getComponentType());
assertEquals("media attributes extracted", event.getDetails());
assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, event.getEventType());
}
@Test
public void testNoFlowFile() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 0);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
}
@Test
public void testTextFile() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/textFile.txt").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "textFile.txt");
flowFile0.assertAttributeExists("txt.Content-Type");
assertTrue(flowFile0.getAttribute("txt.Content-Type").startsWith("text/plain"));
flowFile0.assertAttributeExists("txt.X-Parsed-By");
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.txt.TXTParser"));
flowFile0.assertAttributeExists("txt.Content-Encoding");
flowFile0.assertAttributeEquals("txt.Content-Encoding", "ISO-8859-1");
flowFile0.assertContentEquals("This file is not an image and is used for testing the image metadata extractor.".getBytes("UTF-8"));
}
@Test
public void testBigTextFile() throws IOException {
File textFile = new File("target/test-classes/textFileBig.txt");
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
runner.enqueue(textFile.toPath());
runner.run(2);
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "textFileBig.txt");
flowFile0.assertAttributeExists("txt.Content-Type");
assertTrue(flowFile0.getAttribute("txt.Content-Type").startsWith("text/plain"));
flowFile0.assertAttributeExists("txt.X-Parsed-By");
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.txt.TXTParser"));
flowFile0.assertAttributeExists("txt.Content-Encoding");
flowFile0.assertAttributeEquals("txt.Content-Encoding", "ISO-8859-1");
assertEquals(flowFile0.getSize(), textFile.length());
}
@Test
public void testJunkBytes() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "junk.");
runner.assertValid();
final Map<String, String> attrs = new HashMap<>();
attrs.put("filename", "junk");
Random random = new Random();
byte[] bytes = new byte[2048];
random.nextBytes(bytes);
runner.enqueue(bytes, attrs);
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "junk");
flowFile0.assertAttributeExists("junk.Content-Type");
assertTrue(flowFile0.getAttribute("junk.Content-Type").startsWith("application/octet-stream"));
flowFile0.assertAttributeExists("junk.X-Parsed-By");
assertTrue(flowFile0.getAttribute("junk.X-Parsed-By").contains("org.apache.tika.parser.EmptyParser"));
flowFile0.assertContentEquals(bytes);
}
@Test
public void testMetadataKeyFilter() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "(X-Parsed.*)");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/textFile.txt").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "textFile.txt");
flowFile0.assertAttributeExists("txt.X-Parsed-By");
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("txt.X-Parsed-By").contains("org.apache.tika.parser.txt.TXTParser"));
flowFile0.assertAttributeNotExists("txt.Content-Encoding");
}
@Test
public void testMetadataKeyPrefix() throws IOException {
TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.assertValid();
runner.enqueue(new File("target/test-classes/textFile.txt").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeExists("X-Parsed-By");
runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/textFile.txt").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeExists("txt.X-Parsed-By");
}
@Test
public void testMaxAttributes() throws IOException {
TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/textFile.txt").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
List<MockFlowFile> successFiles0 = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles0.get(0);
int fileAttrCount0 = 0;
for (Map.Entry attr : flowFile0.getAttributes().entrySet()) {
if (attr.getKey().toString().startsWith("txt.")) {
fileAttrCount0++;
}
}
assertTrue(fileAttrCount0 > 1);
runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.MAX_NUMBER_OF_ATTRIBUTES, Integer.toString(fileAttrCount0 - 1));
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/textFile.txt").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile1 = successFiles.get(0);
int fileAttrCount1 = 0;
for (Map.Entry attr : flowFile1.getAttributes().entrySet()) {
if (attr.getKey().toString().startsWith("txt.")) {
fileAttrCount1++;
}
}
assertEquals(fileAttrCount0, fileAttrCount1 + 1);
}
@Test
public void testBmp() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "bmp.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/16color-10x10.bmp").toPath());
runner.run(2);
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "16color-10x10.bmp");
flowFile0.assertAttributeExists("bmp.Content-Type");
flowFile0.assertAttributeEquals("bmp.Content-Type", "image/x-ms-bmp");
flowFile0.assertAttributeExists("bmp.X-Parsed-By");
assertTrue(flowFile0.getAttribute("bmp.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("bmp.X-Parsed-By").contains("org.apache.tika.parser.image.ImageParser"));
flowFile0.assertAttributeExists("bmp.height");
flowFile0.assertAttributeEquals("bmp.height", "10");
flowFile0.assertAttributeExists("bmp.width");
flowFile0.assertAttributeEquals("bmp.width", "10");
}
@Test
public void testJpg() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "jpg.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/simple.jpg").toPath());
runner.run(2);
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "simple.jpg");
flowFile0.assertAttributeExists("jpg.Content-Type");
flowFile0.assertAttributeEquals("jpg.Content-Type", "image/jpeg");
flowFile0.assertAttributeExists("jpg.X-Parsed-By");
assertTrue(flowFile0.getAttribute("jpg.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("jpg.X-Parsed-By").contains("org.apache.tika.parser.jpeg.JpegParser"));
}
@Test
public void testWav() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "wav.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/testWAV.wav").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "testWAV.wav");
flowFile0.assertAttributeExists("wav.Content-Type");
assertTrue(flowFile0.getAttribute("wav.Content-Type").startsWith("audio/x-wav"));
flowFile0.assertAttributeExists("wav.X-Parsed-By");
assertTrue(flowFile0.getAttribute("wav.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("wav.X-Parsed-By").contains("org.apache.tika.parser.audio.AudioParser"));
flowFile0.assertAttributeExists("wav.encoding");
flowFile0.assertAttributeEquals("wav.encoding", "PCM_SIGNED");
}
@Test
public void testOgg() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "ogg.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/testVORBIS.ogg").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "testVORBIS.ogg");
flowFile0.assertAttributeExists("ogg.Content-Type");
assertTrue(flowFile0.getAttribute("ogg.Content-Type").startsWith("audio/vorbis"));
flowFile0.assertAttributeExists("ogg.X-Parsed-By");
assertTrue(flowFile0.getAttribute("ogg.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("ogg.X-Parsed-By").contains("org.gagravarr.tika.VorbisParser"));
}
@Test
public void testOggCorruptFails() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "ogg.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/testVORBIS-corrupt.ogg").toPath());
runner.run(2);
runner.assertTransferCount(ExtractMediaMetadata.SUCCESS, 0);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 1);
final List<MockFlowFile> failureFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.FAILURE);
MockFlowFile flowFile0 = failureFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "testVORBIS-corrupt.ogg");
flowFile0.assertAttributeNotExists("ogg.Content-Type");
flowFile0.assertAttributeNotExists("ogg.X-Parsed-By");
}
@Test
public void testMp3() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "mp3.");
runner.assertValid();
runner.enqueue(new File("target/test-classes/testMP3id3v1.mp3").toPath());
runner.run();
runner.assertAllFlowFilesTransferred(ExtractMediaMetadata.SUCCESS, 1);
runner.assertTransferCount(ExtractMediaMetadata.FAILURE, 0);
final List<MockFlowFile> successFiles = runner.getFlowFilesForRelationship(ExtractMediaMetadata.SUCCESS);
MockFlowFile flowFile0 = successFiles.get(0);
flowFile0.assertAttributeExists("filename");
flowFile0.assertAttributeEquals("filename", "testMP3id3v1.mp3");
flowFile0.assertAttributeExists("mp3.Content-Type");
assertTrue(flowFile0.getAttribute("mp3.Content-Type").startsWith("audio/mpeg"));
flowFile0.assertAttributeExists("mp3.X-Parsed-By");
assertTrue(flowFile0.getAttribute("mp3.X-Parsed-By").contains("org.apache.tika.parser.DefaultParser"));
assertTrue(flowFile0.getAttribute("mp3.X-Parsed-By").contains("org.apache.tika.parser.mp3.Mp3Parser"));
flowFile0.assertAttributeExists("mp3.title");
flowFile0.assertAttributeEquals("mp3.title", "Test Title");
}
}

View File

@ -0,0 +1 @@
This file is not an image and is used for testing the image metadata extractor.

View File

@ -22,13 +22,13 @@
<version>1.0.0-SNAPSHOT</version>
</parent>
<artifactId>nifi-image-bundle</artifactId>
<artifactId>nifi-media-bundle</artifactId>
<version>1.0.0-SNAPSHOT</version>
<packaging>pom</packaging>
<modules>
<module>nifi-image-processors</module>
<module>nifi-image-nar</module>
<module>nifi-media-processors</module>
<module>nifi-media-nar</module>
<module>nifi-image-viewer</module>
</modules>

View File

@ -44,7 +44,7 @@
<module>nifi-flume-bundle</module>
<module>nifi-hbase-bundle</module>
<module>nifi-ambari-bundle</module>
<module>nifi-image-bundle</module>
<module>nifi-media-bundle</module>
<module>nifi-avro-bundle</module>
<module>nifi-couchbase-bundle</module>
<module>nifi-azure-bundle</module>

View File

@ -1025,7 +1025,7 @@ language governing permissions and limitations under the License. -->
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-image-nar</artifactId>
<artifactId>nifi-media-nar</artifactId>
<version>1.0.0-SNAPSHOT</version>
<type>nar</type>
</dependency>