You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "mattyb149 (via GitHub)" <gi...@apache.org> on 2023/04/17 19:33:59 UTC

[GitHub] [nifi] mattyb149 opened a new pull request, #7180: NIFI-11466: Add ModifyCompression processor

mattyb149 opened a new pull request, #7180:
URL: https://github.com/apache/nifi/pull/7180

   # Summary
   
   [NIFI-11466](https://issues.apache.org/jira/browse/NIFI-11466) This PR adds a ModifyCompression processor which combines the functionality of CompressContent with the ability to decompress the input then recompress the output, avoiding any disk usage between multiple CompressContent processors (decompress -> recompress) and uses sensible in-memory buffers to avoid out-of-memory errors.
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [x] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [x] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7180: NIFI-11466: Add ModifyCompression processor

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7180:
URL: https://github.com/apache/nifi/pull/7180#discussion_r1170361256


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven

Review Comment:
   This annotation should be removed since Event-Driven will not be supported in future versions.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")

Review Comment:
   The valid values should be removed from the description because they are provided through the allowable values generated documentation.
   ```suggestion
               .description("The compression format to use for output FlowFiles.")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")

Review Comment:
   Instead of `Decompression Format` and `Compression Format`, what do you think about naming these `Input Compression` and `Output Compression`? That would help clarify the purpose of the properties.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;

Review Comment:
   The properties and relationships can be declared directly as static final values.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;

Review Comment:
   With refactoring of allowable values to an `enum`, this map can be removed.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {

Review Comment:
   This method can be removed with refactoring to static values for properties and relationships.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DECOMPRESSION_FORMAT);
+        properties.add(COMPRESSION_FORMAT);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        mimeTypeMap.put("application/gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/x-gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/x-deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-lzma", COMPRESSION_FORMAT_LZMA);
+        mimeTypeMap.put("application/x-snappy", COMPRESSION_FORMAT_SNAPPY);
+        mimeTypeMap.put("application/x-snappy-hadoop", COMPRESSION_FORMAT_SNAPPY_HADOOP);
+        mimeTypeMap.put("application/x-snappy-framed", COMPRESSION_FORMAT_SNAPPY_FRAMED);
+        mimeTypeMap.put("application/x-lz4-framed", COMPRESSION_FORMAT_LZ4_FRAMED);
+        mimeTypeMap.put("application/zstd", COMPRESSION_FORMAT_ZSTD);
+        mimeTypeMap.put("application/x-brotli", COMPRESSION_FORMAT_BROTLI);
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final long sizeBeforeCompression = flowFile.getSize();
+
+        String decompressionFormatValue = context.getProperty(DECOMPRESSION_FORMAT).getValue();
+        if (decompressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            decompressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (decompressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Decompression Format; routing to failure", flowFile, mimeType);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+        }
+
+        String compressionFormatValue = context.getProperty(COMPRESSION_FORMAT).getValue();
+        if (compressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            compressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (compressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Compression Format; routing to success without decompressing", flowFile, mimeType);
+                session.transfer(flowFile, REL_SUCCESS);
+                return;
+            }
+        }
+
+        final String decompressionFormat = decompressionFormatValue;
+        final String compressionFormat = compressionFormatValue;
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
+        final StopWatch stopWatch = new StopWatch(true);
+
+        final String inputFileExtension = getFileExtension(decompressionFormat);
+        final String outputFileExtension = getFileExtension(compressionFormat);
+
+        try {
+            flowFile = session.write(flowFile, (rawIn, rawOut) -> {
+                final OutputStream compressionOut;
+                final InputStream compressionIn;
+
+                final OutputStream bufferedOut = new BufferedOutputStream(rawOut, 65536);
+                final InputStream bufferedIn = new BufferedInputStream(rawIn, 65536);
+
+                try {
+                    // Decompress data by creating an InputStream for the decompression format, then recompress using the compression format
+                    switch (decompressionFormat.toLowerCase()) {
+                        case COMPRESSION_FORMAT_LZMA:
+                            compressionIn = new LzmaInputStream(bufferedIn, new Decoder());
+                            break;
+                        case COMPRESSION_FORMAT_XZ_LZMA2:
+                            compressionIn = new XZInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_BZIP2:
+                            // need this two-arg constructor to support concatenated streams
+                            compressionIn = new BZip2CompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_GZIP:
+                            compressionIn = new GzipCompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_DEFLATE:
+                            compressionIn = new InflaterInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY:
+                            compressionIn = new SnappyInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_HADOOP:
+                            throw new Exception("Cannot decompress snappy-hadoop.");
+                        case COMPRESSION_FORMAT_SNAPPY_FRAMED:
+                            compressionIn = new SnappyFramedInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_LZ4_FRAMED:
+                            compressionIn = new FramedLZ4CompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_ZSTD:
+                            compressionIn = new ZstdCompressorInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_BROTLI:
+                            Brotli4jLoader.ensureAvailability();
+                            compressionIn = new BrotliInputStream(bufferedIn);
+                            break;
+                        case DECOMPRESSION_FORMAT_NONE:
+                            compressionIn = bufferedIn;
+                            break;
+                        default:
+                            compressionIn = new CompressorStreamFactory().createCompressorInputStream(decompressionFormat.toLowerCase(), bufferedIn);
+                    }
+
+                    switch (compressionFormat.toLowerCase()) {
+                        case COMPRESSION_FORMAT_GZIP:
+                            int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new GZIPOutputStream(bufferedOut, compressionLevel);
+                            mimeTypeRef.set("application/gzip");
+                            break;
+                        case COMPRESSION_FORMAT_DEFLATE:
+                            compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new DeflaterOutputStream(bufferedOut, new Deflater(compressionLevel));
+                            mimeTypeRef.set("application/gzip");
+                            break;
+                        case COMPRESSION_FORMAT_LZMA:
+                            compressionOut = new LzmaOutputStream.Builder(bufferedOut).build();
+                            mimeTypeRef.set("application/x-lzma");
+                            break;
+                        case COMPRESSION_FORMAT_XZ_LZMA2:
+                            final int xzCompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new XZOutputStream(bufferedOut, new LZMA2Options(xzCompressionLevel));
+                            mimeTypeRef.set("application/x-xz");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY:
+                            compressionOut = new SnappyOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_HADOOP:
+                            compressionOut = new SnappyHadoopCompatibleOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy-hadoop");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_FRAMED:
+                            compressionOut = new SnappyFramedOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy-framed");
+                            break;
+                        case COMPRESSION_FORMAT_LZ4_FRAMED:
+                            mimeTypeRef.set("application/x-lz4-framed");
+                            compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut);
+                            break;
+                        case COMPRESSION_FORMAT_ZSTD:
+                            final int zstdcompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger() * 2;
+                            compressionOut = new ZstdCompressorOutputStream(bufferedOut, zstdcompressionLevel);
+                            mimeTypeRef.set("application/zstd");
+                            break;
+                        case COMPRESSION_FORMAT_BROTLI:
+                            Brotli4jLoader.ensureAvailability();
+                            compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            Encoder.Parameters params = new Encoder.Parameters().setQuality(compressionLevel);
+                            compressionOut = new BrotliOutputStream(bufferedOut, params);
+                            mimeTypeRef.set("application/x-brotli");
+                            break;
+                        case COMPRESSION_FORMAT_BZIP2:
+                            mimeTypeRef.set("application/x-bzip2");
+                            compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut);
+                            break;
+                        case COMPRESSION_FORMAT_NONE:
+                        default:
+                            compressionOut = bufferedOut;
+                            break;
+                    }
+
+                } catch (final Exception e) {
+                    closeQuietly(bufferedOut);
+                    throw new IOException(e);
+                }
+
+                try (final InputStream in = compressionIn;
+                     final OutputStream out = compressionOut) {
+                    final byte[] buffer = new byte[8192];
+                    int len;
+                    while ((len = in.read(buffer)) > 0) {
+                        out.write(buffer, 0, len);
+                    }
+                    out.flush();

Review Comment:
   This can be replaced with `StreamUtils.copy()`, which uses an 8 KB buffer as well. The closure of the 
   ```suggestion
                       StreamUtils.copy(in, out);
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DECOMPRESSION_FORMAT);
+        properties.add(COMPRESSION_FORMAT);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        mimeTypeMap.put("application/gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/x-gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/x-deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-lzma", COMPRESSION_FORMAT_LZMA);
+        mimeTypeMap.put("application/x-snappy", COMPRESSION_FORMAT_SNAPPY);
+        mimeTypeMap.put("application/x-snappy-hadoop", COMPRESSION_FORMAT_SNAPPY_HADOOP);
+        mimeTypeMap.put("application/x-snappy-framed", COMPRESSION_FORMAT_SNAPPY_FRAMED);
+        mimeTypeMap.put("application/x-lz4-framed", COMPRESSION_FORMAT_LZ4_FRAMED);
+        mimeTypeMap.put("application/zstd", COMPRESSION_FORMAT_ZSTD);
+        mimeTypeMap.put("application/x-brotli", COMPRESSION_FORMAT_BROTLI);
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final long sizeBeforeCompression = flowFile.getSize();
+
+        String decompressionFormatValue = context.getProperty(DECOMPRESSION_FORMAT).getValue();
+        if (decompressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            decompressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (decompressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Decompression Format; routing to failure", flowFile, mimeType);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+        }
+
+        String compressionFormatValue = context.getProperty(COMPRESSION_FORMAT).getValue();
+        if (compressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            compressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (compressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Compression Format; routing to success without decompressing", flowFile, mimeType);
+                session.transfer(flowFile, REL_SUCCESS);
+                return;
+            }
+        }
+
+        final String decompressionFormat = decompressionFormatValue;
+        final String compressionFormat = compressionFormatValue;
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
+        final StopWatch stopWatch = new StopWatch(true);
+
+        final String inputFileExtension = getFileExtension(decompressionFormat);
+        final String outputFileExtension = getFileExtension(compressionFormat);
+
+        try {
+            flowFile = session.write(flowFile, (rawIn, rawOut) -> {
+                final OutputStream compressionOut;
+                final InputStream compressionIn;
+
+                final OutputStream bufferedOut = new BufferedOutputStream(rawOut, 65536);
+                final InputStream bufferedIn = new BufferedInputStream(rawIn, 65536);
+
+                try {
+                    // Decompress data by creating an InputStream for the decompression format, then recompress using the compression format
+                    switch (decompressionFormat.toLowerCase()) {
+                        case COMPRESSION_FORMAT_LZMA:
+                            compressionIn = new LzmaInputStream(bufferedIn, new Decoder());
+                            break;
+                        case COMPRESSION_FORMAT_XZ_LZMA2:
+                            compressionIn = new XZInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_BZIP2:
+                            // need this two-arg constructor to support concatenated streams
+                            compressionIn = new BZip2CompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_GZIP:
+                            compressionIn = new GzipCompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_DEFLATE:
+                            compressionIn = new InflaterInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY:
+                            compressionIn = new SnappyInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_HADOOP:
+                            throw new Exception("Cannot decompress snappy-hadoop.");
+                        case COMPRESSION_FORMAT_SNAPPY_FRAMED:
+                            compressionIn = new SnappyFramedInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_LZ4_FRAMED:
+                            compressionIn = new FramedLZ4CompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_ZSTD:
+                            compressionIn = new ZstdCompressorInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_BROTLI:
+                            Brotli4jLoader.ensureAvailability();
+                            compressionIn = new BrotliInputStream(bufferedIn);
+                            break;
+                        case DECOMPRESSION_FORMAT_NONE:
+                            compressionIn = bufferedIn;
+                            break;
+                        default:
+                            compressionIn = new CompressorStreamFactory().createCompressorInputStream(decompressionFormat.toLowerCase(), bufferedIn);
+                    }
+
+                    switch (compressionFormat.toLowerCase()) {
+                        case COMPRESSION_FORMAT_GZIP:
+                            int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new GZIPOutputStream(bufferedOut, compressionLevel);
+                            mimeTypeRef.set("application/gzip");
+                            break;
+                        case COMPRESSION_FORMAT_DEFLATE:
+                            compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new DeflaterOutputStream(bufferedOut, new Deflater(compressionLevel));
+                            mimeTypeRef.set("application/gzip");
+                            break;
+                        case COMPRESSION_FORMAT_LZMA:
+                            compressionOut = new LzmaOutputStream.Builder(bufferedOut).build();
+                            mimeTypeRef.set("application/x-lzma");
+                            break;
+                        case COMPRESSION_FORMAT_XZ_LZMA2:
+                            final int xzCompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new XZOutputStream(bufferedOut, new LZMA2Options(xzCompressionLevel));
+                            mimeTypeRef.set("application/x-xz");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY:
+                            compressionOut = new SnappyOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_HADOOP:
+                            compressionOut = new SnappyHadoopCompatibleOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy-hadoop");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_FRAMED:
+                            compressionOut = new SnappyFramedOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy-framed");
+                            break;
+                        case COMPRESSION_FORMAT_LZ4_FRAMED:
+                            mimeTypeRef.set("application/x-lz4-framed");
+                            compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut);
+                            break;
+                        case COMPRESSION_FORMAT_ZSTD:
+                            final int zstdcompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger() * 2;
+                            compressionOut = new ZstdCompressorOutputStream(bufferedOut, zstdcompressionLevel);
+                            mimeTypeRef.set("application/zstd");
+                            break;
+                        case COMPRESSION_FORMAT_BROTLI:
+                            Brotli4jLoader.ensureAvailability();
+                            compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            Encoder.Parameters params = new Encoder.Parameters().setQuality(compressionLevel);
+                            compressionOut = new BrotliOutputStream(bufferedOut, params);
+                            mimeTypeRef.set("application/x-brotli");
+                            break;
+                        case COMPRESSION_FORMAT_BZIP2:
+                            mimeTypeRef.set("application/x-bzip2");
+                            compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut);
+                            break;
+                        case COMPRESSION_FORMAT_NONE:
+                        default:
+                            compressionOut = bufferedOut;
+                            break;
+                    }
+
+                } catch (final Exception e) {
+                    closeQuietly(bufferedOut);

Review Comment:
   The NiFi commons `FileUtils` method can be used.
   ```suggestion
                       FileUtils.closeQuietly(bufferedOut);
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")

Review Comment:
   ```suggestion
               .description("The decompression format to use for input FlowFiles.")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DECOMPRESSION_FORMAT);
+        properties.add(COMPRESSION_FORMAT);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        mimeTypeMap.put("application/gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/x-gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/x-deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-lzma", COMPRESSION_FORMAT_LZMA);
+        mimeTypeMap.put("application/x-snappy", COMPRESSION_FORMAT_SNAPPY);
+        mimeTypeMap.put("application/x-snappy-hadoop", COMPRESSION_FORMAT_SNAPPY_HADOOP);
+        mimeTypeMap.put("application/x-snappy-framed", COMPRESSION_FORMAT_SNAPPY_FRAMED);
+        mimeTypeMap.put("application/x-lz4-framed", COMPRESSION_FORMAT_LZ4_FRAMED);
+        mimeTypeMap.put("application/zstd", COMPRESSION_FORMAT_ZSTD);
+        mimeTypeMap.put("application/x-brotli", COMPRESSION_FORMAT_BROTLI);
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final long sizeBeforeCompression = flowFile.getSize();
+
+        String decompressionFormatValue = context.getProperty(DECOMPRESSION_FORMAT).getValue();
+        if (decompressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            decompressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (decompressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Decompression Format; routing to failure", flowFile, mimeType);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+        }
+
+        String compressionFormatValue = context.getProperty(COMPRESSION_FORMAT).getValue();
+        if (compressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            compressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (compressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Compression Format; routing to success without decompressing", flowFile, mimeType);
+                session.transfer(flowFile, REL_SUCCESS);
+                return;
+            }
+        }
+
+        final String decompressionFormat = decompressionFormatValue;
+        final String compressionFormat = compressionFormatValue;
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
+        final StopWatch stopWatch = new StopWatch(true);
+
+        final String inputFileExtension = getFileExtension(decompressionFormat);
+        final String outputFileExtension = getFileExtension(compressionFormat);
+
+        try {
+            flowFile = session.write(flowFile, (rawIn, rawOut) -> {
+                final OutputStream compressionOut;
+                final InputStream compressionIn;
+
+                final OutputStream bufferedOut = new BufferedOutputStream(rawOut, 65536);
+                final InputStream bufferedIn = new BufferedInputStream(rawIn, 65536);

Review Comment:
   Recommend declaring `65536` as a static value named `STREAM_BUFFER_SIZE` or similar.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();

Review Comment:
   This is a good opportunity to change this approach from a `Boolean` to a strategy property. The property could be named `Output Filename Strategy` with a default value of `ORIGINAL` and and allowed value of `UPDATED`, or something similar.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";

Review Comment:
   These options should be extracted to a separate `enum` named something like `CompressionFormat` that implements `DescribedValue` and also holds associated properties like MIME Type and file extension. That should help simplify the overall implementation.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DECOMPRESSION_FORMAT);
+        properties.add(COMPRESSION_FORMAT);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        mimeTypeMap.put("application/gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/x-gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/x-deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-lzma", COMPRESSION_FORMAT_LZMA);
+        mimeTypeMap.put("application/x-snappy", COMPRESSION_FORMAT_SNAPPY);
+        mimeTypeMap.put("application/x-snappy-hadoop", COMPRESSION_FORMAT_SNAPPY_HADOOP);
+        mimeTypeMap.put("application/x-snappy-framed", COMPRESSION_FORMAT_SNAPPY_FRAMED);
+        mimeTypeMap.put("application/x-lz4-framed", COMPRESSION_FORMAT_LZ4_FRAMED);
+        mimeTypeMap.put("application/zstd", COMPRESSION_FORMAT_ZSTD);
+        mimeTypeMap.put("application/x-brotli", COMPRESSION_FORMAT_BROTLI);
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final long sizeBeforeCompression = flowFile.getSize();
+
+        String decompressionFormatValue = context.getProperty(DECOMPRESSION_FORMAT).getValue();
+        if (decompressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            decompressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (decompressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Decompression Format; routing to failure", flowFile, mimeType);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+        }
+
+        String compressionFormatValue = context.getProperty(COMPRESSION_FORMAT).getValue();
+        if (compressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            compressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (compressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Compression Format; routing to success without decompressing", flowFile, mimeType);
+                session.transfer(flowFile, REL_SUCCESS);
+                return;
+            }
+        }
+
+        final String decompressionFormat = decompressionFormatValue;
+        final String compressionFormat = compressionFormatValue;
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
+        final StopWatch stopWatch = new StopWatch(true);
+
+        final String inputFileExtension = getFileExtension(decompressionFormat);
+        final String outputFileExtension = getFileExtension(compressionFormat);
+
+        try {
+            flowFile = session.write(flowFile, (rawIn, rawOut) -> {
+                final OutputStream compressionOut;
+                final InputStream compressionIn;
+
+                final OutputStream bufferedOut = new BufferedOutputStream(rawOut, 65536);
+                final InputStream bufferedIn = new BufferedInputStream(rawIn, 65536);
+
+                try {
+                    // Decompress data by creating an InputStream for the decompression format, then recompress using the compression format
+                    switch (decompressionFormat.toLowerCase()) {
+                        case COMPRESSION_FORMAT_LZMA:
+                            compressionIn = new LzmaInputStream(bufferedIn, new Decoder());
+                            break;
+                        case COMPRESSION_FORMAT_XZ_LZMA2:
+                            compressionIn = new XZInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_BZIP2:
+                            // need this two-arg constructor to support concatenated streams
+                            compressionIn = new BZip2CompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_GZIP:
+                            compressionIn = new GzipCompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_DEFLATE:
+                            compressionIn = new InflaterInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY:
+                            compressionIn = new SnappyInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_HADOOP:
+                            throw new Exception("Cannot decompress snappy-hadoop.");
+                        case COMPRESSION_FORMAT_SNAPPY_FRAMED:
+                            compressionIn = new SnappyFramedInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_LZ4_FRAMED:
+                            compressionIn = new FramedLZ4CompressorInputStream(bufferedIn, true);
+                            break;
+                        case COMPRESSION_FORMAT_ZSTD:
+                            compressionIn = new ZstdCompressorInputStream(bufferedIn);
+                            break;
+                        case COMPRESSION_FORMAT_BROTLI:
+                            Brotli4jLoader.ensureAvailability();
+                            compressionIn = new BrotliInputStream(bufferedIn);
+                            break;
+                        case DECOMPRESSION_FORMAT_NONE:
+                            compressionIn = bufferedIn;
+                            break;
+                        default:
+                            compressionIn = new CompressorStreamFactory().createCompressorInputStream(decompressionFormat.toLowerCase(), bufferedIn);
+                    }
+
+                    switch (compressionFormat.toLowerCase()) {
+                        case COMPRESSION_FORMAT_GZIP:
+                            int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new GZIPOutputStream(bufferedOut, compressionLevel);
+                            mimeTypeRef.set("application/gzip");
+                            break;
+                        case COMPRESSION_FORMAT_DEFLATE:
+                            compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new DeflaterOutputStream(bufferedOut, new Deflater(compressionLevel));
+                            mimeTypeRef.set("application/gzip");
+                            break;
+                        case COMPRESSION_FORMAT_LZMA:
+                            compressionOut = new LzmaOutputStream.Builder(bufferedOut).build();
+                            mimeTypeRef.set("application/x-lzma");
+                            break;
+                        case COMPRESSION_FORMAT_XZ_LZMA2:
+                            final int xzCompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            compressionOut = new XZOutputStream(bufferedOut, new LZMA2Options(xzCompressionLevel));
+                            mimeTypeRef.set("application/x-xz");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY:
+                            compressionOut = new SnappyOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_HADOOP:
+                            compressionOut = new SnappyHadoopCompatibleOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy-hadoop");
+                            break;
+                        case COMPRESSION_FORMAT_SNAPPY_FRAMED:
+                            compressionOut = new SnappyFramedOutputStream(bufferedOut);
+                            mimeTypeRef.set("application/x-snappy-framed");
+                            break;
+                        case COMPRESSION_FORMAT_LZ4_FRAMED:
+                            mimeTypeRef.set("application/x-lz4-framed");
+                            compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut);
+                            break;
+                        case COMPRESSION_FORMAT_ZSTD:
+                            final int zstdcompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger() * 2;
+                            compressionOut = new ZstdCompressorOutputStream(bufferedOut, zstdcompressionLevel);
+                            mimeTypeRef.set("application/zstd");
+                            break;
+                        case COMPRESSION_FORMAT_BROTLI:
+                            Brotli4jLoader.ensureAvailability();
+                            compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                            Encoder.Parameters params = new Encoder.Parameters().setQuality(compressionLevel);
+                            compressionOut = new BrotliOutputStream(bufferedOut, params);
+                            mimeTypeRef.set("application/x-brotli");
+                            break;
+                        case COMPRESSION_FORMAT_BZIP2:
+                            mimeTypeRef.set("application/x-bzip2");
+                            compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut);
+                            break;
+                        case COMPRESSION_FORMAT_NONE:
+                        default:
+                            compressionOut = bufferedOut;
+                            break;
+                    }
+
+                } catch (final Exception e) {
+                    closeQuietly(bufferedOut);
+                    throw new IOException(e);

Review Comment:
   A message should be included with the exception.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyCompression.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.standard;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.util.StopWatch;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    public static final String DECOMPRESSION_FORMAT_NONE = "no decompression";
+    public static final String COMPRESSION_FORMAT_NONE = "no compression";
+    public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute";
+    public static final String COMPRESSION_FORMAT_GZIP = "gzip";
+    public static final String COMPRESSION_FORMAT_DEFLATE = "deflate";
+    public static final String COMPRESSION_FORMAT_BZIP2 = "bzip2";
+    public static final String COMPRESSION_FORMAT_XZ_LZMA2 = "xz-lzma2";
+    public static final String COMPRESSION_FORMAT_LZMA = "lzma";
+    public static final String COMPRESSION_FORMAT_SNAPPY = "snappy";
+    public static final String COMPRESSION_FORMAT_SNAPPY_HADOOP = "snappy-hadoop";
+    public static final String COMPRESSION_FORMAT_SNAPPY_FRAMED = "snappy framed";
+    public static final String COMPRESSION_FORMAT_LZ4_FRAMED = "lz4-framed";
+    public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
+    public static final String COMPRESSION_FORMAT_BROTLI = "brotli";
+
+    public static final PropertyDescriptor DECOMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Decompression Format")
+            .description("The decompression format to use for input FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Framed, and LZ4-Framed")
+            .allowableValues(DECOMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(DECOMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+    public static final PropertyDescriptor COMPRESSION_FORMAT = new PropertyDescriptor.Builder()
+            .name("Compression Format")
+            .description("The compression format to use for output FlowFiles. Valid values are: None, GZIP, Deflate, ZSTD, BZIP2, XZ-LZMA2, LZMA, Brotli, Snappy, Snappy Hadoop, "
+                    + "Snappy Framed, and LZ4-Framed")
+            .allowableValues(COMPRESSION_FORMAT_NONE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE, COMPRESSION_FORMAT_BZIP2,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_LZMA, COMPRESSION_FORMAT_SNAPPY, COMPRESSION_FORMAT_SNAPPY_HADOOP, COMPRESSION_FORMAT_SNAPPY_FRAMED,
+                    COMPRESSION_FORMAT_LZ4_FRAMED, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .defaultValue(COMPRESSION_FORMAT_NONE)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(COMPRESSION_FORMAT, COMPRESSION_FORMAT_ATTRIBUTE, COMPRESSION_FORMAT_GZIP, COMPRESSION_FORMAT_DEFLATE,
+                    COMPRESSION_FORMAT_XZ_LZMA2, COMPRESSION_FORMAT_ZSTD, COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DECOMPRESSION_FORMAT);
+        properties.add(COMPRESSION_FORMAT);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        mimeTypeMap.put("application/gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/x-gzip", COMPRESSION_FORMAT_GZIP);
+        mimeTypeMap.put("application/deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/x-deflate", COMPRESSION_FORMAT_DEFLATE);
+        mimeTypeMap.put("application/bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-bzip2", COMPRESSION_FORMAT_BZIP2);
+        mimeTypeMap.put("application/x-lzma", COMPRESSION_FORMAT_LZMA);
+        mimeTypeMap.put("application/x-snappy", COMPRESSION_FORMAT_SNAPPY);
+        mimeTypeMap.put("application/x-snappy-hadoop", COMPRESSION_FORMAT_SNAPPY_HADOOP);
+        mimeTypeMap.put("application/x-snappy-framed", COMPRESSION_FORMAT_SNAPPY_FRAMED);
+        mimeTypeMap.put("application/x-lz4-framed", COMPRESSION_FORMAT_LZ4_FRAMED);
+        mimeTypeMap.put("application/zstd", COMPRESSION_FORMAT_ZSTD);
+        mimeTypeMap.put("application/x-brotli", COMPRESSION_FORMAT_BROTLI);
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final long sizeBeforeCompression = flowFile.getSize();
+
+        String decompressionFormatValue = context.getProperty(DECOMPRESSION_FORMAT).getValue();
+        if (decompressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            decompressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (decompressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Decompression Format; routing to failure", flowFile, mimeType);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+        }
+
+        String compressionFormatValue = context.getProperty(COMPRESSION_FORMAT).getValue();
+        if (compressionFormatValue.equals(COMPRESSION_FORMAT_ATTRIBUTE)) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            compressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (compressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Compression Format; routing to success without decompressing", flowFile, mimeType);
+                session.transfer(flowFile, REL_SUCCESS);
+                return;
+            }
+        }
+
+        final String decompressionFormat = decompressionFormatValue;
+        final String compressionFormat = compressionFormatValue;
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
+        final StopWatch stopWatch = new StopWatch(true);
+
+        final String inputFileExtension = getFileExtension(decompressionFormat);
+        final String outputFileExtension = getFileExtension(compressionFormat);
+
+        try {
+            flowFile = session.write(flowFile, (rawIn, rawOut) -> {

Review Comment:
   With the callback method being rather large, recommend refactoring it to a separate inner class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #7180: NIFI-11466: Add ModifyCompression processor

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on PR #7180:
URL: https://github.com/apache/nifi/pull/7180#issuecomment-1528829005

   Thanks for the confirmation on the changes @mattyb149, will plan on merging with the `Co-authored-by` line soon.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] mattyb149 commented on a diff in pull request #7180: NIFI-11466: Add ModifyCompression processor

Posted by "mattyb149 (via GitHub)" <gi...@apache.org>.
mattyb149 commented on code in PR #7180:
URL: https://github.com/apache/nifi/pull/7180#discussion_r1175907945


##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/ModifyCompression.java:
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.compress;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    private final static int STREAM_BUFFER_SIZE = 65536;
+
+    public static final PropertyDescriptor INPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("input-compression-format")
+            .displayName("Input Compression Format")
+            .description("The format to use for decompressing input FlowFiles.")
+            .allowableValues(CompressionInfo.DECOMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.DECOMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+    public static final PropertyDescriptor OUTPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("output-compression-format")
+            .name("Output Compression Format")
+            .description("The format to use for compressing output FlowFiles.")
+            .allowableValues(CompressionInfo.COMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.COMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(OUTPUT_COMPRESSION,
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE,
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP,
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE,
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2,
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD,
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(INPUT_COMPRESSION);
+        properties.add(OUTPUT_COMPRESSION);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        for(CompressionInfo compressionInfo : CompressionInfo.values()) {
+            String[] mimeTypes = compressionInfo.getMimeTypes();
+            if (mimeTypes == null) {
+                continue;
+            }
+            for(String mimeType : mimeTypes) {
+                mimeTypeMap.put(mimeType, compressionInfo.getValue());
+            }
+        }
+
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);

Review Comment:
   We still need to have entries in a map or otherwise for the multiple possible MIME types for a single CompressionInfo, I moved this to a static initializer so it's only done once.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory closed pull request #7180: NIFI-11466: Add ModifyCompression processor

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #7180: NIFI-11466: Add ModifyCompression processor
URL: https://github.com/apache/nifi/pull/7180


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7180: NIFI-11466: Add ModifyCompression processor

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7180:
URL: https://github.com/apache/nifi/pull/7180#discussion_r1175244598


##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-nar/pom.xml:
##########
@@ -0,0 +1,36 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->

Review Comment:
   Although this passes the license check, generally the header comes before the `<project>` element and after the XML declaration `<?xml version="1.0" encoding="UTF-8"?>`.



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-nar/src/main/resources/META-INF/LICENSE:
##########
@@ -0,0 +1,551 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+
+This product bundles 'SLF4J-API 1.7.21' which is available under an MIT license.
+
+    Copyright (c) 2004-2007 QOS.ch
+    All rights reserved.
+
+    Permission is hereby granted, free  of charge, to any person obtaining
+    a  copy  of this  software  and  associated  documentation files  (the
+    "Software"), to  deal in  the Software without  restriction, including
+    without limitation  the rights to  use, copy, modify,  merge, publish,
+    distribute,  sublicense, and/or sell  copies of  the Software,  and to
+    permit persons to whom the Software  is furnished to do so, subject to
+    the following conditions:
+
+    The  above  copyright  notice  and  this permission  notice  shall  be
+    included in all copies or substantial portions of the Software.
+
+    THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+    EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+    MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'Jakarta Activation' 'Jakarta Activation API' under the Eclipse Distribution License 1.0.
+
+    Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors.
+
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or
+    without modification, are permitted provided that the following
+    conditions are met:
+
+    - Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+
+    - Redistributions in binary form must reproduce the above
+      copyright notice, this list of conditions and the following
+      disclaimer in the documentation and/or other materials provided
+      with the distribution.
+
+    - Neither the name of the Eclipse Foundation, Inc. nor the
+      names of its contributors may be used to endorse or promote
+      products derived from this software without specific prior
+      written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND
+    CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES,
+    INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+    OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+    CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+    SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+    NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+    LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+    CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+    STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+    ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+This product bundles 'Jakarta Mail' and 'Jakarta Mail API' under an Eclipse Public License 2.0.

Review Comment:
   Does this NAR include Jakarta Mail? If not, this line and the following license section can be removed.



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-nar/src/main/resources/META-INF/LICENSE:
##########
@@ -0,0 +1,551 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+
+This product bundles 'SLF4J-API 1.7.21' which is available under an MIT license.
+
+    Copyright (c) 2004-2007 QOS.ch
+    All rights reserved.
+
+    Permission is hereby granted, free  of charge, to any person obtaining
+    a  copy  of this  software  and  associated  documentation files  (the
+    "Software"), to  deal in  the Software without  restriction, including
+    without limitation  the rights to  use, copy, modify,  merge, publish,
+    distribute,  sublicense, and/or sell  copies of  the Software,  and to
+    permit persons to whom the Software  is furnished to do so, subject to
+    the following conditions:
+
+    The  above  copyright  notice  and  this permission  notice  shall  be
+    included in all copies or substantial portions of the Software.
+
+    THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+    EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+    MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+This product bundles 'Jakarta Activation' 'Jakarta Activation API' under the Eclipse Distribution License 1.0.

Review Comment:
   Is Jakarta Activation included?



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/ModifyCompression.java:
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.compress;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    private final static int STREAM_BUFFER_SIZE = 65536;
+
+    public static final PropertyDescriptor INPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("input-compression-format")
+            .displayName("Input Compression Format")
+            .description("The format to use for decompressing input FlowFiles.")
+            .allowableValues(CompressionInfo.DECOMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.DECOMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+    public static final PropertyDescriptor OUTPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("output-compression-format")
+            .name("Output Compression Format")
+            .description("The format to use for compressing output FlowFiles.")
+            .allowableValues(CompressionInfo.COMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.COMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(OUTPUT_COMPRESSION,
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE,
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP,
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE,
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2,
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD,
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(INPUT_COMPRESSION);
+        properties.add(OUTPUT_COMPRESSION);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        for(CompressionInfo compressionInfo : CompressionInfo.values()) {
+            String[] mimeTypes = compressionInfo.getMimeTypes();
+            if (mimeTypes == null) {
+                continue;
+            }
+            for(String mimeType : mimeTypes) {
+                mimeTypeMap.put(mimeType, compressionInfo.getValue());
+            }
+        }
+
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);

Review Comment:
   This map of compression format to MIME Type could be moved to a static initializer. Alternatively, it seems like an Output MIME Type property could be added to `CompressionInfo`, would that avoid the need for this Map?



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/ModifyCompression.java:
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.compress;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    private final static int STREAM_BUFFER_SIZE = 65536;
+
+    public static final PropertyDescriptor INPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("input-compression-format")
+            .displayName("Input Compression Format")
+            .description("The format to use for decompressing input FlowFiles.")
+            .allowableValues(CompressionInfo.DECOMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.DECOMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+    public static final PropertyDescriptor OUTPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("output-compression-format")
+            .name("Output Compression Format")
+            .description("The format to use for compressing output FlowFiles.")
+            .allowableValues(CompressionInfo.COMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.COMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(OUTPUT_COMPRESSION,
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE,
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP,
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE,
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2,
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD,
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(INPUT_COMPRESSION);
+        properties.add(OUTPUT_COMPRESSION);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        for(CompressionInfo compressionInfo : CompressionInfo.values()) {
+            String[] mimeTypes = compressionInfo.getMimeTypes();
+            if (mimeTypes == null) {
+                continue;
+            }
+            for(String mimeType : mimeTypes) {
+                mimeTypeMap.put(mimeType, compressionInfo.getValue());
+            }
+        }
+
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final long sizeBeforeCompression = flowFile.getSize();
+
+        String decompressionFormatValue = context.getProperty(INPUT_COMPRESSION).getValue();
+        if (decompressionFormatValue.equals(CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.getValue())) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            decompressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (decompressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Decompression Format; routing to failure", flowFile, mimeType);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+        }
+
+        String compressionFormatValue = context.getProperty(OUTPUT_COMPRESSION).getValue();
+        if (compressionFormatValue.equals(CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.getValue())) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            compressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (compressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Compression Format; routing to success without decompressing", flowFile, mimeType);
+                session.transfer(flowFile, REL_SUCCESS);
+                return;
+            }
+        }
+
+        final String decompressionFormat = decompressionFormatValue;
+        final String compressionFormat = compressionFormatValue;
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
+        final StopWatch stopWatch = new StopWatch(true);
+
+        String inputFileExtension;
+        String outputFileExtension;
+        try {
+            inputFileExtension = CompressionInfo.fromAllowableValue(decompressionFormat).getFileExtension();
+        } catch (IllegalArgumentException iae) {
+            inputFileExtension = "";
+        }
+        try {
+            outputFileExtension = CompressionInfo.fromAllowableValue(compressionFormat).getFileExtension();
+        } catch (IllegalArgumentException iae) {
+            outputFileExtension = "";
+        }
+
+        try {
+            flowFile = session.write(flowFile, (rawIn, rawOut) -> {
+                final OutputStream compressionOut;
+                final InputStream compressionIn;
+
+                final OutputStream bufferedOut = new BufferedOutputStream(rawOut, STREAM_BUFFER_SIZE);
+                final InputStream bufferedIn = new BufferedInputStream(rawIn, STREAM_BUFFER_SIZE);
+
+                try {
+                    // Decompress data by creating an InputStream for the decompression format, then recompress using the compression format
+                    compressionIn = getCompressionInputStream(decompressionFormat, bufferedIn);
+
+                    int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                    compressionOut = getCompressionOutputStream(compressionFormat, compressionLevel, mimeTypeRef, bufferedOut);
+
+                } catch (final Exception e) {
+                    closeQuietly(bufferedOut);
+                    throw new IOException(e);
+                }
+
+                StreamUtils.copy(compressionIn, compressionOut);
+            });
+            stopWatch.stop();
+
+            final long sizeAfterCompression = flowFile.getSize();
+
+            if (StringUtils.isEmpty(mimeTypeRef.get())) {
+                flowFile = session.removeAttribute(flowFile, CoreAttributes.MIME_TYPE.key());
+            } else {
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), mimeTypeRef.get());
+            }
+
+            if (context.getProperty(UPDATE_FILENAME).asBoolean()) {
+                String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
+                // Remove the input file extension if necessary
+                if (filename.toLowerCase().endsWith(inputFileExtension)) {
+                    filename = filename.substring(0, filename.length() - inputFileExtension.length());
+                }
+                flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), filename + outputFileExtension);
+            }
+
+            logger.info("Successfully recompressed {} using {} compression format; size changed from {} to {} bytes",
+                    flowFile, compressionFormat, sizeBeforeCompression, sizeAfterCompression);
+            session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getDuration(TimeUnit.MILLISECONDS));
+            session.transfer(flowFile, REL_SUCCESS);
+        } catch (final ProcessException e) {
+            logger.error("Unable to recompress {} using {} compression format due to {}; routing to failure", flowFile, compressionFormat, e);
+            session.transfer(flowFile, REL_FAILURE);
+        }
+    }
+
+    private void closeQuietly(final Closeable closeable) {
+        if (closeable != null) {
+            try {
+                closeable.close();
+            } catch (final Exception e) {
+                // Ignore
+            }
+        }
+    }

Review Comment:
   This can be removed and replaced with FileUtils.closeQuietly().



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/ModifyCompression.java:
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.compress;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    private final static int STREAM_BUFFER_SIZE = 65536;
+
+    public static final PropertyDescriptor INPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("input-compression-format")
+            .displayName("Input Compression Format")
+            .description("The format to use for decompressing input FlowFiles.")
+            .allowableValues(CompressionInfo.DECOMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.DECOMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+    public static final PropertyDescriptor OUTPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("output-compression-format")
+            .name("Output Compression Format")
+            .description("The format to use for compressing output FlowFiles.")
+            .allowableValues(CompressionInfo.COMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.COMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(OUTPUT_COMPRESSION,
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE,
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP,
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE,
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2,
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD,
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(INPUT_COMPRESSION);
+        properties.add(OUTPUT_COMPRESSION);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final Map<String, String> mimeTypeMap = new HashMap<>();
+        for(CompressionInfo compressionInfo : CompressionInfo.values()) {
+            String[] mimeTypes = compressionInfo.getMimeTypes();
+            if (mimeTypes == null) {
+                continue;
+            }
+            for(String mimeType : mimeTypes) {
+                mimeTypeMap.put(mimeType, compressionInfo.getValue());
+            }
+        }
+
+        this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final long sizeBeforeCompression = flowFile.getSize();
+
+        String decompressionFormatValue = context.getProperty(INPUT_COMPRESSION).getValue();
+        if (decompressionFormatValue.equals(CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.getValue())) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            decompressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (decompressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Decompression Format; routing to failure", flowFile, mimeType);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+        }
+
+        String compressionFormatValue = context.getProperty(OUTPUT_COMPRESSION).getValue();
+        if (compressionFormatValue.equals(CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.getValue())) {
+            final String mimeType = flowFile.getAttribute(CoreAttributes.MIME_TYPE.key());
+            if (mimeType == null) {
+                logger.error("No {} attribute exists for {}; routing to failure", CoreAttributes.MIME_TYPE.key(), flowFile);
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            compressionFormatValue = compressionFormatMimeTypeMap.get(mimeType);
+            if (compressionFormatValue == null) {
+                logger.info("MIME Type of {} is '{}', which does not indicate a supported Compression Format; routing to success without decompressing", flowFile, mimeType);
+                session.transfer(flowFile, REL_SUCCESS);
+                return;
+            }
+        }
+
+        final String decompressionFormat = decompressionFormatValue;
+        final String compressionFormat = compressionFormatValue;
+        final AtomicReference<String> mimeTypeRef = new AtomicReference<>(null);
+        final StopWatch stopWatch = new StopWatch(true);
+
+        String inputFileExtension;
+        String outputFileExtension;
+        try {
+            inputFileExtension = CompressionInfo.fromAllowableValue(decompressionFormat).getFileExtension();
+        } catch (IllegalArgumentException iae) {
+            inputFileExtension = "";
+        }
+        try {
+            outputFileExtension = CompressionInfo.fromAllowableValue(compressionFormat).getFileExtension();
+        } catch (IllegalArgumentException iae) {
+            outputFileExtension = "";
+        }
+
+        try {
+            flowFile = session.write(flowFile, (rawIn, rawOut) -> {
+                final OutputStream compressionOut;
+                final InputStream compressionIn;
+
+                final OutputStream bufferedOut = new BufferedOutputStream(rawOut, STREAM_BUFFER_SIZE);
+                final InputStream bufferedIn = new BufferedInputStream(rawIn, STREAM_BUFFER_SIZE);
+
+                try {
+                    // Decompress data by creating an InputStream for the decompression format, then recompress using the compression format
+                    compressionIn = getCompressionInputStream(decompressionFormat, bufferedIn);
+
+                    int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger();
+                    compressionOut = getCompressionOutputStream(compressionFormat, compressionLevel, mimeTypeRef, bufferedOut);
+
+                } catch (final Exception e) {
+                    closeQuietly(bufferedOut);
+                    throw new IOException(e);

Review Comment:
   A message should be added to the exception.
   ```suggestion
                       throw new IOException(String.format("Input Type [%s] Output Type [%s] Compression operation failed", decompressionFormat, compressionFormat), e);
   ```



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/test/java/org/apache/nifi/processors/compress/TestModifyCompression.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * 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.compress;
+
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+
+class TestModifyCompression {
+
+    @Test
+    public void testSnappyCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);

Review Comment:
   Recommend moving this line to a `setRunner()` method annotated with `@BeforeEach` to reduce duplication across the test methods.



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/test/java/org/apache/nifi/processors/compress/TestModifyCompression.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * 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.compress;
+
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+
+class TestModifyCompression {
+
+    @Test
+    public void testSnappyCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/x-snappy");
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.snappy");
+    }
+
+    @Test
+    public void testSnappyDecompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.snappy"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+    }
+
+    @Test
+    public void testSnappyHadoopCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/x-snappy-hadoop");
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.snappy");
+    }
+
+    @Test
+    public void testSnappyHadoopDecompress() {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testSnappyFramedCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/x-snappy-framed");
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.sz");
+    }
+
+    @Test
+    public void testSnappyFramedDecompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.sz"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+    }
+
+    @Test
+    public void testBzip2DecompressConcatenated() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, "bzip2");

Review Comment:
   Instead of using the string values, the `CompressionInfo.getValue()` can be used in this and other methods.



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/ModifyCompression.java:
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.compress;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "
+        + "are generally fine to process")
+@ReadsAttribute(attribute = "mime.type", description = "If the Decompression Format is set to 'use mime.type attribute', this attribute is used to "
+        + "determine the decompression type. Otherwise, this attribute is ignored.")
+@WritesAttribute(attribute = "mime.type", description = "The appropriate MIME Type is set based on the value of the Compression Format property. If the Compression Format is 'no compression' this "
+        + "attribute is removed as the MIME Type is no longer known.")
+@SystemResourceConsideration(resource = SystemResource.CPU)
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class ModifyCompression extends AbstractProcessor {
+
+    private final static int STREAM_BUFFER_SIZE = 65536;
+
+    public static final PropertyDescriptor INPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("input-compression-format")
+            .displayName("Input Compression Format")
+            .description("The format to use for decompressing input FlowFiles.")
+            .allowableValues(CompressionInfo.DECOMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.DECOMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+    public static final PropertyDescriptor OUTPUT_COMPRESSION = new PropertyDescriptor.Builder()
+            .name("output-compression-format")
+            .name("Output Compression Format")
+            .description("The format to use for compressing output FlowFiles.")
+            .allowableValues(CompressionInfo.COMPRESSION_FORMAT_NONE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BZIP2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZMA.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD.asAllowableValue(),
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI.asAllowableValue())
+            .defaultValue(CompressionInfo.COMPRESSION_FORMAT_NONE.getValue())
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor COMPRESSION_LEVEL = new PropertyDescriptor.Builder()
+            .name("Compression Level")
+            .description("The compression level to use; this is valid only when using supported formats. A lower value results in faster processing "
+                    + "but less compression; a value of 0 indicates no (that is, simple archiving) for gzip or minimal for xz-lzma2 compression."
+                    + " Higher levels can mean much larger memory usage such as the case with levels 7-9 for xz-lzma/2 so be careful relative to heap size.")
+            .defaultValue("1")
+            .required(true)
+            .allowableValues("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
+            .dependsOn(OUTPUT_COMPRESSION,
+                    CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE,
+                    CompressionInfo.COMPRESSION_FORMAT_GZIP,
+                    CompressionInfo.COMPRESSION_FORMAT_DEFLATE,
+                    CompressionInfo.COMPRESSION_FORMAT_XZ_LZMA2,
+                    CompressionInfo.COMPRESSION_FORMAT_ZSTD,
+                    CompressionInfo.COMPRESSION_FORMAT_BROTLI)
+            .build();
+
+    public static final PropertyDescriptor UPDATE_FILENAME = new PropertyDescriptor.Builder()
+            .name("Update Filename")
+            .description("If true, will remove the filename extension when decompressing data (only if the extension indicates the appropriate "
+                    + "compression format) and add the appropriate extension when compressing data")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("FlowFiles will be transferred to the success relationship after successfully being compressed or decompressed")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("FlowFiles will be transferred to the failure relationship if they fail to compress/decompress")
+            .build();
+
+    private List<PropertyDescriptor> properties;
+    private Set<Relationship> relationships;
+    private Map<String, String> compressionFormatMimeTypeMap;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(INPUT_COMPRESSION);
+        properties.add(OUTPUT_COMPRESSION);
+        properties.add(COMPRESSION_LEVEL);
+        properties.add(UPDATE_FILENAME);
+        this.properties = Collections.unmodifiableList(properties);
+
+        final Set<Relationship> relationships = new HashSet<>();
+        relationships.add(REL_SUCCESS);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);

Review Comment:
   The property and relationship collections can be declared statically.



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/util/CompressionInfo.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.compress.util;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.DescribedValue;
+
+public enum CompressionInfo implements DescribedValue {
+
+    DECOMPRESSION_FORMAT_NONE("no decompression", "no decompression", "Do not decompress the input content", ""),
+    COMPRESSION_FORMAT_NONE("no compression", "no compression", "Do not compress the output content", ""),
+    COMPRESSION_FORMAT_ATTRIBUTE("use mime.type attribute", "use mime.type attribute", "TBD", ""),
+    COMPRESSION_FORMAT_GZIP("gzip", "gzip", "TBD", ".gz","application/gzip", "application/x-gzip"),
+    COMPRESSION_FORMAT_DEFLATE("deflate", "deflate", "TBD", ".zlib","application/deflate", "application/x-deflate"),
+    COMPRESSION_FORMAT_BZIP2("bzip2", "bzip2", "TBD", ".bz2","application/x-bzip2", "application/bzip2"),
+    COMPRESSION_FORMAT_XZ_LZMA2("xz-lzma2", "xz-lzma2", "TBD", ".xz","application/x-lzma"),
+    COMPRESSION_FORMAT_LZMA("lzma", "lzma", "TBD", ".lzma","application/x-lzma"),
+    COMPRESSION_FORMAT_SNAPPY("snappy", "snappy", "TBD", ".snappy","application/x-snappy"),
+    COMPRESSION_FORMAT_SNAPPY_HADOOP("snappy-hadoop", "snappy-hadoop", "TBD", ".snappy","application/x-snappy-hadoop"),
+    COMPRESSION_FORMAT_SNAPPY_FRAMED("snappy framed", "snappy framed", "TBD", ".sz","application/x-snappy-framed"),
+    COMPRESSION_FORMAT_LZ4_FRAMED("lz4-framed", "lz4-framed", "TBD", ".lz4","application/x-lz4-framed"),
+    COMPRESSION_FORMAT_ZSTD("zstd", "zstd", "TBD", ".zst","application/zstd"),
+    COMPRESSION_FORMAT_BROTLI("brotli", "brotli", "TBD", ".br","application/x-brotli");

Review Comment:
   The description is `TBD` for these elements.



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/test/java/org/apache/nifi/processors/compress/TestModifyCompression.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * 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.compress;
+
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+
+class TestModifyCompression {
+
+    @Test
+    public void testSnappyCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/x-snappy");
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.snappy");
+    }
+
+    @Test
+    public void testSnappyDecompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.snappy"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+    }
+
+    @Test
+    public void testSnappyHadoopCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/x-snappy-hadoop");
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.snappy");
+    }
+
+    @Test
+    public void testSnappyHadoopDecompress() {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_HADOOP.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testSnappyFramedCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), "application/x-snappy-framed");
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.sz");
+    }
+
+    @Test
+    public void testSnappyFramedDecompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_SNAPPY_FRAMED.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.sz"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+    }
+
+    @Test
+    public void testBzip2DecompressConcatenated() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, "bzip2");
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "false");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFileConcat.txt.bz2"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFileConcat.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFileConcat.txt.bz2"); // not updating filename
+    }
+
+    @Test
+    public void testBzip2DecompressLz4FramedCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, "bzip2");
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_LZ4_FRAMED.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.bz2"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.lz4");
+
+        runner.clearTransferState();
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile1.txt.bz2"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals("filename", "SampleFile1.txt.lz4");
+    }
+
+    @Test
+    public void testProperMimeTypeFromBzip2() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, "bzip2");
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "false");
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals("mime.type", "application/x-bzip2");
+    }
+
+    @Test
+    public void testBzip2DecompressWithBothMimeTypes() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.getValue());
+        runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true");
+
+        // ensure that we can decompress with a mime type of application/x-bzip2
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("mime.type", "application/x-bzip2");
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.bz2"), attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+
+        // ensure that we can decompress with a mime type of application/bzip2. The appropriate mime type is
+        // application/x-bzip2, but we used to use application/bzip2. We want to ensure that we are still
+        // backward compatible.
+        runner.clearTransferState();
+        attributes.put("mime.type", "application/bzip2");
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile1.txt.bz2"), attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile1.txt");
+    }
+
+
+    @Test
+    public void testGzipDecompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, "gzip");
+        Assertions.assertTrue(runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true").isValid());
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.gz"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+
+        runner.clearTransferState();
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile1.txt.gz"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile1.txt");
+
+        runner.clearTransferState();
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, CompressionInfo.COMPRESSION_FORMAT_ATTRIBUTE.getValue());
+        Map<String, String> attributes = new HashMap<>();
+        attributes.put(CoreAttributes.MIME_TYPE.key(), "application/x-gzip");
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.gz"), attributes);
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+    }
+
+
+    @Test
+    public void testDeflateDecompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, "deflate");
+        Assertions.assertTrue(runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true").isValid());
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt.zlib"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        System.err.println(new String(flowFile.toByteArray()));
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt");
+    }
+
+
+    @Test
+    public void testDeflateCompress() throws Exception {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.COMPRESSION_LEVEL, "6");
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, "deflate");
+        Assertions.assertTrue(runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true").isValid());
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertContentEquals(Paths.get("src/test/resources/CompressedData/SampleFile.txt.zlib"));
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.zlib");
+    }
+
+    @Test
+    public void testFilenameUpdatedOnCompress() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+
+        runner.setProperty(ModifyCompression.OUTPUT_COMPRESSION, "gzip");
+        Assertions.assertTrue(runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true").isValid());
+
+        runner.enqueue(Paths.get("src/test/resources/CompressedData/SampleFile.txt"));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ModifyCompression.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ModifyCompression.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals("filename", "SampleFile.txt.gz");
+
+    }
+
+    @Test
+    public void testDecompressFailure() throws IOException {
+        final TestRunner runner = TestRunners.newTestRunner(ModifyCompression.class);
+        runner.setProperty(ModifyCompression.INPUT_COMPRESSION, "gzip");
+
+        byte[] data = new byte[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
+        runner.enqueue(data);
+
+        Assertions.assertTrue(runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true").isValid());

Review Comment:
   ```suggestion
           assertTrue(runner.setProperty(ModifyCompression.UPDATE_FILENAME, "true").isValid());
   ```



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/util/CompressionInfo.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.compress.util;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.DescribedValue;
+
+public enum CompressionInfo implements DescribedValue {
+
+    DECOMPRESSION_FORMAT_NONE("no decompression", "no decompression", "Do not decompress the input content", ""),
+    COMPRESSION_FORMAT_NONE("no compression", "no compression", "Do not compress the output content", ""),
+    COMPRESSION_FORMAT_ATTRIBUTE("use mime.type attribute", "use mime.type attribute", "TBD", ""),
+    COMPRESSION_FORMAT_GZIP("gzip", "gzip", "TBD", ".gz","application/gzip", "application/x-gzip"),
+    COMPRESSION_FORMAT_DEFLATE("deflate", "deflate", "TBD", ".zlib","application/deflate", "application/x-deflate"),
+    COMPRESSION_FORMAT_BZIP2("bzip2", "bzip2", "TBD", ".bz2","application/x-bzip2", "application/bzip2"),
+    COMPRESSION_FORMAT_XZ_LZMA2("xz-lzma2", "xz-lzma2", "TBD", ".xz","application/x-lzma"),
+    COMPRESSION_FORMAT_LZMA("lzma", "lzma", "TBD", ".lzma","application/x-lzma"),
+    COMPRESSION_FORMAT_SNAPPY("snappy", "snappy", "TBD", ".snappy","application/x-snappy"),
+    COMPRESSION_FORMAT_SNAPPY_HADOOP("snappy-hadoop", "snappy-hadoop", "TBD", ".snappy","application/x-snappy-hadoop"),
+    COMPRESSION_FORMAT_SNAPPY_FRAMED("snappy framed", "snappy framed", "TBD", ".sz","application/x-snappy-framed"),
+    COMPRESSION_FORMAT_LZ4_FRAMED("lz4-framed", "lz4-framed", "TBD", ".lz4","application/x-lz4-framed"),
+    COMPRESSION_FORMAT_ZSTD("zstd", "zstd", "TBD", ".zst","application/zstd"),
+    COMPRESSION_FORMAT_BROTLI("brotli", "brotli", "TBD", ".br","application/x-brotli");
+
+
+    private final String displayName;
+    private final String description;
+    private final String value;
+    private final String fileExtension;
+    private final String[] mimeTypes;
+
+    public static CompressionInfo fromAllowableValue(String allowableValue) {
+        for (CompressionInfo compressionInfo : CompressionInfo.values()) {
+            if (compressionInfo.getValue().equalsIgnoreCase(allowableValue)) {
+                return compressionInfo;
+            }
+        }
+        return null;
+    }
+
+    CompressionInfo(final String displayName, final String value, final String description, final String fileExtension, final String... mimeTypes) {

Review Comment:
   It looks like the `displayName` and `value` are always the same. If that is the case, recommend removing `displayName` and returning `value` for `getDisplayName`.



##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-processors/src/main/java/org/apache/nifi/processors/compress/ModifyCompression.java:
##########
@@ -0,0 +1,420 @@
+/*
+ * 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.compress;
+
+import com.aayushatharva.brotli4j.Brotli4jLoader;
+import com.aayushatharva.brotli4j.decoder.BrotliInputStream;
+import com.aayushatharva.brotli4j.encoder.BrotliOutputStream;
+import com.aayushatharva.brotli4j.encoder.Encoder;
+import lzma.sdk.lzma.Decoder;
+import lzma.streams.LzmaInputStream;
+import lzma.streams.LzmaOutputStream;
+import org.apache.commons.compress.compressors.CompressorException;
+import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
+import org.apache.commons.compress.compressors.lz4.FramedLZ4CompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorInputStream;
+import org.apache.commons.compress.compressors.zstandard.ZstdCompressorOutputStream;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.compress.util.CompressionInfo;
+import org.apache.nifi.stream.io.GZIPOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.StopWatch;
+import org.apache.nifi.util.StringUtils;
+import org.tukaani.xz.LZMA2Options;
+import org.tukaani.xz.XZInputStream;
+import org.tukaani.xz.XZOutputStream;
+import org.xerial.snappy.SnappyFramedInputStream;
+import org.xerial.snappy.SnappyFramedOutputStream;
+import org.xerial.snappy.SnappyHadoopCompatibleOutputStream;
+import org.xerial.snappy.SnappyInputStream;
+import org.xerial.snappy.SnappyOutputStream;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+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.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.zip.Deflater;
+import java.util.zip.DeflaterOutputStream;
+import java.util.zip.InflaterInputStream;
+
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"content", "compress", "recompress", "gzip", "bzip2", "lzma", "xz-lzma2", "snappy", "snappy-hadoop", "snappy framed", "lz4-framed", "deflate", "zstd", "brotli"})
+@CapabilityDescription("Decompresses the contents of FlowFiles using a user-specified compression algorithm and recompresses the contents using the specified compression format properties. "
+        + "Also updates the mime.type attribute as appropriate. This processor operates in a very memory efficient way so very large objects well beyond the heap size "

Review Comment:
   The mention of `mime.type` can be removed since it is documented in `WritesAttributes`.
   ```suggestion
           + "This processor operates in a very memory efficient way so very large objects well beyond the heap size "
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7180: NIFI-11466: Add ModifyCompression processor

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #7180:
URL: https://github.com/apache/nifi/pull/7180#discussion_r1177058260


##########
nifi-nar-bundles/nifi-compress-bundle/nifi-compress-nar/src/main/resources/META-INF/LICENSE:
##########
@@ -0,0 +1,234 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+
+This product bundles 'SLF4J-API 1.7.21' which is available under an MIT license.
+
+    Copyright (c) 2004-2007 QOS.ch
+    All rights reserved.
+
+    Permission is hereby granted, free  of charge, to any person obtaining
+    a  copy  of this  software  and  associated  documentation files  (the
+    "Software"), to  deal in  the Software without  restriction, including
+    without limitation  the rights to  use, copy, modify,  merge, publish,
+    distribute,  sublicense, and/or sell  copies of  the Software,  and to
+    permit persons to whom the Software  is furnished to do so, subject to
+    the following conditions:
+
+    The  above  copyright  notice  and  this permission  notice  shall  be
+    included in all copies or substantial portions of the Software.
+
+    THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+    EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+    MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+    NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+    LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+    OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+    WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.

Review Comment:
   It looks like this section can also be removed, since only the assembly should include the SLF4J API, not individual NAR files.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] mattyb149 commented on pull request #7180: NIFI-11466: Add ModifyCompression processor

Posted by "mattyb149 (via GitHub)" <gi...@apache.org>.
mattyb149 commented on PR #7180:
URL: https://github.com/apache/nifi/pull/7180#issuecomment-1528810986

   +1 LGTM, thanks for the assist! Do you mind merging this after adding yourself as a co-author? Want to make sure you get credit even after the commits are squashed.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org