You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "exceptionfactory (via GitHub)" <gi...@apache.org> on 2023/03/24 23:56:27 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

exceptionfactory commented on code in PR #6769:
URL: https://github.com/apache/nifi/pull/6769#discussion_r1148144597


##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -134,10 +135,35 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
         .required(false)
         .build();
 
+    private static final PropertyDescriptor DO_ADDITIONAL_PREPROCESSING = new PropertyDescriptor.Builder()
+        .name("do-additional-preprocessing")
+        .displayName("Do Additional Preprocessing")
+        .description("When set, NiFi will do additional preprocessing steps that creates modified versions of the provided ASN files," +
+                " removing unsupported features in a way that makes them less strict but otherwise should still be compatible with incoming data." +
+                " The original files will remain intact and new ones will be created with the same names in the directory defined in the 'Additional Preprocessing Output Directory' property." +
+                " For more information about these additional preprocessing steps please see Additional Details - Additional Preprocessing.")
+        .allowableValues("true", "false")
+        .required(true)
+        .defaultValue("false")
+        .build();
+
+    private static final PropertyDescriptor ADDITIONAL_PREPROCESSING_OUTPUT_DIRECTORY = new PropertyDescriptor.Builder()
+        .name("additional-preprocessing-output-directory")
+        .displayName("Additional Preprocessing Output Directory")

Review Comment:
   Following the above naming suggestion, recommend renaming this property to `Schema Preparation Directory`.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/preprocess/AsnPreprocessorEngine.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.jasn1.preprocess;
