[
https://issues.apache.org/jira/browse/NIFI-1568?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15338097#comment-15338097
]
ASF GitHub Bot commented on NIFI-1568:
--------------------------------------
Github user joewitt commented on a diff in the pull request:
https://github.com/apache/nifi/pull/248#discussion_r67604647
--- Diff:
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java
---
@@ -154,75 +171,88 @@ protected void init(final
ProcessorInitializationContext context) {
return properties;
}
- @Override
- public void onTrigger(final ProcessContext context, final
ProcessSession session) throws ProcessException {
- FlowFile flowFile = session.get();
- if (flowFile == null) {
- return;
- }
+ @OnStopped
+ public void onStopped() {
+ unpacker = null;
+ fileFilter = null;
+ }
- final ComponentLog logger = getLogger();
- String packagingFormat =
context.getProperty(PACKAGING_FORMAT).getValue().toLowerCase();
- if (AUTO_DETECT_FORMAT.equals(packagingFormat)) {
- final String mimeType =
flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
- if (mimeType == null) {
- logger.error("No mime.type attribute set for {}; routing
to failure", new Object[]{flowFile});
- session.transfer(flowFile, REL_FAILURE);
- return;
- }
+ @OnScheduled
+ public void onScheduled(ProcessContext context) throws
ProcessException {
+ if (fileFilter == null) {
+ fileFilter =
Pattern.compile(context.getProperty(FILE_FILTER).getValue());
+ tarUnpacker = new TarUnpacker(fileFilter);
+ zipUnpacker = new ZipUnpacker(fileFilter);
+ flowFileStreamV3Unpacker = new FlowFileStreamUnpacker(new
FlowFileUnpackagerV3());
+ flowFileStreamV2Unpacker = new FlowFileStreamUnpacker(new
FlowFileUnpackagerV2());
+ flowFileTarUnpacker = new FlowFileStreamUnpacker(new
FlowFileUnpackagerV1());
+ }
- switch (mimeType.toLowerCase()) {
- case "application/tar":
- packagingFormat = TAR_FORMAT;
- break;
- case "application/x-tar":
- packagingFormat = TAR_FORMAT;
- break;
- case "application/zip":
- packagingFormat = ZIP_FORMAT;
- break;
- case "application/flowfile-v3":
- packagingFormat = FLOWFILE_STREAM_FORMAT_V3;
- break;
- case "application/flowfile-v2":
- packagingFormat = FLOWFILE_STREAM_FORMAT_V2;
- break;
- case "application/flowfile-v1":
- packagingFormat = FLOWFILE_TAR_FORMAT;
- break;
- default: {
- logger.info("Cannot unpack {} because its mime.type
attribute is set to '{}', which is not a format that can be unpacked; routing
to 'success'", new Object[]{flowFile, mimeType});
- session.transfer(flowFile, REL_SUCCESS);
- return;
- }
- }
+ PackageFormat format =
PackageFormat.getFormat(context.getProperty(PACKAGING_FORMAT).getValue());
+ if (format != PackageFormat.AUTO_DETECT_FORMAT && unpacker ==
null) {
+ initUnpacker(format);
}
+ }
- final Unpacker unpacker;
- final boolean addFragmentAttrs;
+ public void initUnpacker(PackageFormat packagingFormat) {
switch (packagingFormat) {
case TAR_FORMAT:
- unpacker = new TarUnpacker();
+ case X_TAR_FORMAT:
+ unpacker = tarUnpacker;
addFragmentAttrs = true;
break;
case ZIP_FORMAT:
- unpacker = new ZipUnpacker();
+ unpacker = zipUnpacker;
addFragmentAttrs = true;
break;
case FLOWFILE_STREAM_FORMAT_V2:
- unpacker = new FlowFileStreamUnpacker(new
FlowFileUnpackagerV2());
+ unpacker = flowFileStreamV2Unpacker;
addFragmentAttrs = false;
break;
case FLOWFILE_STREAM_FORMAT_V3:
- unpacker = new FlowFileStreamUnpacker(new
FlowFileUnpackagerV3());
+ unpacker = flowFileStreamV3Unpacker;
addFragmentAttrs = false;
break;
case FLOWFILE_TAR_FORMAT:
- unpacker = new FlowFileStreamUnpacker(new
FlowFileUnpackagerV1());
+ unpacker = flowFileTarUnpacker;
addFragmentAttrs = false;
break;
- default:
- throw new AssertionError("Packaging Format was " +
context.getProperty(PACKAGING_FORMAT).getValue());
+ case AUTO_DETECT_FORMAT:
+ // The format of the unpacker should be known before
initialization
+ throw new ProcessException(packagingFormat + " is not a
valid packaging format");
+ }
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context, final
ProcessSession session) throws ProcessException {
+ FlowFile flowFile = session.get();
+ if (flowFile == null) {
+ return;
+ }
+
+ final ComponentLog logger = getLogger();
+ PackageFormat packagingFormat =
PackageFormat.getFormat(context.getProperty(PACKAGING_FORMAT).getValue().toLowerCase());
+ if (packagingFormat == PackageFormat.AUTO_DETECT_FORMAT) {
+ packagingFormat = null;
+ final String mimeType =
flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+ if (mimeType == null) {
+ logger.error("No mime.type attribute set for {}; routing
to failure", new Object[]{flowFile});
+ session.transfer(flowFile, REL_FAILURE);
+ return;
+ }
+
+ for (PackageFormat format: PackageFormat.values()) {
+ if (mimeType.toLowerCase().equals(format.getMimeType())) {
+ packagingFormat = format;
+ }
+ }
+ if (packagingFormat == null) {
+ logger.info("Cannot unpack {} because its mime.type
attribute is set to '{}', which is not a format that can be unpacked; routing
to 'success'", new Object[]{flowFile, mimeType});
+ session.transfer(flowFile, REL_SUCCESS);
--- End diff --
I agree the naming of 'success' is unfortunate given this case. That said
not sure it would be worth the 'weight' so to speak of adding another
relationship just for this case because it should be pretty rare/odd that
something would go that route. I would be fine if you guys do decide to go
that route (even though yes it means flows would need adjustment) just saying
it may not really be worth it. We do allow for this sort of change in our
versioning guidance along a minor release. Making it so that a flow becomes
invalid is ok if the juice is considered worth the squeeze. What is not ok is
fundamentally changing the behavior of a processor such that flows could be
broken with the user having a chance to know that.
> Allow user to specify file filter regex when unpacking zip/tar archives
> ------------------------------------------------------------------------
>
> Key: NIFI-1568
> URL: https://issues.apache.org/jira/browse/NIFI-1568
> Project: Apache NiFi
> Issue Type: Improvement
> Reporter: Ricky Saltzer
> Assignee: Ricky Saltzer
> Fix For: 1.0.0, 0.7.0
>
>
> There are times where you may want to only extract a portion of an archive,
> such as a specific folder, or perhaps a specific file. Similar to how the
> {{GetFile}} processor works, we should provide a property ({{File Filter}})
> which by default extracts all files. The user can modify this property to
> extract only files they wish to process downstream.
>
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)