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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to