You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by jskora <gi...@git.apache.org> on 2016/02/29 17:41:04 UTC

[GitHub] nifi pull request: NIFI-615 - Create a processor to extract WAV fi...

GitHub user jskora opened a pull request:

    https://github.com/apache/nifi/pull/252

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

    * Created 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.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/jskora/nifi NIFI-615

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/252.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #252
    
----
commit 319d4e4643a9958e196c7226155c25c74371137c
Author: Joe Skora <js...@gmail.com>
Date:   2016-02-26T20:33:40Z

    NIFI-615 - Create a processor to extract WAV file characteristics.
    * Created 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.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67214948
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    --- End diff --
    
    I know the ATTRIBUTES_MODIFIED provenance event is emitted automatically but would be nice to have some additional details added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67227890
  
    --- Diff: nifi-assembly/NOTICE ---
    @@ -289,6 +289,11 @@ The following binary components are provided under the Apache Software License v
           Apache Tika Core
           Copyright 2007-2015 The Apache Software Foundation
     
    +  (ASLv2) Apache Tika
    +    The following NOTICE information applies:
    --- End diff --
    
    The Tika notice should be reflected here: https://github.com/apache/tika/blob/master/NOTICE.txt


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-615 - Create a processor to extract WAV fi...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the pull request:

    https://github.com/apache/nifi/pull/252#issuecomment-211868734
  
    @joewitt On [NIFI-1717|https://issues.apache.org/jira/browse/NIFI-1717] and [NIFI-1718|https://issues.apache.org/jira/browse/NIFI-1718] Dmitry Goldenberg and I discussed using Tika to extract content (OCR) documents and images.  @markap14 also suggested removing the filters.
    
    I don't know where the OCR changes stand, those tickets have been quiet for a couple of weeks.  I think that's a tougher capability to test, and as pointed out on [NIFI-1717|https://issues.apache.org/jira/browse/NIFI-1717] and [NIFI-1718|https://issues.apache.org/jira/browse/NIFI-1718] it is an expensive process that may need special consideration.
    
    As for the filters, I like having them in the processor, especially since this one includes filename and mimetype filters.  If consensus is to remove them, I can update the PR for that, but I think they are affective for this purpose as it currently is.
    
    I don't think we should hold this for the OCR, but if you want the filters removed let me know.  It'd be nice to get the metadata functionality in.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67393537
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    +        } catch (ProcessException e) {
    +            logger.error("Failed to extract media metadata from {} due to {}", new Object[]{flowfile, e});
    +            session.transfer(flowfile, FAILURE);
    +        }
    +    }
    +
    +    private Map<String, String> tika_parse(FlowFile ff, InputStream sourceStream, String prefix, Integer max) throws IOException, TikaException, SAXException {
    +        final Metadata metadata = new Metadata();
    +        final TikaInputStream tikaInputStream = TikaInputStream.get(sourceStream);
    +        new AutoDetectParser().parse(tikaInputStream, new BodyContentHandler(), metadata);
    +        final String content_type = metadata.get(Metadata.CONTENT_TYPE);
    +
    +        // if parsed MIME type doesn't match filter fail fast without processing attributes
    +        final Pattern mimeTypeFilter = mimeTypeFilterRef.get();
    +        if (mimeTypeFilter != null && (content_type == null || !mimeTypeFilter.matcher(metadata.get(Metadata.CONTENT_TYPE)).matches())) {
    +            return null;
    +        }
    +
    +        final Map<String, String> results = new HashMap<>();
    +        final Pattern metadataKeyFilter = metadataKeyFilterRef.get();
    +        final StringBuilder dataBuilder = new StringBuilder();
    +        final String safePrefix = (prefix == null) ? "" : prefix;
    +        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(", ");
    +                    }
    +                    dataBuilder.append(val);
    --- End diff --
    
    Property added for maximum length of an attribute value.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67187797
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    --- End diff --
    
    Should this be required? How many attributes could Tika potentially create?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67384364
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    +        } catch (ProcessException e) {
    +            logger.error("Failed to extract media metadata from {} due to {}", new Object[]{flowfile, e});
    +            session.transfer(flowfile, FAILURE);
    +        }
    +    }
    +
    +    private Map<String, String> tika_parse(FlowFile ff, InputStream sourceStream, String prefix, Integer max) throws IOException, TikaException, SAXException {
    +        final Metadata metadata = new Metadata();
    +        final TikaInputStream tikaInputStream = TikaInputStream.get(sourceStream);
    +        new AutoDetectParser().parse(tikaInputStream, new BodyContentHandler(), metadata);
    +        final String content_type = metadata.get(Metadata.CONTENT_TYPE);
    +
    +        // if parsed MIME type doesn't match filter fail fast without processing attributes
    +        final Pattern mimeTypeFilter = mimeTypeFilterRef.get();
    +        if (mimeTypeFilter != null && (content_type == null || !mimeTypeFilter.matcher(metadata.get(Metadata.CONTENT_TYPE)).matches())) {
    +            return null;
    +        }
    +
    +        final Map<String, String> results = new HashMap<>();
    +        final Pattern metadataKeyFilter = metadataKeyFilterRef.get();
    +        final StringBuilder dataBuilder = new StringBuilder();
    +        final String safePrefix = (prefix == null) ? "" : prefix;
    +        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(", ");
    +                    }
    +                    dataBuilder.append(val);
    +                }
    +            } else {
    +                dataBuilder.append(metadata.get(key));
    +            }
    +            results.put(safePrefix + key, dataBuilder.toString().trim());
    --- End diff --
    
    Removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67218079
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    --- End diff --
    
    Since there's no way to validate a regular expression based on EL, it seemed more valuable to make sure the regex is valid.  We can consider adding this later, but it would require a good bit of testing to change now and still would never be able to guarantee that the EL would produce a valid regex at runtime.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-615 - Create a processor to extract WAV fi...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the pull request:

    https://github.com/apache/nifi/pull/252#issuecomment-198408663
  
    @joewitt, the attribute names are created by Tika and seem to be based on the source file types.  The core Tika module was already in the project and the Tika parsers module appears to add ~600KB to the NAR.  Also, this is named "ExtractMediaMetadata" because using Tika allows it to support over 1,000 file types, instead of just WAV files, and allows it to be extended with custom parsers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    Ok, I think I've resolved the problems.  The Tika conflict was fixed by udpating Tika and Metadata Extractor dependency versions and adjusting ExtractImageMetadata for new attribute names in the newer parser.
    
    I've stored my current branch on my Github branch [NIFI-615-v2](https://github.com/jskora/nifi/tree/NIFI-615-v2).  I still need to rebase that, I probably won't be able to do that until Monday.  Please take a look if you get a chance and I'll try to get the pull updated on Monday.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67213076
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    --- End diff --
    
    Where in the code does "tika" get added to the attribute name? This should include where the METADATA_KEY_PREFIX property fits


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    Ironically, this boils down to a version dependency conflict between ExtractMediaMetadata which uses Tika 1.7 which uses Drew Noakes' Metadata-Extract 2.6.2 and the existing ExtractImageMetadata which used Drew Noakes' Metadata-Extract 2.7.2.
    
    I'm looking into ways to resolve this.  One option may be to return ExtractImageMetadata to it's own nifi-image-bundle nar and create a new nifi-media-bundle-nar instead the rename of n-image-b-n to n-media-b-n that I originally did.  Another is to look into newer Tika versions, but I don't know if that will create other problems.  I'll keep you posted.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67222367
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/test/java/org/apache/nifi/processors/media/TestExtractMediaMetadata.java ---
    @@ -0,0 +1,411 @@
    +/*
    + * 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.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(5, 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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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 testNoFlowFile() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.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 testJunkBytes() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*");
    +        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];
    +        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 testFilenameFilter() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[dDoOcC]");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.txt");
    +        flowFile0.assertAttributeNotExists("txt.Content-Type");
    +        flowFile0.assertAttributeNotExists("txt.X-Parsed-By");
    +    }
    +
    +    @Test
    +    public void testMimeTypeFilter() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "doc.*");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.txt");
    +        flowFile0.assertAttributeNotExists("txt.Content-Type");
    +        flowFile0.assertAttributeNotExists("txt.X-Parsed-By");
    +    }
    +
    +    @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/notImage.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", "notImage.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"));
    +    }
    +
    +    @Test
    +    public void testMetadataKeyPrefix() throws IOException {
    +        TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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/notImage.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/notImage.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/notImage.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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.txt").toPath());
    --- End diff --
    
    This test is titled "testBmp" but runs and verifies the same "notImage.txt" as prior tests use. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    I set this up to analyze all the files I have in my downloads folder to see what happened and what errors I came across. I found a couple interesting ones.
    
    The first should be a configurable property. The second looks like problems attempting to extract metadata from JPGs. You can recreate the problem by setting up a unit test to analyze simple.jpg (in test resources):
    
    org.apache.tika.sax.WriteOutContentHandler$WriteLimitReachedException: Your document contained more than 100000 characters, and so your requested limit has been reached. To receive the full text of the document, increase your limit. (Text up to the limit is however available).
    
    2016-06-15 15:33:22,677 ERROR [Timer-Driven Process Thread-7] o.a.n.p.media.ExtractMediaMetadata ExtractMediaMetadata[id=c4e52258-dac5-43b1-b951-2d7f9a7ebf6c] ExtractMediaMetadata[id=c4e52258-dac5-43b1-b951-2d7f9a7ebf6c] failed to process due to java.lang.NoClassDefFoundError: com/drew/lang/BufferReader; rolling back session: java.lang.NoClassDefFoundError: com/drew/lang/BufferReader
    2016-06-15 15:33:22,679 ERROR [Timer-Driven Process Thread-7] o.a.n.p.media.ExtractMediaMetadata
    java.lang.NoClassDefFoundError: com/drew/lang/BufferReader
            at org.apache.tika.parser.jpeg.JpegParser.parse(JpegParser.java:56) ~[na:na]
            at org.apache.tika.parser.CompositeParser.parse(CompositeParser.java:256) ~[na:na]
            at org.apache.tika.parser.CompositeParser.parse(CompositeParser.java:256) ~[na:na]
            at org.apache.tika.parser.AutoDetectParser.parse(AutoDetectParser.java:120) ~[na:na]
            at org.apache.tika.parser.AutoDetectParser.parse(AutoDetectParser.java:136) ~[na:na]
            at org.apache.nifi.processors.media.ExtractMediaMetadata.tika_parse(ExtractMediaMetadata.java:239) ~[na:na]
            at org.apache.nifi.processors.media.ExtractMediaMetadata.access$000(ExtractMediaMetadata.java:71) ~[na:na]
            at org.apache.nifi.processors.media.ExtractMediaMetadata$1.process(ExtractMediaMetadata.java:215) ~[na:na]
            at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:1806) ~[nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
            at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:1777) ~[nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
            at org.apache.nifi.processors.media.ExtractMediaMetadata.onTrigger(ExtractMediaMetadata.java:211) ~[na:na]
            at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) ~[nifi-api-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
            at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1139) [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
            at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:139) [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
            at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:49) [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
            at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:124) [nifi-framework-core-0.6.0-SNAPSHOT.jar:0.6.0-SNAPSHOT]
            at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_74]
            at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_74]
            at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_74]
            at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_74]
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_74]
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_74]
            at java.lang.Thread.run(Thread.java:745) [na:1.8.0_74]
    Caused by: java.lang.ClassNotFoundException: com.drew.lang.BufferReader
            at java.net.URLClassLoader.findClass(URLClassLoader.java:381) ~[na:1.8.0_74]
            at java.lang.ClassLoader.loadClass(ClassLoader.java:424) ~[na:1.8.0_74]
            at java.lang.ClassLoader.loadClass(ClassLoader.java:357) ~[na:1.8.0_74]
            ... 23 common frames omitted


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67218529
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    --- End diff --
    
    That is a very good point, I agree to not make support EL


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-615 - Create a processor to extract WAV fi...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the pull request:

    https://github.com/apache/nifi/pull/252#issuecomment-211712568
  
    @jskora I recall there was some discussion where you were considering supporting this as part of something a bit broader.  Is that right or is this one you think good to go as is? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-615 - Create a processor to extract WAV fi...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the pull request:

    https://github.com/apache/nifi/pull/252#issuecomment-198427593
  
    I don't love all the attribute names, I might have come up with a cleaner taxonomy myself.  But, letting Tika handle it leverages the work done on another active Apache project by people familiar with the formats.  Incorporating Tika was a little harder than writing custom code to only handle WAV files would have been, but it provided significantly more functionality.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67387964
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/test/java/org/apache/nifi/processors/media/TestExtractMediaMetadata.java ---
    @@ -0,0 +1,411 @@
    +/*
    + * 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.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(5, propertyValues.size());
    +    }
    +
    +    @Test
    +    public void testRelationShips() {
    --- End diff --
    
    Got it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67215325
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    --- End diff --
    
    The filters allow selective processing of files and mime types without requiring the flow to be split into multiple distinct paths.  It's easier to adjust the filters on the processor to include another mimetype than it is to create and maintain numerous paths on the flow to selectively handle files that are ultimately headed to the same destination.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67383325
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    --- End diff --
    
    Yeah, I finally caught up last night.  It's copy and paste followed by a context change.  I made them "media metadata".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67218396
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    --- End diff --
    
    Same, "image" needs to be removed now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67221005
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/test/java/org/apache/nifi/processors/media/TestExtractMediaMetadata.java ---
    @@ -0,0 +1,411 @@
    +/*
    + * 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.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(5, 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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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 testNoFlowFile() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.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 testJunkBytes() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*");
    +        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];
    --- End diff --
    
    Random is not used and the byte[] will always be initialized to be full of zeros[1]. Random should be used to fill it like so: 
            `random.nextBytes(bytes);`
    
    [1] http://docs.oracle.com/javase/tutorial/java/nutsandbolts/datatypes.html


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67387942
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    --- End diff --
    
    Added.  I also added penalization on the transfer to failure.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67191945
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    +        } catch (ProcessException e) {
    +            logger.error("Failed to extract media metadata from {} due to {}", new Object[]{flowfile, e});
    +            session.transfer(flowfile, FAILURE);
    +        }
    +    }
    +
    +    private Map<String, String> tika_parse(FlowFile ff, InputStream sourceStream, String prefix, Integer max) throws IOException, TikaException, SAXException {
    --- End diff --
    
    The FlowFile is not used in the method (only the inputstream is) suggest removing it from the method signature.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67218355
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    --- End diff --
    
    The original version I created was just for WAV files, "image" needs to be removed now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    Gonna review this for potential inclusion in 0.7.0.
    
    There are merge conflicts, @jskora can you rebase it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67221558
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/test/java/org/apache/nifi/processors/media/TestExtractMediaMetadata.java ---
    @@ -0,0 +1,411 @@
    +/*
    + * 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.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(5, 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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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 testNoFlowFile() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.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 testJunkBytes() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*");
    +        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];
    +        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 testFilenameFilter() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[dDoOcC]");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.txt");
    +        flowFile0.assertAttributeNotExists("txt.Content-Type");
    +        flowFile0.assertAttributeNotExists("txt.X-Parsed-By");
    +    }
    +
    +    @Test
    +    public void testMimeTypeFilter() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "doc.*");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.txt");
    +        flowFile0.assertAttributeNotExists("txt.Content-Type");
    +        flowFile0.assertAttributeNotExists("txt.X-Parsed-By");
    +    }
    +
    +    @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/notImage.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", "notImage.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"));
    --- End diff --
    
    This should assert that the attributes that were filtered out are not added to the FlowFile


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    @jskora were you able to rebase and open a new PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    Thanks for the updates @jskora, unless if you say otherwise I'm going to assume you're gonna get this knocked out for 0.7.0 as soon as possible. If you think you need more time and can have it slide then feel free to remove the 0.7.0 tag.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    Hoping to do that today.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    Replaced with [Pull Request 556](https://github.com/apache/nifi/pull/556).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    @JPercivall, I'll try to get this updated tomorrow morning.  This was originally done in February, but I'm hoping the rebase won't be too bad.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67385461
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    --- End diff --
    
    Corrected reference to "<Metadata Key Prefix>" and noted that if not provided it's just the attribute name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67189534
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    --- End diff --
    
    Same comment as FILENAME_FILTER


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67189960
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    --- End diff --
    
    Should support for expression language be added here? As an example, in the event multiple different types of data are coming through one processor the user may want to construct the regex using attributes on the flowfile


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #252: NIFI-615 - Create a processor to extract WAV file character...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on the issue:

    https://github.com/apache/nifi/pull/252
  
    So, the problems you had with the JPG file are two separate issues.
    
    - The default `BodyContentHandler` passed to the parser can only handle 100,000 bytes, so any file larger than that produces that message and is only partially parsed.  **_I fixed this by adding an optional `Content Buffer Size` property to allow that to be increased or set to unlimited and related tests._**
    - The Tika JPEG parser appears to have a reference error.  It references the class `com.drew.lang.BufferReader` but that does not appear to be in the Tika package or the version of the Drew Noakes Metadata Extractor package that Tika references.  **_I'm looking into whether updating to a newer Tika will fix this or if we have dependency conflicts causing the problem._**


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67386979
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/test/java/org/apache/nifi/processors/media/TestExtractMediaMetadata.java ---
    @@ -0,0 +1,411 @@
    +/*
    + * 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.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(5, 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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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 testNoFlowFile() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        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.FILENAME_FILTER, ".*[tTxXtT]");
    +        runner.setProperty(ExtractMediaMetadata.MIME_TYPE_FILTER, "text/.*");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_FILTER, "");
    +        runner.setProperty(ExtractMediaMetadata.METADATA_KEY_PREFIX, "txt.");
    +        runner.assertValid();
    +
    +        runner.enqueue(new File("target/test-classes/notImage.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", "notImage.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 testJunkBytes() throws IOException {
    +        final TestRunner runner = TestRunners.newTestRunner(new ExtractMediaMetadata());
    +        runner.setProperty(ExtractMediaMetadata.FILENAME_FILTER, ".*");
    +        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];
    --- End diff --
    
    Good catch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-615 - Create a processor to extract WAV fi...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the pull request:

    https://github.com/apache/nifi/pull/252#issuecomment-198149231
  
    @jskora do the attribute names that come from the media bundles have anything special to them in terms of how tika handles them or are they purely as found in the metadata of the raw entities?  Just want to make sure there isn't some special mapping/normalization to worry about as versions of tika evolves.
    
    Also, i've not built this yet but do you know how large those parsers end up being when pulled in for the  nar?  I recall for some reason they can be quite huge which is why we have avoided them so far.  Thinking being the are perfectly fine once we have the registry.  Might be fine now too but curious.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67190110
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    --- End diff --
    
    Copy and paste error: "have image metadata extracted"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67189259
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    --- End diff --
    
    I am leaning towards removing this. In what use-case would a user want to send multiple different types of data through this processor and then continue to process filtered out FlowFiles in the same way as the rest of the stream? 
    
    I don't see it being a heavily used feature and in the interested of making processors as modular and simple to use as possible this use-case would be better be implemented as a "RouteOnAttribute" processor prior to this processor.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67217364
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/test/java/org/apache/nifi/processors/media/TestExtractMediaMetadata.java ---
    @@ -0,0 +1,411 @@
    +/*
    + * 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.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(5, propertyValues.size());
    +    }
    +
    +    @Test
    +    public void testRelationShips() {
    --- End diff --
    
    nit pick: relationship shouldn't be camel-cased  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67402505
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    --- End diff --
    
    Why can't the user filter on the mime-type already on the flowfile? If it's hit the point where Tika has has already parsed it (the most significant processing time), to get the parsed mime-type, then the only part left is to add the metadata as attributes. 
    
    There would be a use-case where the user doesn't want to add the attributes after it's been parsed? 
    
    The main use-case of this filter I was thinking of was to filter the mime-types that are parsed so that only the ones that will succeed are attempted. This use-case would not be achieved by this mime-type filter because it would have already attempted parsing the document. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-615 - Create a processor to extract WAV fi...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the pull request:

    https://github.com/apache/nifi/pull/252#issuecomment-198411900
  
    size = cool should be just fine then.
    attributes = interesting if they are named by tika.  rather than being as they're found in the source media.  Probably makes sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67193173
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    +        } catch (ProcessException e) {
    +            logger.error("Failed to extract media metadata from {} due to {}", new Object[]{flowfile, e});
    +            session.transfer(flowfile, FAILURE);
    +        }
    +    }
    +
    +    private Map<String, String> tika_parse(FlowFile ff, InputStream sourceStream, String prefix, Integer max) throws IOException, TikaException, SAXException {
    +        final Metadata metadata = new Metadata();
    +        final TikaInputStream tikaInputStream = TikaInputStream.get(sourceStream);
    +        new AutoDetectParser().parse(tikaInputStream, new BodyContentHandler(), metadata);
    --- End diff --
    
    This should be created once and re-used. According to this[1] thread it is "immutable and thus fully thread-safe. "
    
    [1] http://lucene.472066.n3.nabble.com/Thread-Safety-td646195.html


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67389343
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    --- End diff --
    
    I get it for FILENAME_FILTER, but the MIME type is extracted in this processor so it can not be used as a filter in front of this processor.
    
    I'll pull FILENAME_FILTER and leave MIME_TYPE_FILTER.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67216685
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    --- End diff --
    
    You would only have to have two paths and the only difference would be if they include this processor. You have a RouteOnAttribute that routes using the "matches" EL function[1]. If it matches route to ExtractMediaMetadata and then onto the rest of the flow. If it doesn't match then route directly to the rest of the flow.
    
    [1] https://nifi.apache.org/docs/nifi-docs/html/expression-language-guide.html#matches


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67190072
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    --- End diff --
    
    Copy and paste error: "has image metadata extracted"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67222856
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    --- End diff --
    
    WAV is an audio type, this is the same description as the "ExtractImageMetadata" processor but yup just a small fix


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67193966
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    +        } catch (ProcessException e) {
    +            logger.error("Failed to extract media metadata from {} due to {}", new Object[]{flowfile, e});
    +            session.transfer(flowfile, FAILURE);
    +        }
    +    }
    +
    +    private Map<String, String> tika_parse(FlowFile ff, InputStream sourceStream, String prefix, Integer max) throws IOException, TikaException, SAXException {
    +        final Metadata metadata = new Metadata();
    +        final TikaInputStream tikaInputStream = TikaInputStream.get(sourceStream);
    +        new AutoDetectParser().parse(tikaInputStream, new BodyContentHandler(), metadata);
    +        final String content_type = metadata.get(Metadata.CONTENT_TYPE);
    +
    +        // if parsed MIME type doesn't match filter fail fast without processing attributes
    +        final Pattern mimeTypeFilter = mimeTypeFilterRef.get();
    +        if (mimeTypeFilter != null && (content_type == null || !mimeTypeFilter.matcher(metadata.get(Metadata.CONTENT_TYPE)).matches())) {
    +            return null;
    +        }
    +
    +        final Map<String, String> results = new HashMap<>();
    +        final Pattern metadataKeyFilter = metadataKeyFilterRef.get();
    +        final StringBuilder dataBuilder = new StringBuilder();
    +        final String safePrefix = (prefix == null) ? "" : prefix;
    +        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(", ");
    +                    }
    +                    dataBuilder.append(val);
    --- End diff --
    
    How large can multi-valued keys be? Should there be a property for max length?
    
    I am worried about concatenating many values together and creating a huge attribute. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by jskora <gi...@git.apache.org>.
Github user jskora closed the pull request at:

    https://github.com/apache/nifi/pull/252


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #252: NIFI-615 - Create a processor to extract WAV file ch...

Posted by JPercivall <gi...@git.apache.org>.
Github user JPercivall commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/252#discussion_r67195219
  
    --- Diff: nifi-nar-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.flowfile.attributes.CoreAttributes;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.nifi.util.ObjectHolder;
    +
    +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.apache.tika.sax.BodyContentHandler;
    +import org.xml.sax.SAXException;
    +
    +@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."
    +        + "For the more details and the list of supported file types, visit the library's website "
    +        + "at http://tika.apache.org/.")
    +@WritesAttributes({@WritesAttribute(attribute = "tika.<attribute>", description = "The extracted content metadata "
    +        + "will be inserted with the attribute name \"tika.<attribute>\". ")})
    +@SupportsBatching
    +public class ExtractMediaMetadata extends AbstractProcessor {
    +
    +    public 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)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor FILENAME_FILTER = new PropertyDescriptor.Builder()
    +            .name("File Name Filter")
    +            .description("A regular expression identifying file names which metadata should extracted.  As flowfiles"
    +                    + " are processed, if the file name matches this regular expression or this expression is"
    +                    + " blank, the flowfile will be scanned for it's MIME type and metadata.  If left blank, all"
    +                    + " flowfiles will be scanned.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MIME_TYPE_FILTER = new PropertyDescriptor.Builder()
    +            .name("MIME Type Filter")
    +            .description("A regular expression identifying MIME types for which metadata should extracted.  Flowfiles"
    +                    + " selected for scanning by the File Name Filter are parsed to determine the MIME type and extract"
    +                    + " metadata.  If the MIME type found matches this regular expression or this expression is"
    +                    + " blank, the metadata keys that match the Metadata Key Filter will be added to the flowfile"
    +                    + " attributes.  There is no guarantee in what order attributes will be produced.  If"
    +                    + " left blank, metadata will be extracted from all flow files selected for scanning.")
    +            .required(false)
    +            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
    +            .build();
    +
    +    public 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();
    +
    +    public 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();
    +
    +    public static final Relationship SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Any FlowFile that successfully has image metadata extracted will be routed to success")
    +            .build();
    +
    +    public static final Relationship FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("Any FlowFile that fails to have image metadata extracted will be routed to failure")
    +            .build();
    +
    +    private Set<Relationship> relationships;
    +    private List<PropertyDescriptor> properties;
    +
    +    private final AtomicReference<Pattern> filenameFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> mimeTypeFilterRef = new AtomicReference<>();
    +    private final AtomicReference<Pattern> metadataKeyFilterRef = new AtomicReference<>();
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(MAX_NUMBER_OF_ATTRIBUTES);
    +        properties.add(FILENAME_FILTER);
    +        properties.add(MIME_TYPE_FILTER);
    +        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;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) {
    +        final String fileNamePatternInput = context.getProperty(FILENAME_FILTER).getValue();
    +        if (fileNamePatternInput != null && fileNamePatternInput.length() > 0) {
    +            filenameFilterRef.set(Pattern.compile(fileNamePatternInput));
    +        } else {
    +            filenameFilterRef.set(null);
    +        }
    +
    +        final String mimeTypeFilterInput = context.getProperty(MIME_TYPE_FILTER).getValue();
    +        if (mimeTypeFilterInput != null && mimeTypeFilterInput.length() > 0) {
    +            mimeTypeFilterRef.set(Pattern.compile(mimeTypeFilterInput));
    +        } else {
    +            mimeTypeFilterRef.set(null);
    +        }
    +
    +        String metadataKeyFilterInput = context.getProperty(METADATA_KEY_FILTER).getValue();
    +        if (metadataKeyFilterInput != null && metadataKeyFilterInput.length() > 0) {
    +            metadataKeyFilterRef.set(Pattern.compile(metadataKeyFilterInput));
    +        } else {
    +            metadataKeyFilterRef.set(null);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        FlowFile flowfile = session.get();
    +        if (flowfile == null) {
    +            return;
    +        }
    +
    +        // fail fast if filename doesn't match filter
    +        Pattern filenameFilter = filenameFilterRef.get();
    +        if (filenameFilter != null && !filenameFilter.matcher(flowfile.getAttribute(CoreAttributes.FILENAME.key())).matches()) {
    +            session.transfer(flowfile, SUCCESS);
    +            return;
    +        }
    +
    +        final ProcessorLog logger = this.getLogger();
    +        final ObjectHolder<Map<String, String>> value = new ObjectHolder<>(null);
    +        final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger();
    +        final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions().getValue();
    +        final FlowFile ff = flowfile;
    +
    +        try {
    +            session.read(flowfile, new InputStreamCallback() {
    +                @Override
    +                public void process(InputStream in) throws IOException {
    +                    try {
    +                        Map<String, String> results = tika_parse(ff, in, prefix, max);
    +                        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);
    +        } catch (ProcessException e) {
    +            logger.error("Failed to extract media metadata from {} due to {}", new Object[]{flowfile, e});
    +            session.transfer(flowfile, FAILURE);
    +        }
    +    }
    +
    +    private Map<String, String> tika_parse(FlowFile ff, InputStream sourceStream, String prefix, Integer max) throws IOException, TikaException, SAXException {
    +        final Metadata metadata = new Metadata();
    +        final TikaInputStream tikaInputStream = TikaInputStream.get(sourceStream);
    +        new AutoDetectParser().parse(tikaInputStream, new BodyContentHandler(), metadata);
    +        final String content_type = metadata.get(Metadata.CONTENT_TYPE);
    +
    +        // if parsed MIME type doesn't match filter fail fast without processing attributes
    +        final Pattern mimeTypeFilter = mimeTypeFilterRef.get();
    +        if (mimeTypeFilter != null && (content_type == null || !mimeTypeFilter.matcher(metadata.get(Metadata.CONTENT_TYPE)).matches())) {
    +            return null;
    +        }
    +
    +        final Map<String, String> results = new HashMap<>();
    +        final Pattern metadataKeyFilter = metadataKeyFilterRef.get();
    +        final StringBuilder dataBuilder = new StringBuilder();
    +        final String safePrefix = (prefix == null) ? "" : prefix;
    +        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(", ");
    +                    }
    +                    dataBuilder.append(val);
    +                }
    +            } else {
    +                dataBuilder.append(metadata.get(key));
    +            }
    +            results.put(safePrefix + key, dataBuilder.toString().trim());
    --- End diff --
    
    In terms of performance, it's probably better to remove safePrefix and just have a check to see if prefix is null. It's my understanding that checking if things are null is much faster/more efficient than string concatenation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---