Github user mattyb149 commented on a diff in the pull request: https://github.com/apache/nifi/pull/248#discussion_r67246009 --- 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 -- @rickysaltzer everything LGTM but wante to get your opinion on the above, will merge once addressed/discussed :)
--- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---