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/09/23 03:16:39 UTC

[GitHub] [nifi] Lehel44 opened a new pull request, #6444: NIFI-10513: Added capture non-record fields to JsonTreeRowRecordReader

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

   …r, added pagination to QuerySalesforceObject
   
   <!-- 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-10513](https://issues.apache.org/jira/browse/NIFI-10513)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI-10513) 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 #6444: NIFI-10513: Added capture non-record fields to JsonTreeRowRecordReader

Posted by GitBox <gi...@apache.org>.
tpalfy commented on code in PR #6444:
URL: https://github.com/apache/nifi/pull/6444#discussion_r980339020


##########
nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-json-record-utils/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java:
##########
@@ -76,26 +81,60 @@ private AbstractJsonRowRecordReader(final ComponentLog logger, final String date
         LAZY_TIMESTAMP_FORMAT = () -> tsf;
     }
 
-    protected AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger, final String dateFormat, final String timeFormat, final String timestampFormat)
+    protected AbstractJsonRowRecordReader(final InputStream in,
+                                          final ComponentLog logger,
+                                          final String dateFormat,
+                                          final String timeFormat,
+                                          final String timestampFormat)
             throws IOException, MalformedRecordException {
 
-        this(in, logger, dateFormat, timeFormat, timestampFormat, null, null);
+        this(in, logger, dateFormat, timeFormat, timestampFormat, null, null, null);
     }
 