+
+import org.apache.nifi.jasn1.preprocess.preprocessors.ConstraintAsnPreprocessor;
+import org.apache.nifi.jasn1.preprocess.preprocessors.HuggingCommentAsnPreprocessor;
+import org.apache.nifi.jasn1.preprocess.preprocessors.VersionBracketAsnPreprocessor;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+
+public class AsnPreprocessorEngine {
+    public static final String COMMA = "\\s*,\\s*";
+
+    private static final List<AsnPreprocessor> PREPROCESSORS = Arrays.asList(
+            new HuggingCommentAsnPreprocessor(),
+            new VersionBracketAsnPreprocessor(),
+            new ConstraintAsnPreprocessor()
+    );
+
+    public String preprocess(
+            ComponentLog componentLog,
+            String asnFilesString,
+            String outputDirectory
+    ) {
+        final String[] inputFiles = asnFilesString.split(COMMA);
+
+        final StringJoiner preprocessedInputFiles = new StringJoiner(",");
+
+        for (String inputFile : inputFiles) {
+            final Path inputFilePath = Paths.get(inputFile);
+            final Path fileName = inputFilePath.getFileName();
+
+            final List<String> lines = readAsnLines(componentLog, inputFile, inputFilePath);
+
+            final List<String> preprocessedLines = preprocessAsn(lines);
+
+            final String preprocessedAsn = preprocessedLines
+                    .stream()
+                    .collect(Collectors.joining(System.lineSeparator()));
+
+            final Path preprocessedAsnPath = Paths.get(outputDirectory, fileName.toString());
+            preprocessedInputFiles.add(preprocessedAsnPath.toString());
+
+            writePreprocessedAsn(componentLog, preprocessedAsn, preprocessedAsnPath);
+        }
+
+        return preprocessedInputFiles.toString();
+    }
+
+    List<String> preprocessAsn(List<String> lines) {
+        List<String> preprocessedAsn = lines;
+
+        for (AsnPreprocessor preprocessor : getPreprocessors()) {
+            preprocessedAsn = preprocessor.preprocessAsn(preprocessedAsn);
+        }
+
+        return preprocessedAsn;
+    }
+
+    List<String> readAsnLines(ComponentLog componentLog, String inputFile, Path inputFilePath) {
+        List<String> lines;
+        try {
+            lines = Files.readAllLines(inputFilePath);
+        } catch (IOException e) {
+            componentLog.error("Couldn't read {}", inputFile, e);

Review Comment:
   It is best to avoid contractions in log messages, and it would also be helpful to provide more context to the message:
   ```suggestion
               componentLog.error("Read ASN.1 Schema failed [{}]", inputFile, e);
   ```



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/preprocess/AsnPreprocessorEngine.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.jasn1.preprocess;
+
+import org.apache.nifi.jasn1.preprocess.preprocessors.ConstraintAsnPreprocessor;
+import org.apache.nifi.jasn1.preprocess.preprocessors.HuggingCommentAsnPreprocessor;
+import org.apache.nifi.jasn1.preprocess.preprocessors.VersionBracketAsnPreprocessor;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+
+public class AsnPreprocessorEngine {
+    public static final String COMMA = "\\s*,\\s*";
+
+    private static final List<AsnPreprocessor> PREPROCESSORS = Arrays.asList(
+            new HuggingCommentAsnPreprocessor(),
+            new VersionBracketAsnPreprocessor(),
+            new ConstraintAsnPreprocessor()
+    );
+
+    public String preprocess(
+            ComponentLog componentLog,
+            String asnFilesString,
+            String outputDirectory
+    ) {
+        final String[] inputFiles = asnFilesString.split(COMMA);
+
+        final StringJoiner preprocessedInputFiles = new StringJoiner(",");
+
+        for (String inputFile : inputFiles) {
+            final Path inputFilePath = Paths.get(inputFile);
+            final Path fileName = inputFilePath.getFileName();
+
+            final List<String> lines = readAsnLines(componentLog, inputFile, inputFilePath);
+
+            final List<String> preprocessedLines = preprocessAsn(lines);
+
+            final String preprocessedAsn = preprocessedLines
+                    .stream()
+                    .collect(Collectors.joining(System.lineSeparator()));
+
+            final Path preprocessedAsnPath = Paths.get(outputDirectory, fileName.toString());
+            preprocessedInputFiles.add(preprocessedAsnPath.toString());
+
+            writePreprocessedAsn(componentLog, preprocessedAsn, preprocessedAsnPath);
+        }
+
+        return preprocessedInputFiles.toString();
+    }
+
+    List<String> preprocessAsn(List<String> lines) {
+        List<String> preprocessedAsn = lines;
+
+        for (AsnPreprocessor preprocessor : getPreprocessors()) {
+            preprocessedAsn = preprocessor.preprocessAsn(preprocessedAsn);
+        }
+
+        return preprocessedAsn;
+    }
+
+    List<String> readAsnLines(ComponentLog componentLog, String inputFile, Path inputFilePath) {
+        List<String> lines;
+        try {
+            lines = Files.readAllLines(inputFilePath);
+        } catch (IOException e) {
+            componentLog.error("Couldn't read {}", inputFile, e);
+            throw new UncheckedIOException(e);

Review Comment:
   Instead of logging the error, this exception should contain the message. The caller should catch and log the error.
   ```suggestion
               throw new UncheckedIOException(String.format("Read ASN.1 Schema failed [%s]", inputFile), e);
   ```



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/preprocess/AsnPreprocessorEngine.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.jasn1.preprocess;
+
+import org.apache.nifi.jasn1.preprocess.preprocessors.ConstraintAsnPreprocessor;
+import org.apache.nifi.jasn1.preprocess.preprocessors.HuggingCommentAsnPreprocessor;
+import org.apache.nifi.jasn1.preprocess.preprocessors.VersionBracketAsnPreprocessor;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.StringJoiner;
+import java.util.stream.Collectors;
+
+public class AsnPreprocessorEngine {
+    public static final String COMMA = "\\s*,\\s*";
+
+    private static final List<AsnPreprocessor> PREPROCESSORS = Arrays.asList(
+            new HuggingCommentAsnPreprocessor(),
+            new VersionBracketAsnPreprocessor(),
+            new ConstraintAsnPreprocessor()
+    );
+
+    public String preprocess(
+            ComponentLog componentLog,
+            String asnFilesString,
+            String outputDirectory
+    ) {
+        final String[] inputFiles = asnFilesString.split(COMMA);
+
+        final StringJoiner preprocessedInputFiles = new StringJoiner(",");
+
+        for (String inputFile : inputFiles) {
+            final Path inputFilePath = Paths.get(inputFile);
+            final Path fileName = inputFilePath.getFileName();
+
+            final List<String> lines = readAsnLines(componentLog, inputFile, inputFilePath);
+
+            final List<String> preprocessedLines = preprocessAsn(lines);
+
+            final String preprocessedAsn = preprocessedLines
+                    .stream()
+                    .collect(Collectors.joining(System.lineSeparator()));
+
+            final Path preprocessedAsnPath = Paths.get(outputDirectory, fileName.toString());
+            preprocessedInputFiles.add(preprocessedAsnPath.toString());
+
+            writePreprocessedAsn(componentLog, preprocessedAsn, preprocessedAsnPath);
+        }
+
+        return preprocessedInputFiles.toString();
+    }
+
+    List<String> preprocessAsn(List<String> lines) {
+        List<String> preprocessedAsn = lines;
+
+        for (AsnPreprocessor preprocessor : getPreprocessors()) {
+            preprocessedAsn = preprocessor.preprocessAsn(preprocessedAsn);
+        }
+
+        return preprocessedAsn;
+    }
+
+    List<String> readAsnLines(ComponentLog componentLog, String inputFile, Path inputFilePath) {
+        List<String> lines;
+        try {
+            lines = Files.readAllLines(inputFilePath);
+        } catch (IOException e) {
+            componentLog.error("Couldn't read {}", inputFile, e);
+            throw new UncheckedIOException(e);
+        }
+        return lines;
+    }
+
+    void writePreprocessedAsn(ComponentLog componentLog, String preprocessedAsn, Path preprocessedAsnPath) {
+        try {
+            Files.write(preprocessedAsnPath, preprocessedAsn.getBytes(StandardCharsets.UTF_8));
+        } catch (IOException e) {
+            componentLog.error("Couldn't write {}", preprocessedAsnPath.toString(), e);
+            throw new UncheckedIOException(e);

Review Comment:
   ```suggestion
               throw new UncheckedIOException(String.format("Write ASN.1 Schema failed [%s]", preprocessedAsnPath), e);
   ```



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -134,17 +135,32 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
         .required(false)
         .build();
 
+    private static final PropertyDescriptor PREPROCESS_OUTPUT_DIRECTORY = new PropertyDescriptor.Builder()
+        .name("additional-preprocesszing-output-directory")
+        .displayName("Additional Preprocessing Output Directory")
+        .description("When set, NiFi will do additional preprocessing steps that creates modified versions of the provided ASN files," +
+                " removing unsupported features in a way that makes them less strict but otherwise should still be compatible with incoming data." +
+                " The original files will remain intact and new ones will be created with the same names in the provided directory." +
+                " For more information about these additional preprocessing steps please see Additional Details - Additional Preprocessing.")
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)

Review Comment:
   Although flow-based Variable Registry support will be removed, environment variable-based access can be supported. However, this could also be configured using Parameter Contexts, without the need for environment variables. From that perspective, the current setting might be useful, but not necessary.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -193,10 +219,23 @@ protected Collection<ValidationResult> customValidate(ValidationContext validati
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) throws InitializationException {
         if (context.getProperty(ASN_FILES) != null && context.getProperty(ASN_FILES).isSet()) {
-            String[] asnFilesPaths = Arrays.stream(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue().split(","))
-                .map(String::trim)
-                .toArray(String[]::new);
+            String asnFilesString = context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue();

Review Comment:
   In this particular case, the variable can be re-assigned on line 229, so this is acceptable.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/preprocess/AsnPreprocessorEngineTest.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.jasn1.preprocess;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.StringJoiner;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class AsnPreprocessorEngineTest {
+    private AsnPreprocessorEngine testSubject;
+    private AsnPreprocessorEngine helper;
+
+    private AsnPreprocessor mockPreprocessor1;
+    private AsnPreprocessor mockPreprocessor2;
+    private List<AsnPreprocessor> preprocessors;
+
+    private ComponentLog log;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        mockPreprocessor1 = mock(AsnPreprocessor.class);
+        mockPreprocessor2 = mock(AsnPreprocessor.class);
+
+        preprocessors = Arrays.asList(
+                mockPreprocessor1,
+                mockPreprocessor2
+        );
+
+        log = mock(ComponentLog.class);
+
+        helper = mock(AsnPreprocessorEngine.class);
+        testSubject = new AsnPreprocessorEngine() {
+            @Override
+            List<String> readAsnLines(ComponentLog componentLog, String inputFile, Path inputFilePath) {
+                return helper.readAsnLines(componentLog, inputFile, inputFilePath);
+            }
+
+            @Override
+            void writePreprocessedAsn(ComponentLog componentLog, String preprocessedAsn, Path preprocessedAsnPath) {
+                helper.writePreprocessedAsn(componentLog, preprocessedAsn, preprocessedAsnPath);
+            }
+
+            @Override
+            List<AsnPreprocessor> getPreprocessors() {
+                return preprocessors;
+            }
+        };
+
+        Files.createDirectories(Path.of("target/" + this.getClass().getSimpleName()));
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        Files.walk(Path.of("target/" + this.getClass().getSimpleName()))
+                .sorted(Comparator.reverseOrder())
+                .map(Path::toFile)
+                .forEach(File::delete);
+    }
+
+    @Test
+    void testPreprocess() {
+        // GIVEN
+        Path asnFile1Path = Paths.get("path", "to", "asn_file_1");
+        Path asnFile2Path = Paths.get("path", "to", "asn_file_2");
+
+        String asnFilesString = new StringJoiner(",")
+                .add(asnFile1Path.toString())
+                .add(asnFile2Path.toString())
+                .toString();
+
+        String outputDirectory = Paths.get("path", "to", "directory_for_transformed_asn_files").toString();
+
+        List<String> originalLines1 = Arrays.asList("original_lines_1_1", "original_lines_1_2");
+        List<String> preprocessedLines1_1 = Arrays.asList("preprocessed_lines_1_1_1", "preprocessed_lines_1_1_2");
+        List<String> preprocessedLines1_2 = Arrays.asList("final_lines_1_1", "final_lines_1_2");
+
+        List<String> originalLines2 = Arrays.asList("original_lines_2_1", "original_lines_2_2");
+        List<String> preprocessedLines2_1 = Arrays.asList("preprocessed_lines_2_1_1", "preprocessed_lines_2_1_2");
+        List<String> preprocessedLines2_2 = Arrays.asList("final_lines_2_1", "final_lines_2_2");
+
+        when(helper.readAsnLines(eq(log), eq(asnFile1Path.toString()), eq(asnFile1Path)))
+                .thenReturn(originalLines1);
+        when(mockPreprocessor1.preprocessAsn(originalLines1)).thenReturn(preprocessedLines1_1);
+        when(mockPreprocessor2.preprocessAsn(preprocessedLines1_1)).thenReturn(preprocessedLines1_2);
+
+        when(helper.readAsnLines(eq(log), eq(asnFile2Path.toString()), eq(asnFile2Path)))
+                .thenReturn(originalLines2);
+        when(mockPreprocessor1.preprocessAsn(originalLines2)).thenReturn(preprocessedLines2_1);
+        when(mockPreprocessor2.preprocessAsn(preprocessedLines2_1)).thenReturn(preprocessedLines2_2);
+
+        String expected = new StringJoiner(",")
+                .add(Paths.get("path", "to", "directory_for_transformed_asn_files", "asn_file_1").toString())
+                .add(Paths.get("path", "to", "directory_for_transformed_asn_files", "asn_file_2").toString())
+                .toString();
+
+        // WHEN

Review Comment:
   The `GIVEN`/`WHEN`/`THEN` comments should be removed.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -134,10 +135,35 @@ public class JASN1Reader extends AbstractConfigurableComponent implements Record
         .required(false)
         .build();
 
+    private static final PropertyDescriptor DO_ADDITIONAL_PREPROCESSING = new PropertyDescriptor.Builder()
+        .name("do-additional-preprocessing")
+        .displayName("Do Additional Preprocessing")

Review Comment:
   This property name should be changed to something more descriptive. Starting with `Do` seems a bit too casual for a property name.
   
   Instead of having this as a Boolean property, what do you think about naming it `Schema Preparation Strategy`? With a default value of `DISABLED` and and allowable value of `PREPROCESSED`? This would help communicate what the property does and leave the option open for alternative strategies.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/preprocess/preprocessors/AbstractAsnPreprocessorTest.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.jasn1.preprocess.preprocessors;
+
+import org.apache.nifi.jasn1.preprocess.AsnPreprocessor;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public abstract class AbstractAsnPreprocessorTest {
+    protected AsnPreprocessor testSubject;
+
+    protected void testPreprocess(String input) throws IOException, URISyntaxException {
+        // GIVEN
+        List<String> lines = Files.readAllLines(Paths.get(getClass().getClassLoader().getResource(input).toURI()));
+
+        // WHEN
+        String actual = testSubject.preprocessAsn(lines)
+                .stream()
+                .collect(Collectors.joining(System.lineSeparator()));
+
+        // THEN

Review Comment:
   ```suggestion
           List<String> lines = Files.readAllLines(Paths.get(getClass().getClassLoader().getResource(input).toURI()));
   
           String actual = testSubject.preprocessAsn(lines)
                   .stream()
                   .collect(Collectors.joining(System.lineSeparator()));
   
   ```



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -247,7 +280,7 @@ private void compileAsnToClass(String... asnFilePaths) {
                 logger.error("ASN.1 file not found [{}]", asn1File, e);
                 parseException = e;
             } catch (TokenStreamException | RecognitionException e) {
-                logger.error("ASN.1 stream parsing failed [{}]", asn1File, e);
+                logger.error("ASN.1 stream parsing failed [{}] due to {}", asn1File, e.toString(), e);

Review Comment:
   The general convention of `due to` should be avoided. The stack trace includes the exception message, and that is carried through to Bulletins, so this change should be reverted.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/preprocess/preprocessors/ConstraintAsnPreprocessor.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.jasn1.preprocess.preprocessors;
+
+import org.apache.nifi.jasn1.preprocess.AsnPreprocessor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ConstraintAsnPreprocessor implements AsnPreprocessor {
+    public static final String OPEN_BRACKET = "(";
+    public static final String CLOSE_BRACKET = ")";
+
+    public static final Pattern ALLOWED = Pattern.compile("^(\\d+\\))(.*)");
+
+    @Override
+    public List<String> preprocessAsn(List<String> lines) {
+        final List<String> preprocessedLines = new ArrayList<>();
+
+        final AtomicInteger unclosedCounter = new AtomicInteger(0);
+        lines.forEach(line -> {
+            final StringBuilder preprocessedLine = new StringBuilder();
+
+            String contentToProcess = line;
+
+            while (contentToProcess.contains(OPEN_BRACKET) || contentToProcess.contains(CLOSE_BRACKET)) {
+                if (contentToProcess.matches("^\\s*--.*$")) {
+                    break;
+                }
+
+                final int openBracketIndex = contentToProcess.indexOf(OPEN_BRACKET);
+                final int closeBracketIndex = contentToProcess.indexOf(CLOSE_BRACKET);
+
+                if (openBracketIndex != -1 && (openBracketIndex < closeBracketIndex) || closeBracketIndex == -1) {
+                    final String contentBeforeOpenBracket = contentToProcess.substring(0, openBracketIndex);
+                    final String contentAfterOpenBracket = contentToProcess.substring(openBracketIndex + 1);
+
+                    if (unclosedCounter.get() < 1) {
+                        if (!contentBeforeOpenBracket.isEmpty()) {
+                            preprocessedLine.append(contentBeforeOpenBracket + " ");
+                            // Adding a space " " because (...) blocks can serve as separators so removing them might
+                            //  join together parts that should stay separated
+                        }
+
+                        final Matcher supportedMatcher = ALLOWED.matcher(contentAfterOpenBracket);
+                        if (supportedMatcher.matches()) {
+                            preprocessedLine.append("(" + supportedMatcher.group(1));

Review Comment:
   ```suggestion
                               preprocessedLine.append(OPEN_BRACKET + supportedMatcher.group(1));
   ```



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/preprocess/AsnPreprocessorEngineTest.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.jasn1.preprocess;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.StringJoiner;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class AsnPreprocessorEngineTest {
+    private AsnPreprocessorEngine testSubject;
+    private AsnPreprocessorEngine helper;
+
+    private AsnPreprocessor mockPreprocessor1;
+    private AsnPreprocessor mockPreprocessor2;
+    private List<AsnPreprocessor> preprocessors;
+
+    private ComponentLog log;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        mockPreprocessor1 = mock(AsnPreprocessor.class);
+        mockPreprocessor2 = mock(AsnPreprocessor.class);
+
+        preprocessors = Arrays.asList(
+                mockPreprocessor1,
+                mockPreprocessor2
+        );
+
+        log = mock(ComponentLog.class);
+
+        helper = mock(AsnPreprocessorEngine.class);
+        testSubject = new AsnPreprocessorEngine() {
+            @Override
+            List<String> readAsnLines(ComponentLog componentLog, String inputFile, Path inputFilePath) {
+                return helper.readAsnLines(componentLog, inputFile, inputFilePath);
+            }
+
+            @Override
+            void writePreprocessedAsn(ComponentLog componentLog, String preprocessedAsn, Path preprocessedAsnPath) {
+                helper.writePreprocessedAsn(componentLog, preprocessedAsn, preprocessedAsnPath);
+            }
+
+            @Override
+            List<AsnPreprocessor> getPreprocessors() {
+                return preprocessors;
+            }
+        };
+
+        Files.createDirectories(Path.of("target/" + this.getClass().getSimpleName()));

Review Comment:
   Agreed, the `target` directory should not be used for temporary test files. The `@TempDir` annotation is used in other tests and provides a better approach for cleaning up after test completion.



-- 
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