You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/12/07 18:21:38 UTC

[GitHub] [nifi] tpalfy opened a new pull request, #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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

   <!-- 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. -->
   
   # Summary
   
   [NIFI-10955](https://issues.apache.org/jira/browse/NIFI-10955)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] 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`
     - [ ] JDK 8
     - [ ] 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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/resources/docs/org.apache.nifi.jasn1.JASN1Reader/additionalDetails.html:
##########
@@ -122,5 +122,61 @@ <h3>Troubleshooting</h3>
                 </li>
             </ol>
         </p>
+
+        <h3>Additional Preprocessing</h3>
+
+        <p>
+            NiFi doesn't support every feature that the ASN standard allows. To alleviate problems when encountering ASN files with unsupported features,
+            NiFi can 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 a directory set in the 'Additional Preprocessing Output Directory' property.
+            Please note that this is a best-effort attempt. It is also strongly recommended to compare the resulting ASN files to the originals and make sure they are still appropriate.
+            <br />
+            <br />
+            The following modification are applied:
+            <ol>
+                <li>

Review Comment:
   I can do that.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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:
   I guess this comment is overruled by the next (meaning there shouldn't be any logging here at all).



-- 
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] nandorsoma commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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.NiFiASNPreprocessor;
+
+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 NiFiASNPreprocessor {
+    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) {
+        List<String> preprocessedLines = new ArrayList<>();
+
+        AtomicInteger unclosedCounter = new AtomicInteger(0);
+        lines.forEach(line -> {
+            StringBuilder preprocessedLine = new StringBuilder();
+
+            String contentToProcess = line;

Review Comment:
   Could you help me understand why?



##########
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:
   Did you consider using `@TempDir` feature provided by JUnit5? It is experimental, but already used in the project.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -366,13 +387,13 @@ public void reportError(RecognitionException e) {
             }
         };
 
+        AsnModel model = new AsnModel();
+        parser.module_definitions(model);
+
         if (parseError.get()) {

Review Comment:
   I see, makes sense!



##########
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:
   There are still a few lines that are missing finals.



##########
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:
   Any thoughts on this? It is still there but in the new property.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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()

Review Comment:
   I'll add a "switch" property.



-- 
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] nandorsoma commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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();
 
+            if (context.getProperty(DO_ADDITIONAL_PREPROCESSING).asBoolean()) {
+                AsnPreprocessorEngine asnPreprocessorEngine = new AsnPreprocessorEngine();

Review Comment:
   final is missing



##########
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();
 
+            if (context.getProperty(DO_ADDITIONAL_PREPROCESSING).asBoolean()) {
+                AsnPreprocessorEngine asnPreprocessorEngine = new AsnPreprocessorEngine();
+
+                String preprocessOutputDirectory = context.getProperty(ADDITIONAL_PREPROCESSING_OUTPUT_DIRECTORY).evaluateAttributeExpressions().getValue();

Review Comment:
   final is missing. Also in line 236, but I cannot comment on it.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/preprocessed_test_complex_for_preprocessing.asn:
##########
@@ -0,0 +1,15 @@
+ORG-APACHE-NIFI-JASN1-TEST
+
+DEFINITIONS IMPLICIT TAGS ::=
+
+BEGIN
+
+MyType := SEQUENCE {

Review Comment:
   An additional colon is missing.
   ```suggestion
   MyType ::= SEQUENCE {
   ```



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/resources/test_complex_for_preprocessing.asn:
##########
@@ -0,0 +1,15 @@
+ORG-APACHE-NIFI-JASN1-TEST
+
+DEFINITIONS IMPLICIT TAGS ::=
+
+BEGIN
+
+MyType := SEQUENCE {

Review Comment:
   An additional colon is missing.
   ```suggestion
   MyType ::= SEQUENCE {
   ```



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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)
+        .addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
+        .required(false)
+        .build();
+
     private final List<PropertyDescriptor> propertyDescriptors = Arrays.asList(
         ROOT_MODEL_NAME,
         ROOT_CLASS_NAME,
-        ASN_FILES
+        ASN_FILES,
+        PREPROCESS_OUTPUT_DIRECTORY
     );
 
     private String identifier;
     ComponentLog logger;
 
     private RecordSchemaProvider schemaProvider = new RecordSchemaProvider();
 
+    private NiFiASNPreprocessorEngine asnPreprocessorEngine = new NiFiASNPreprocessorEngine();

Review Comment:
   Yeah, we don't even need it as field.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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:
   We can go this route. However I think we should remain consistent with our communication in that the documentation already discusses a different pre-processing, namely parsing and compiling the schema.
   That's why I went with the "additional" prefix.
   I'll add something that incorporates both ideas. 



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/preprocess/NiFiASNPreprocessorEngineTest.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+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 NiFiASNPreprocessorEngineTest {

Review Comment:
   In general I'm not a fan of complex tests. A test method - similar to production ones - should do one thing and one thing only. If it fails, it should be narrow enough to know where to look.
   
   In this case due to possible unexpected cross-interaction between the preprocessors it may have some additional value.
   I'll add a very simple one.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/preprocess/NiFiASNPreprocessor.java:
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 java.util.List;
+
+public interface NiFiASNPreprocessor {

Review Comment:
   Usually. But there is a concept of preprocessing _without_ NiFi so this distinction is warranted.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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:
   Good idea, wasn't familiar with this feature.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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.NiFiASNPreprocessor;
+
+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 NiFiASNPreprocessor {
+    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) {
+        List<String> preprocessedLines = new ArrayList<>();
+
+        AtomicInteger unclosedCounter = new AtomicInteger(0);
+        lines.forEach(line -> {
+            StringBuilder preprocessedLine = new StringBuilder();
+
+            String contentToProcess = line;

Review Comment:
   yes



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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()

Review Comment:
   I think the misconception here is that we only need the output directory so that we can look at the resulting asn files. That is not true, it is also required for NiFi itself.
   With that in my I think creating 2 separate properties - neither of which making sense without the other - would only clutter the configuration of the processor uncessarily.



##########
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()

Review Comment:
   I think the misconception here is that we only need the output directory so that we can look at the resulting asn files. That is not true, it is also required for NiFi itself.
   With that in my I think creating 2 separate properties - neither of which making sense without the other - would only clutter the configuration of the processor unnecessarily.



-- 
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 #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features
URL: https://github.com/apache/nifi/pull/6769


-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/preprocess/NiFiASNPreprocessor.java:
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 java.util.List;
+
+public interface NiFiASNPreprocessor {

Review Comment:
   I'll remove the NiFi prefix.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -366,13 +387,13 @@ public void reportError(RecognitionException e) {
             }
         };
 
+        AsnModel model = new AsnModel();
+        parser.module_definitions(model);
+
         if (parseError.get()) {

Review Comment:
   The reportError can be - and should be - called multiple times. We want to know as much errors as possible. Throwing the exception there would allow to report only the first encountered issue. 



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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:
   We can go this route. However I think we should remain consistent with our communication in that the documentation already discusses pre-processing, namely parsing and compiling the schema.
   That's why I went with the "additional" prefix.
   I'll add something that incorporates both ideas. 



-- 
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] nandorsoma commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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:
   Any thoughts on this? It is still there but in the new property.



-- 
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 #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
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


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

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


##########
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:
   I think we have discussed this earlier and agreed that it's not a strict policy, we can leave them there.
   
   It has basically zero disadvantage and significant advantage to some. Tests inherently do at least 3 different things:  prepare a scenario, execute production code and then check the results. It's not practical to extract these, hence these fairly standard comments.
   
   I think it's good to remember: we cannot predict, let alone guarantee how people are trying to read the code. It's up to the reader to decide. And they can decide that they want to start at the WHERE section. I myself am falling into to this category. Stripping them off of the help that make this easier for them (for no practical benefit in return) is not fair in my opinion.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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.NiFiASNPreprocessor;
+
+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 NiFiASNPreprocessor {

Review Comment:
   The preprocessing feature needs to be intentionally switched on. It's not a default behaviour. With that the documentation _is_ enough in my opinion. No need to overcomplicate.



-- 
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] nandorsoma commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
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.NiFiASNPreprocessor;
+
+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 NiFiASNPreprocessor {

Review Comment:
   I recommend using the same naming convention for asn in class names.



##########
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.NiFiASNPreprocessor;
+
+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 NiFiASNPreprocessor {
+    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) {
+        List<String> preprocessedLines = new ArrayList<>();

Review Comment:
   finals are missing in multiple places, can you add them?



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/JASN1ReaderTest.java:
##########
@@ -84,6 +84,7 @@ public void testCanLoadClassCompiledFromAsn() throws Exception {
         ConfigurationContext context = mock(ConfigurationContext.class, RETURNS_DEEP_STUBS);
         when(context.getProperty(ASN_FILES).isSet()).thenReturn(true);
         when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue()).thenReturn(Paths.get("src", "test", "resources", "test.asn").toString());
+//        when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue()).thenReturn(Paths.get("src", "test", "resources", "test_error.asn").toString());

Review Comment:
   Is this line commented out intentionally?



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/preprocess/NiFiASNPreprocessor.java:
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 java.util.List;
+
+public interface NiFiASNPreprocessor {

Review Comment:
   It is usually recommended to avoid prefixing classes with NiFi.



##########
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.NiFiASNPreprocessor;
+
+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 NiFiASNPreprocessor {
+    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) {
+        List<String> preprocessedLines = new ArrayList<>();
+
+        AtomicInteger unclosedCounter = new AtomicInteger(0);
+        lines.forEach(line -> {
+            StringBuilder preprocessedLine = new StringBuilder();
+
+            String contentToProcess = line;

Review Comment:
   Do we need this declaration?



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/preprocess/preprocessors/AbstractPreprocessorTest.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.NiFiASNPreprocessor;
+
+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 AbstractPreprocessorTest {

Review Comment:
   I recommend including ASN in the name of the class.



##########
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:
   Recommend removing support for Variable Registry expressions because this feature is subject to removal and Parameters should be used instead.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/java/org/apache/nifi/jasn1/JASN1Reader.java:
##########
@@ -366,13 +387,13 @@ public void reportError(RecognitionException e) {
             }
         };
 
+        AsnModel model = new AsnModel();
+        parser.module_definitions(model);
+
         if (parseError.get()) {

Review Comment:
   Wouldn't it be less wordy if we threw the exception in the reportError method?



##########
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.NiFiASNPreprocessor;
+
+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 NiFiASNPreprocessor {

Review Comment:
   Overall I feel uneasy about this preprocessor. We are automatically removing constraints that the user otherwise would require. I understand that this is documented in the additionalDetails, but I don't think it is enough. When using this preprocessor, I would notify the user that we are doing something risky and will transfer files without checking for constraints.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/main/resources/docs/org.apache.nifi.jasn1.JASN1Reader/additionalDetails.html:
##########
@@ -122,5 +122,61 @@ <h3>Troubleshooting</h3>
                 </li>
             </ol>
         </p>
+
+        <h3>Additional Preprocessing</h3>
+
+        <p>
+            NiFi doesn't support every feature that the ASN standard allows. To alleviate problems when encountering ASN files with unsupported features,
+            NiFi can 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 a directory set in the 'Additional Preprocessing Output Directory' property.
+            Please note that this is a best-effort attempt. It is also strongly recommended to compare the resulting ASN files to the originals and make sure they are still appropriate.
+            <br />
+            <br />
+            The following modification are applied:
+            <ol>
+                <li>

Review Comment:
   I would instead order this list by creating some precedence. Since "hugging comments" are not ASN features, I would move them to the end of the list.



##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/preprocess/NiFiASNPreprocessorEngineTest.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+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 NiFiASNPreprocessorEngineTest {

Review Comment:
   I recommend adding a test which tests a file which requires all processors. Cases like `[[ integerField3       INTEGER(SIZE(1..8,...,10|12|20)) OPTIONAL]]--hugging`. What do you think?



##########
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)
+        .addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
+        .required(false)
+        .build();
+
     private final List<PropertyDescriptor> propertyDescriptors = Arrays.asList(
         ROOT_MODEL_NAME,
         ROOT_CLASS_NAME,
-        ASN_FILES
+        ASN_FILES,
+        PREPROCESS_OUTPUT_DIRECTORY
     );
 
     private String identifier;
     ComponentLog logger;
 
     private RecordSchemaProvider schemaProvider = new RecordSchemaProvider();
 
+    private NiFiASNPreprocessorEngine asnPreprocessorEngine = new NiFiASNPreprocessorEngine();

Review Comment:
   I recommend creating this engine only when the property is set that enables it. I'm wondering if it would make sense to make the preprocess (and the hierarchy behind it) static since it's not stateful.



##########
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")

Review Comment:
   There is a typo in preprocessing.



##########
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()

Review Comment:
   I think this property is too crowded. I recommend separating it into one that enables preprocessing and one that allows printing out the modified asn files for debugging purposes.



-- 
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] tpalfy commented on a diff in pull request #6769: NIFI-10955 - Added JASN1Reader the ability to try to adjust for unsupported ASN features

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


##########
nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/JASN1ReaderTest.java:
##########
@@ -84,6 +84,7 @@ public void testCanLoadClassCompiledFromAsn() throws Exception {
         ConfigurationContext context = mock(ConfigurationContext.class, RETURNS_DEEP_STUBS);
         when(context.getProperty(ASN_FILES).isSet()).thenReturn(true);
         when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue()).thenReturn(Paths.get("src", "test", "resources", "test.asn").toString());
+//        when(context.getProperty(ASN_FILES).evaluateAttributeExpressions().getValue()).thenReturn(Paths.get("src", "test", "resources", "test_error.asn").toString());

Review Comment:
   It shouldn't be there at all.



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