-    protected AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger, final String dateFormat, final String timeFormat, final String timestampFormat,
-                                          final StartingFieldStrategy strategy, final String nestedFieldName) throws IOException, MalformedRecordException {
+    /**
+     * Constructor with initial logic for JSON to NiFi record parsing.
+     *
+     * @param in                     the input stream to parse
+     * @param logger                 ComponentLog
+     * @param dateFormat             format for parsing date fields
+     * @param timeFormat             format for parsing time fields
+     * @param timestampFormat        format for parsing timestamp fields
+     * @param strategy               whether to start processing from a specific field
+     * @param nestedFieldName        the name of the field to start the processing from
+     * @param captureFieldPredicate predicate that takes a JSON fieldName and fieldValue to capture top-level non-processed fields which can
+     *                               be accessed by calling {@link #getCapturedFields()}
+     * @throws IOException              in case of JSON stream processing failure
+     * @throws MalformedRecordException in case of malformed JSON input
+     */
+    protected AbstractJsonRowRecordReader(final InputStream in,
+                                          final ComponentLog logger,
+                                          final String dateFormat,
+                                          final String timeFormat,
+                                          final String timestampFormat,
+                                          final StartingFieldStrategy strategy,
+                                          final String nestedFieldName,
+                                          final BiPredicate<String, String> captureFieldPredicate)
+            throws IOException, MalformedRecordException {
 
         this(logger, dateFormat, timeFormat, timestampFormat);
 
         this.strategy = strategy;
+        this.captureFieldPredicate = captureFieldPredicate;
+        capturedFields = new HashMap<>();
 
         try {
             jsonParser = jsonFactory.createParser(in);
             jsonParser.setCodec(codec);
 
             if (strategy == StartingFieldStrategy.NESTED_FIELD) {
-                final SerializedString serializedStartingFieldName = new SerializedString(nestedFieldName);
-                while (!jsonParser.nextFieldName(serializedStartingFieldName) && jsonParser.hasCurrentToken());
+                while (jsonParser.nextToken() != null) {
+                    if (nestedFieldName.equals(jsonParser.getCurrentName())) {
+                        break;
+                    }

Review Comment:
   ```suggestion
                       if (nestedFieldName.equals(jsonParser.getCurrentName())) {
                           logger.debug("Parsing starting at nested field [{}]", nestedFieldName);
                           break;
                       }
   ```



##########
nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-json-record-utils/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java:
##########
@@ -76,26 +81,60 @@ private AbstractJsonRowRecordReader(final ComponentLog logger, final String date
         LAZY_TIMESTAMP_FORMAT = () -> tsf;
     }
 
-    protected AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger, final String dateFormat, final String timeFormat, final String timestampFormat)
+    protected AbstractJsonRowRecordReader(final InputStream in,
+                                          final ComponentLog logger,
+                                          final String dateFormat,
+                                          final String timeFormat,
+                                          final String timestampFormat)
             throws IOException, MalformedRecordException {
 
-        this(in, logger, dateFormat, timeFormat, timestampFormat, null, null);
+        this(in, logger, dateFormat, timeFormat, timestampFormat, null, null, null);
     }
 
-    protected AbstractJsonRowRecordReader(final InputStream in, final ComponentLog logger, final String dateFormat, final String timeFormat, final String timestampFormat,
-                                          final StartingFieldStrategy strategy, final String nestedFieldName) throws IOException, MalformedRecordException {
+    /**
+     * Constructor with initial logic for JSON to NiFi record parsing.
+     *
+     * @param in                     the input stream to parse
+     * @param logger                 ComponentLog
+     * @param dateFormat             format for parsing date fields
+     * @param timeFormat             format for parsing time fields
+     * @param timestampFormat        format for parsing timestamp fields
+     * @param strategy               whether to start processing from a specific field
+     * @param nestedFieldName        the name of the field to start the processing from
+     * @param captureFieldPredicate predicate that takes a JSON fieldName and fieldValue to capture top-level non-processed fields which can
+     *                               be accessed by calling {@link #getCapturedFields()}
+     * @throws IOException              in case of JSON stream processing failure
+     * @throws MalformedRecordException in case of malformed JSON input
+     */
+    protected AbstractJsonRowRecordReader(final InputStream in,
+                                          final ComponentLog logger,
+                                          final String dateFormat,
+                                          final String timeFormat,
+                                          final String timestampFormat,
+                                          final StartingFieldStrategy strategy,
+                                          final String nestedFieldName,
+                                          final BiPredicate<String, String> captureFieldPredicate)
+            throws IOException, MalformedRecordException {
 
         this(logger, dateFormat, timeFormat, timestampFormat);
 
         this.strategy = strategy;
+        this.captureFieldPredicate = captureFieldPredicate;
+        capturedFields = new HashMap<>();
 
         try {
             jsonParser = jsonFactory.createParser(in);
             jsonParser.setCodec(codec);
 
             if (strategy == StartingFieldStrategy.NESTED_FIELD) {
-                final SerializedString serializedStartingFieldName = new SerializedString(nestedFieldName);
-                while (!jsonParser.nextFieldName(serializedStartingFieldName) && jsonParser.hasCurrentToken());
+                while (jsonParser.nextToken() != null) {
+                    if (nestedFieldName.equals(jsonParser.getCurrentName())) {
+                        break;
+                    }
+                    if (captureFieldPredicate != null) {
+                        captureCurrentField(captureFieldPredicate);
+                    }
+                }
                 logger.debug("Parsing starting at nested field [{}]", nestedFieldName);

Review Comment:
   ```suggestion
   ```



-- 
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] asfgit closed pull request #6444: NIFI-10513: Added capture non-record fields to JsonTreeRowRecordReader

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #6444: NIFI-10513: Added capture non-record fields to JsonTreeRowRecordReader
URL: https://github.com/apache/nifi/pull/6444


-- 
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 #6444: NIFI-10513: Added capture non-record fields to JsonTreeRowRecordReader

Posted by GitBox <gi...@apache.org>.
tpalfy commented on code in PR #6444:
URL: https://github.com/apache/nifi/pull/6444#discussion_r981142413


##########
nifi-nar-bundles/nifi-salesforce-bundle/nifi-salesforce-processors/src/main/java/org/apache/nifi/processors/salesforce/QuerySalesforceObject.java:
##########
@@ -330,76 +334,96 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
                 ageFilterUpper
         );
 
+        AtomicReference<String> nextRecordsUrl = new AtomicReference<>();
+
+        do {
+
         FlowFile flowFile = session.create();
 
         Map<String, String> originalAttributes = flowFile.getAttributes();
         Map<String, String> attributes = new HashMap<>();
 
         AtomicInteger recordCountHolder = new AtomicInteger();
 
-        flowFile = session.write(flowFile, out -> {
-            try (
-                    InputStream querySObjectResultInputStream = salesforceRestService.query(querySObject);
-                    JsonTreeRowRecordReader jsonReader = new JsonTreeRowRecordReader(
-                            querySObjectResultInputStream,
-                            getLogger(),
-                            convertedSalesforceSchema.recordSchema,
-                            DATE_FORMAT,
-                            TIME_FORMAT,
-                            DATE_TIME_FORMAT,
-                            StartingFieldStrategy.NESTED_FIELD,
-                            STARTING_FIELD_NAME,
-                            SchemaApplicationStrategy.SELECTED_PART
-                    );
-
-                    RecordSetWriter writer = writerFactory.createWriter(
-                            getLogger(),
-                            writerFactory.getSchema(
-                                    originalAttributes,
-                                    convertedSalesforceSchema.recordSchema
-                            ),
-                            out,
-                            originalAttributes
-                    )
-            ) {
-                writer.beginRecordSet();
-
-                Record querySObjectRecord;
-                while ((querySObjectRecord = jsonReader.nextRecord()) != null) {
-                    writer.write(querySObjectRecord);
-                }
-
-                WriteResult writeResult = writer.finishRecordSet();
-
-                attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
-                attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
-                attributes.putAll(writeResult.getAttributes());
 
-                recordCountHolder.set(writeResult.getRecordCount());
 
-                if (ageFilterUpper != null) {
-                    Map<String, String> newState = new HashMap<>(state.toMap());
-                    newState.put(LAST_AGE_FILTER, ageFilterUpper);
-                    updateState(context, newState);
+            flowFile = session.write(flowFile, out -> {
+                try (
+                        InputStream querySObjectResultInputStream = getResultInputStream(nextRecordsUrl, querySObject);
+
+                        JsonTreeRowRecordReader jsonReader = new JsonTreeRowRecordReader(
+                                querySObjectResultInputStream,
+                                getLogger(),
+                                convertedSalesforceSchema.recordSchema,
+                                DATE_FORMAT,
+                                TIME_FORMAT,
+                                DATE_TIME_FORMAT,
+                                StartingFieldStrategy.NESTED_FIELD,
+                                STARTING_FIELD_NAME,
+                                SchemaApplicationStrategy.SELECTED_PART,
+                                CAPTURE_PREDICATE
+                        );
+
+                        RecordSetWriter writer = writerFactory.createWriter(
+                                getLogger(),
+                                writerFactory.getSchema(
+                                        originalAttributes,
+                                        convertedSalesforceSchema.recordSchema
+                                ),
+                                out,
+                                originalAttributes
+                        )
+                ) {
+                    writer.beginRecordSet();
+
+                    Record querySObjectRecord;
+                    while ((querySObjectRecord = jsonReader.nextRecord()) != null) {
+                        writer.write(querySObjectRecord);
+                    }
+
+                    WriteResult writeResult = writer.finishRecordSet();
+
+                    Map<String, String> storedFields = jsonReader.getCapturedFields();
+
+                    nextRecordsUrl.set(storedFields.getOrDefault(CURSOR_URL, null));

Review Comment:
   ```suggestion
                       Map<String, String> capturedFields = jsonReader.getCapturedFields();
   
                       nextRecordsUrl.set(capturedFields.getOrDefault(CURSOR_URL, null));
   ```



##########
nifi-nar-bundles/nifi-salesforce-bundle/nifi-salesforce-processors/src/main/java/org/apache/nifi/processors/salesforce/QuerySalesforceObject.java:
##########
@@ -330,76 +334,96 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
                 ageFilterUpper
         );
 
+        AtomicReference<String> nextRecordsUrl = new AtomicReference<>();
+
+        do {
+
         FlowFile flowFile = session.create();
 
         Map<String, String> originalAttributes = flowFile.getAttributes();
         Map<String, String> attributes = new HashMap<>();
 
         AtomicInteger recordCountHolder = new AtomicInteger();
 
-        flowFile = session.write(flowFile, out -> {
-            try (
-                    InputStream querySObjectResultInputStream = salesforceRestService.query(querySObject);
-                    JsonTreeRowRecordReader jsonReader = new JsonTreeRowRecordReader(
-                            querySObjectResultInputStream,
-                            getLogger(),
-                            convertedSalesforceSchema.recordSchema,
-                            DATE_FORMAT,
-                            TIME_FORMAT,
-                            DATE_TIME_FORMAT,
-                            StartingFieldStrategy.NESTED_FIELD,
-                            STARTING_FIELD_NAME,
-                            SchemaApplicationStrategy.SELECTED_PART
-                    );
-
-                    RecordSetWriter writer = writerFactory.createWriter(
-                            getLogger(),
-                            writerFactory.getSchema(
-                                    originalAttributes,
-                                    convertedSalesforceSchema.recordSchema
-                            ),
-                            out,
-                            originalAttributes
-                    )
-            ) {
-                writer.beginRecordSet();
-
-                Record querySObjectRecord;
-                while ((querySObjectRecord = jsonReader.nextRecord()) != null) {
-                    writer.write(querySObjectRecord);
-                }
-
-                WriteResult writeResult = writer.finishRecordSet();
-
-                attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
-                attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
-                attributes.putAll(writeResult.getAttributes());
 
-                recordCountHolder.set(writeResult.getRecordCount());
 
-                if (ageFilterUpper != null) {
-                    Map<String, String> newState = new HashMap<>(state.toMap());
-                    newState.put(LAST_AGE_FILTER, ageFilterUpper);
-                    updateState(context, newState);
+            flowFile = session.write(flowFile, out -> {
+                try (
+                        InputStream querySObjectResultInputStream = getResultInputStream(nextRecordsUrl, querySObject);
+
+                        JsonTreeRowRecordReader jsonReader = new JsonTreeRowRecordReader(
+                                querySObjectResultInputStream,
+                                getLogger(),
+                                convertedSalesforceSchema.recordSchema,
+                                DATE_FORMAT,
+                                TIME_FORMAT,
+                                DATE_TIME_FORMAT,
+                                StartingFieldStrategy.NESTED_FIELD,
+                                STARTING_FIELD_NAME,
+                                SchemaApplicationStrategy.SELECTED_PART,
+                                CAPTURE_PREDICATE
+                        );
+
+                        RecordSetWriter writer = writerFactory.createWriter(
+                                getLogger(),
+                                writerFactory.getSchema(
+                                        originalAttributes,
+                                        convertedSalesforceSchema.recordSchema
+                                ),
+                                out,
+                                originalAttributes
+                        )
+                ) {
+                    writer.beginRecordSet();
+
+                    Record querySObjectRecord;
+                    while ((querySObjectRecord = jsonReader.nextRecord()) != null) {
+                        writer.write(querySObjectRecord);
+                    }
+
+                    WriteResult writeResult = writer.finishRecordSet();
+
+                    Map<String, String> storedFields = jsonReader.getCapturedFields();
+
+                    nextRecordsUrl.set(storedFields.getOrDefault(CURSOR_URL, null));
+
+                    attributes.put("record.count", String.valueOf(writeResult.getRecordCount()));
+                    attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType());
+                    attributes.putAll(writeResult.getAttributes());
+
+                    recordCountHolder.set(writeResult.getRecordCount());
+
+                    if (ageFilterUpper != null) {
+                        Map<String, String> newState = new HashMap<>(state.toMap());
+                        newState.put(LAST_AGE_FILTER, ageFilterUpper);
+                        updateState(context, newState);
+                    }
+                } catch (SchemaNotFoundException e) {
+                    throw new ProcessException("Couldn't create record writer", e);
+                } catch (MalformedRecordException e) {
+                    throw new ProcessException("Couldn't read records from input", e);
                 }
-            } catch (SchemaNotFoundException e) {
-                throw new ProcessException("Couldn't create record writer", e);
-            } catch (MalformedRecordException e) {
-                throw new ProcessException("Couldn't read records from input", e);
-            }
-        });
+            });
 
-        int recordCount = recordCountHolder.get();
+            int recordCount = recordCountHolder.get();
 
-        if (!createZeroRecordFlowFiles && recordCount == 0) {
-            session.remove(flowFile);
-        } else {
-            flowFile = session.putAllAttributes(flowFile, attributes);
-            session.transfer(flowFile, REL_SUCCESS);
+            if (!createZeroRecordFlowFiles && recordCount == 0) {
+                session.remove(flowFile);
+            } else {
+                flowFile = session.putAllAttributes(flowFile, attributes);
+                session.transfer(flowFile, REL_SUCCESS);
+
+                session.adjustCounter("Records Processed", recordCount, false);
+                getLogger().info("Successfully written {} records for {}", recordCount, flowFile);
+            }
+        } while (nextRecordsUrl.get() != null);
+    }
 
-            session.adjustCounter("Records Processed", recordCount, false);
-            getLogger().info("Successfully written {} records for {}", recordCount, flowFile);
+    private InputStream getResultInputStream(AtomicReference<String> nextRecordsUrl, String querySObject) {
+        if (nextRecordsUrl.get() == null) {
+            return salesforceRestService.query(querySObject);
         }
+        return salesforceRestService.queryNextRecords(nextRecordsUrl.get());

Review Comment:
   ```suggestion
           return salesforceRestService.getNextRecords(nextRecordsUrl.get());
   ```



##########
nifi-nar-bundles/nifi-salesforce-bundle/nifi-salesforce-processors/src/main/java/org/apache/nifi/processors/salesforce/QuerySalesforceObject.java:
##########
@@ -220,6 +222,8 @@ public class QuerySalesforceObject extends AbstractProcessor {
     private static final String DATE_FORMAT = "yyyy-MM-dd";
     private static final String TIME_FORMAT = "HH:mm:ss.SSSX";
     private static final String DATE_TIME_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSZZZZ";
+    private static final String CURSOR_URL = "nextRecordsUrl";

Review Comment:
   ```suggestion
       private static final String NEXT_RECORDS_URL = "nextRecordsUrl";
   ```



##########
nifi-nar-bundles/nifi-salesforce-bundle/nifi-salesforce-processors/src/main/java/org/apache/nifi/processors/salesforce/util/SalesforceRestService.java:
##########
@@ -69,6 +69,21 @@ public InputStream query(String query) {
         return request(request);
     }
 
+    public InputStream queryNextRecords(String uri) {
+        String url = baseUrl + uri;

Review Comment:
   ```suggestion
       public InputStream getNextRecords(String nextRecordsUrl) {
           String url = baseUrl + nextRecordsUrl;
   ```



##########
nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-json-record-utils/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java:
##########
@@ -48,9 +47,12 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.function.BiPredicate;
 import java.util.function.Supplier;
 
 public abstract class AbstractJsonRowRecordReader implements RecordReader {
+    private static final String ARRAY_OR_OBJECT_TYPE = "array or object type field";
+

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-salesforce-bundle/nifi-salesforce-processors/src/main/java/org/apache/nifi/processors/salesforce/QuerySalesforceObject.java:
##########
@@ -330,76 +334,96 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
                 ageFilterUpper
         );
 
+        AtomicReference<String> nextRecordsUrl = new AtomicReference<>();
+
+        do {
+
         FlowFile flowFile = session.create();
 
         Map<String, String> originalAttributes = flowFile.getAttributes();
         Map<String, String> attributes = new HashMap<>();
 
         AtomicInteger recordCountHolder = new AtomicInteger();
 

Review Comment:
   ```suggestion
               FlowFile flowFile = session.create();
   
               Map<String, String> originalAttributes = flowFile.getAttributes();
               Map<String, String> attributes = new HashMap<>();
   
               AtomicInteger recordCountHolder = new AtomicInteger();
   ```



-- 
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 pull request #6444: NIFI-10513: Added capture non-record fields to JsonTreeRowRecordReader

Posted by GitBox <gi...@apache.org>.
tpalfy commented on PR #6444:
URL: https://github.com/apache/nifi/pull/6444#issuecomment-1259854828

   LGTM
   Thanks for your work @Lehel44 !
   Merged to main.


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