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

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #7122: NIFI-8294 ADX as source for various NIFI destinations

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


##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-api/pom.xml:
##########
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-api</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.azure.kusto</groupId>
+            <artifactId>kusto-ingest</artifactId>
+            <version>${azure-kusto-java-sdk-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes>
+                        <!-- test data -->
+                        <exclude>src/test/resources/*</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <systemPropertyVariables>
+                        <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
+                    </systemPropertyVariables>
+                </configuration>
+            </plugin>

Review Comment:
   This configuration should be removed. The standard temporary directory should be used without changing it to the project build directory.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-api/pom.xml:
##########
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-api</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.azure.kusto</groupId>
+            <artifactId>kusto-ingest</artifactId>
+            <version>${azure-kusto-java-sdk-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>

Review Comment:
   The code coverage plugin configuration should be removed. This is something that can be considered at a project level, but it should not be configured for specific modules.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-api/pom.xml:
##########
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-api</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.azure.kusto</groupId>
+            <artifactId>kusto-ingest</artifactId>
+            <version>${azure-kusto-java-sdk-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/pom.xml:
##########
@@ -0,0 +1,126 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx</artifactId>
+            <version>${nifi-release-version}</version>

Review Comment:
   Using this `nifi-release-version` is not a standard across the repository. It should be replaced with the explicit version `2.0.0-SNAPSHOT`.
   ```suggestion
               <version>2.0.0-SNAPSHOT</version>
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-api/pom.xml:
##########
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-api</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.azure.kusto</groupId>
+            <artifactId>kusto-ingest</artifactId>
+            <version>${azure-kusto-java-sdk-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes>
+                        <!-- test data -->
+                        <exclude>src/test/resources/*</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <systemPropertyVariables>
+                        <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
+                    </systemPropertyVariables>
+                </configuration>
+            </plugin>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/pom.xml:
##########
@@ -0,0 +1,126 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>

Review Comment:
   These dependencies are applied at the project level and do not need to be declared for individual modules.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/pom.xml:
##########
@@ -0,0 +1,126 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.9.0</version>

Review Comment:
   The version for JUnit is set at the project level, so this explicit version should be removed.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessor.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.data.Client;
+import com.microsoft.azure.kusto.data.KustoOperationResult;
+import com.microsoft.azure.kusto.data.exceptions.DataClientException;
+import com.microsoft.azure.kusto.data.exceptions.DataServiceException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.nifi.adx.AdxSourceConnectionService;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.adx.enums.AzureAdxSourceProcessorParamsEnum;
+import org.apache.nifi.processors.adx.enums.RelationshipStatusEnum;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"azure", "adx", "microsoft", "data", "explorer", "source"})
+@CapabilityDescription("This Processor acts as a ADX source connector which queries data from Azure Data Explorer."+
+        "This connector can act only as a start of the data pipeline getting data from ADX."+
+        "The queries which can be used further details can be found here https://learn.microsoft.com/en-us/azure/data-explorer/kusto/concepts/querylimits")
+@WritesAttributes({
+        @WritesAttribute(attribute = "ADX_QUERY_ERROR_MESSAGE", description = "Azure Data Explorer error message."),
+        @WritesAttribute(attribute = "ADX_EXECUTED_QUERY", description = "Azure Data Explorer executed query.")
+})
+public class AzureAdxSourceProcessor extends AbstractProcessor {
+    public static final String ADX_QUERY_ERROR_MESSAGE = "adx.query.error.message";
+    public static final String ADX_EXECUTED_QUERY = "adx.executed.query";
+    public static final Relationship RL_SUCCEEDED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_SUCCEEDED.name())
+            .description(RelationshipStatusEnum.RL_SUCCEEDED.getDescription())
+            .build();
+    public static final Relationship RL_FAILED = new Relationship.Builder()

Review Comment:
   ```suggestion
       public static final Relationship FAILED = new Relationship.Builder()
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessor.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.data.Client;
+import com.microsoft.azure.kusto.data.KustoOperationResult;
+import com.microsoft.azure.kusto.data.exceptions.DataClientException;
+import com.microsoft.azure.kusto.data.exceptions.DataServiceException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.nifi.adx.AdxSourceConnectionService;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.adx.enums.AzureAdxSourceProcessorParamsEnum;
+import org.apache.nifi.processors.adx.enums.RelationshipStatusEnum;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"azure", "adx", "microsoft", "data", "explorer", "source"})
+@CapabilityDescription("This Processor acts as a ADX source connector which queries data from Azure Data Explorer."+
+        "This connector can act only as a start of the data pipeline getting data from ADX."+
+        "The queries which can be used further details can be found here https://learn.microsoft.com/en-us/azure/data-explorer/kusto/concepts/querylimits")
+@WritesAttributes({
+        @WritesAttribute(attribute = "ADX_QUERY_ERROR_MESSAGE", description = "Azure Data Explorer error message."),
+        @WritesAttribute(attribute = "ADX_EXECUTED_QUERY", description = "Azure Data Explorer executed query.")
+})
+public class AzureAdxSourceProcessor extends AbstractProcessor {
+    public static final String ADX_QUERY_ERROR_MESSAGE = "adx.query.error.message";
+    public static final String ADX_EXECUTED_QUERY = "adx.executed.query";
+    public static final Relationship RL_SUCCEEDED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_SUCCEEDED.name())
+            .description(RelationshipStatusEnum.RL_SUCCEEDED.getDescription())
+            .build();
+    public static final Relationship RL_FAILED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_FAILED.name())
+            .description(RelationshipStatusEnum.RL_FAILED.getDescription())
+            .build();
+    public static final PropertyDescriptor DB_NAME = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.DB_NAME.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_QUERY = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_SOURCE_SERVICE = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDescription())
+            .required(true)
+            .identifiesControllerService(AdxSourceConnectionService.class)
+            .build();
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> descriptors;
+    private Client executionClient;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        this.descriptors = List.of(ADX_SOURCE_SERVICE,DB_NAME,ADX_QUERY);
+        this.relationships = Set.of(RL_SUCCEEDED,RL_FAILED);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        AdxSourceConnectionService service = context.getProperty(ADX_SOURCE_SERVICE).asControllerService(AdxSourceConnectionService.class);
+        executionClient = service.getKustoQueryClient();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile outgoingFlowFile;
+        String databaseName = context.getProperty(DB_NAME).getValue();
+        String adxQuery;
+        KustoOperationResult kustoOperationResult;
+
+        //checks if this processor has any preceding connection, if yes retrieve
+        if ( context.hasIncomingConnection() ) {

Review Comment:
   ```suggestion
           if (context.hasIncomingConnection()) {
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessor.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.data.Client;
+import com.microsoft.azure.kusto.data.KustoOperationResult;
+import com.microsoft.azure.kusto.data.exceptions.DataClientException;
+import com.microsoft.azure.kusto.data.exceptions.DataServiceException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.nifi.adx.AdxSourceConnectionService;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.adx.enums.AzureAdxSourceProcessorParamsEnum;
+import org.apache.nifi.processors.adx.enums.RelationshipStatusEnum;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"azure", "adx", "microsoft", "data", "explorer", "source"})
+@CapabilityDescription("This Processor acts as a ADX source connector which queries data from Azure Data Explorer."+
+        "This connector can act only as a start of the data pipeline getting data from ADX."+
+        "The queries which can be used further details can be found here https://learn.microsoft.com/en-us/azure/data-explorer/kusto/concepts/querylimits")
+@WritesAttributes({
+        @WritesAttribute(attribute = "ADX_QUERY_ERROR_MESSAGE", description = "Azure Data Explorer error message."),
+        @WritesAttribute(attribute = "ADX_EXECUTED_QUERY", description = "Azure Data Explorer executed query.")
+})
+public class AzureAdxSourceProcessor extends AbstractProcessor {
+    public static final String ADX_QUERY_ERROR_MESSAGE = "adx.query.error.message";
+    public static final String ADX_EXECUTED_QUERY = "adx.executed.query";
+    public static final Relationship RL_SUCCEEDED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_SUCCEEDED.name())
+            .description(RelationshipStatusEnum.RL_SUCCEEDED.getDescription())
+            .build();
+    public static final Relationship RL_FAILED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_FAILED.name())
+            .description(RelationshipStatusEnum.RL_FAILED.getDescription())
+            .build();
+    public static final PropertyDescriptor DB_NAME = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.DB_NAME.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_QUERY = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_SOURCE_SERVICE = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDescription())
+            .required(true)
+            .identifiesControllerService(AdxSourceConnectionService.class)
+            .build();
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> descriptors;
+    private Client executionClient;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        this.descriptors = List.of(ADX_SOURCE_SERVICE,DB_NAME,ADX_QUERY);
+        this.relationships = Set.of(RL_SUCCEEDED,RL_FAILED);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        AdxSourceConnectionService service = context.getProperty(ADX_SOURCE_SERVICE).asControllerService(AdxSourceConnectionService.class);
+        executionClient = service.getKustoQueryClient();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile outgoingFlowFile;
+        String databaseName = context.getProperty(DB_NAME).getValue();
+        String adxQuery;
+        KustoOperationResult kustoOperationResult;
+
+        //checks if this processor has any preceding connection, if yes retrieve
+        if ( context.hasIncomingConnection() ) {
+            FlowFile incomingFlowFile = session.get();
+            //incoming connection exists but the incoming flowfile is null
+            if ( incomingFlowFile == null && context.hasNonLoopConnection() ) {
+                getLogger().error("Incoming connection exists but Incoming flow file is null");
+                return;
+            }
+            //incoming connection exists and retrieve adxQuery from context
+            if ( incomingFlowFile != null && incomingFlowFile.getSize() == 0 ) {
+                if ( context.getProperty(ADX_QUERY).isSet() ) {
+                    adxQuery = context.getProperty(ADX_QUERY).evaluateAttributeExpressions(incomingFlowFile).getValue();
+                } else {
+                    String message = "FlowFile query is empty and no scheduled query is set";
+                    getLogger().error(message);
+                    incomingFlowFile = session.putAttribute(incomingFlowFile, ADX_QUERY_ERROR_MESSAGE, message);
+                    session.transfer(incomingFlowFile, RL_FAILED);
+                    return;
+                }
+            } else {
+                try {
+                    adxQuery = getQuery(session, incomingFlowFile);
+                } catch(IOException ioe) {
+                    getLogger().error("Exception while reading from FlowFile " + ioe.getLocalizedMessage(), ioe);
+                    throw new ProcessException(ioe);
+                }
+            }
+            outgoingFlowFile = incomingFlowFile;
+        } else {
+            outgoingFlowFile = session.create();
+            adxQuery = context.getProperty(ADX_QUERY).evaluateAttributeExpressions(outgoingFlowFile).getValue();
+        }
+
+        try {
+            //execute Query
+            kustoOperationResult = executeQuery(databaseName,adxQuery);
+            List<List<Object>> tableData = kustoOperationResult.getPrimaryResults().getData();
+            String json = tableData.size() == 1 ? objectMapper.writeValueAsString(tableData.get(0)) : objectMapper.writeValueAsString(tableData);
+            if ( getLogger().isDebugEnabled() ) {
+                getLogger().debug("Query result {} ", tableData);
+            }
+            try(ByteArrayInputStream bais = new ByteArrayInputStream(json.getBytes())){
+                session.importFrom(bais, outgoingFlowFile);
+            }

Review Comment:
   This type of approach should be changed to use stream the data to an OutputStream using ObjectMapper, instead of holding it all in memory.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/pom.xml:
##########
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.15.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+            </exclusions>

Review Comment:
   This explicit version and exclusion should be removed.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/pom.xml:
##########
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.15.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.14.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-nar/pom.xml:
##########
@@ -0,0 +1,63 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-nar</artifactId>
+    <packaging>nar</packaging>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-processors</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes>
+                        <!-- test data -->
+                        <exclude>src/test/resources/*</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <systemPropertyVariables>
+                        <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
+                    </systemPropertyVariables>
+                </configuration>
+            </plugin>
+        </plugins>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-nar/pom.xml:
##########
@@ -0,0 +1,63 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-nar</artifactId>
+    <packaging>nar</packaging>
+    <properties>
+        <maven.javadoc.skip>true</maven.javadoc.skip>
+        <source.skip>true</source.skip>
+    </properties>

Review Comment:
   This options can be removed.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/pom.xml:
##########
@@ -0,0 +1,126 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.9.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-params</artifactId>
+            <version>5.9.0</version>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/pom.xml:
##########
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.15.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.14.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.9.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>${nifi-release-version}</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.azure.kusto</groupId>
+            <artifactId>kusto-ingest</artifactId>
+            <version>${azure-kusto-java-sdk-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes>
+                        <!-- test data -->
+                        <exclude>src/test/resources/*</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <systemPropertyVariables>
+                        <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
+                    </systemPropertyVariables>
+                </configuration>
+            </plugin>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/pom.xml:
##########
@@ -0,0 +1,126 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.9.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-params</artifactId>
+            <version>5.9.0</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/src/main/java/org/apache/nifi/adx/AzureAdxConnectionServiceParamsEnum.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.adx;
+
+public enum AzureAdxConnectionServiceParamsEnum {

Review Comment:
   ```suggestion
   public enum AzureAdxConnectionServiceParameter {
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/src/main/java/org/apache/nifi/adx/NiFiVersion.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.adx;
+
+import org.apache.commons.lang3.tuple.Pair;import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.InputStream;
+import java.util.Properties;
+
+public class NiFiVersion {
+    public static final String CLIENT_NAME = "Kusto.Nifi";
+
+    public static final Pair<String,String> NIFI_SOURCE = Pair.of("processor", "nifi-source");
+    private static final Logger log = LoggerFactory.getLogger(NiFiVersion.class);
+    private static final String VERSION_FILE = "/azure-kusto-nifi-version.properties";
+    private static String version = "unknown";
+
+    private NiFiVersion() {
+    }
+
+    static {
+        try {
+            Properties props = new Properties();
+            try (InputStream versionFileStream = NiFiVersion.class.getResourceAsStream(VERSION_FILE)) {
+                props.load(versionFileStream);
+                version = props.getProperty("version", version).trim();
+            }
+        } catch (Exception e) {
+            log.warn("Error while loading version:", e);
+        }

Review Comment:
   This approach should be refactored to use `getPackage().getImplementationVersion()` along with adding the information to the module manifest configuration.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessor.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.data.Client;
+import com.microsoft.azure.kusto.data.KustoOperationResult;
+import com.microsoft.azure.kusto.data.exceptions.DataClientException;
+import com.microsoft.azure.kusto.data.exceptions.DataServiceException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.nifi.adx.AdxSourceConnectionService;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.adx.enums.AzureAdxSourceProcessorParamsEnum;
+import org.apache.nifi.processors.adx.enums.RelationshipStatusEnum;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"azure", "adx", "microsoft", "data", "explorer", "source"})
+@CapabilityDescription("This Processor acts as a ADX source connector which queries data from Azure Data Explorer."+
+        "This connector can act only as a start of the data pipeline getting data from ADX."+
+        "The queries which can be used further details can be found here https://learn.microsoft.com/en-us/azure/data-explorer/kusto/concepts/querylimits")
+@WritesAttributes({
+        @WritesAttribute(attribute = "ADX_QUERY_ERROR_MESSAGE", description = "Azure Data Explorer error message."),
+        @WritesAttribute(attribute = "ADX_EXECUTED_QUERY", description = "Azure Data Explorer executed query.")
+})
+public class AzureAdxSourceProcessor extends AbstractProcessor {
+    public static final String ADX_QUERY_ERROR_MESSAGE = "adx.query.error.message";
+    public static final String ADX_EXECUTED_QUERY = "adx.executed.query";
+    public static final Relationship RL_SUCCEEDED = new Relationship.Builder()

Review Comment:
   Using the `RL` prefix is not a standard convention. Although `REL` is sometimes used, recommend eliminating the prefix and changing the name to `SUCCESS`.
   ```suggestion
       public static final Relationship SUCCESS = new Relationship.Builder()
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/src/main/resources/azure-kusto-nifi-version.properties:
##########
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+version=${project.version}

Review Comment:
   As mentioned in the `NiFiVersion` class, this approach should be changed to use the standard Java `getPackage().getImplementationVersion()` approach based on module manifest information.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/pom.xml:
##########
@@ -0,0 +1,126 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.9.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-params</artifactId>
+            <version>5.9.0</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes>
+                        <!-- test data -->
+                        <exclude>src/test/resources/*</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <configuration>
+                    <systemPropertyVariables>
+                        <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
+                    </systemPropertyVariables>
+                </configuration>
+            </plugin>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/enums/RelationshipStatusEnum.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx.enums;
+
+public enum RelationshipStatusEnum {

Review Comment:
   This enum is probably not necessary as the values can be declared directly in the Relationship definition.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/enums/AzureAdxSourceProcessorParamsEnum.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx.enums;
+
+public enum AzureAdxSourceProcessorParamsEnum {

Review Comment:
   The `Enum` suffix should be avoided.
   ```suggestion
   public enum AzureAdxSourceProcessorParameter {
   ```
   



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/src/main/java/org/apache/nifi/adx/NiFiVersion.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.adx;
+
+import org.apache.commons.lang3.tuple.Pair;import org.slf4j.Logger;

Review Comment:
   ```suggestion
   import org.apache.commons.lang3.tuple.Pair;
   import org.slf4j.Logger;
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessor.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.data.Client;
+import com.microsoft.azure.kusto.data.KustoOperationResult;
+import com.microsoft.azure.kusto.data.exceptions.DataClientException;
+import com.microsoft.azure.kusto.data.exceptions.DataServiceException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.nifi.adx.AdxSourceConnectionService;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.adx.enums.AzureAdxSourceProcessorParamsEnum;
+import org.apache.nifi.processors.adx.enums.RelationshipStatusEnum;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"azure", "adx", "microsoft", "data", "explorer", "source"})
+@CapabilityDescription("This Processor acts as a ADX source connector which queries data from Azure Data Explorer."+
+        "This connector can act only as a start of the data pipeline getting data from ADX."+
+        "The queries which can be used further details can be found here https://learn.microsoft.com/en-us/azure/data-explorer/kusto/concepts/querylimits")
+@WritesAttributes({
+        @WritesAttribute(attribute = "ADX_QUERY_ERROR_MESSAGE", description = "Azure Data Explorer error message."),
+        @WritesAttribute(attribute = "ADX_EXECUTED_QUERY", description = "Azure Data Explorer executed query.")
+})
+public class AzureAdxSourceProcessor extends AbstractProcessor {
+    public static final String ADX_QUERY_ERROR_MESSAGE = "adx.query.error.message";
+    public static final String ADX_EXECUTED_QUERY = "adx.executed.query";
+    public static final Relationship RL_SUCCEEDED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_SUCCEEDED.name())
+            .description(RelationshipStatusEnum.RL_SUCCEEDED.getDescription())
+            .build();
+    public static final Relationship RL_FAILED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_FAILED.name())
+            .description(RelationshipStatusEnum.RL_FAILED.getDescription())
+            .build();
+    public static final PropertyDescriptor DB_NAME = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.DB_NAME.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_QUERY = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_SOURCE_SERVICE = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDescription())
+            .required(true)
+            .identifiesControllerService(AdxSourceConnectionService.class)
+            .build();
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> descriptors;
+    private Client executionClient;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        this.descriptors = List.of(ADX_SOURCE_SERVICE,DB_NAME,ADX_QUERY);
+        this.relationships = Set.of(RL_SUCCEEDED,RL_FAILED);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        AdxSourceConnectionService service = context.getProperty(ADX_SOURCE_SERVICE).asControllerService(AdxSourceConnectionService.class);
+        executionClient = service.getKustoQueryClient();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile outgoingFlowFile;
+        String databaseName = context.getProperty(DB_NAME).getValue();
+        String adxQuery;
+        KustoOperationResult kustoOperationResult;
+
+        //checks if this processor has any preceding connection, if yes retrieve
+        if ( context.hasIncomingConnection() ) {
+            FlowFile incomingFlowFile = session.get();
+            //incoming connection exists but the incoming flowfile is null
+            if ( incomingFlowFile == null && context.hasNonLoopConnection() ) {
+                getLogger().error("Incoming connection exists but Incoming flow file is null");
+                return;
+            }
+            //incoming connection exists and retrieve adxQuery from context
+            if ( incomingFlowFile != null && incomingFlowFile.getSize() == 0 ) {
+                if ( context.getProperty(ADX_QUERY).isSet() ) {
+                    adxQuery = context.getProperty(ADX_QUERY).evaluateAttributeExpressions(incomingFlowFile).getValue();
+                } else {
+                    String message = "FlowFile query is empty and no scheduled query is set";
+                    getLogger().error(message);
+                    incomingFlowFile = session.putAttribute(incomingFlowFile, ADX_QUERY_ERROR_MESSAGE, message);
+                    session.transfer(incomingFlowFile, RL_FAILED);
+                    return;
+                }
+            } else {
+                try {
+                    adxQuery = getQuery(session, incomingFlowFile);
+                } catch(IOException ioe) {
+                    getLogger().error("Exception while reading from FlowFile " + ioe.getLocalizedMessage(), ioe);
+                    throw new ProcessException(ioe);

Review Comment:
   It is not necessary to log an error when throwing an exception, since the framework will log exception as an error. Also recommend making the error message more descriptive.
   ```suggestion
                       throw new ProcessException("Failed to read Query from FlowFile", ioe);
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessor.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.data.Client;
+import com.microsoft.azure.kusto.data.KustoOperationResult;
+import com.microsoft.azure.kusto.data.exceptions.DataClientException;
+import com.microsoft.azure.kusto.data.exceptions.DataServiceException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.nifi.adx.AdxSourceConnectionService;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.adx.enums.AzureAdxSourceProcessorParamsEnum;
+import org.apache.nifi.processors.adx.enums.RelationshipStatusEnum;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"azure", "adx", "microsoft", "data", "explorer", "source"})
+@CapabilityDescription("This Processor acts as a ADX source connector which queries data from Azure Data Explorer."+
+        "This connector can act only as a start of the data pipeline getting data from ADX."+
+        "The queries which can be used further details can be found here https://learn.microsoft.com/en-us/azure/data-explorer/kusto/concepts/querylimits")
+@WritesAttributes({
+        @WritesAttribute(attribute = "ADX_QUERY_ERROR_MESSAGE", description = "Azure Data Explorer error message."),
+        @WritesAttribute(attribute = "ADX_EXECUTED_QUERY", description = "Azure Data Explorer executed query.")
+})
+public class AzureAdxSourceProcessor extends AbstractProcessor {
+    public static final String ADX_QUERY_ERROR_MESSAGE = "adx.query.error.message";
+    public static final String ADX_EXECUTED_QUERY = "adx.executed.query";
+    public static final Relationship RL_SUCCEEDED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_SUCCEEDED.name())
+            .description(RelationshipStatusEnum.RL_SUCCEEDED.getDescription())
+            .build();
+    public static final Relationship RL_FAILED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_FAILED.name())
+            .description(RelationshipStatusEnum.RL_FAILED.getDescription())
+            .build();
+    public static final PropertyDescriptor DB_NAME = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.DB_NAME.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_QUERY = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_SOURCE_SERVICE = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDescription())
+            .required(true)
+            .identifiesControllerService(AdxSourceConnectionService.class)
+            .build();
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> descriptors;
+    private Client executionClient;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        this.descriptors = List.of(ADX_SOURCE_SERVICE,DB_NAME,ADX_QUERY);
+        this.relationships = Set.of(RL_SUCCEEDED,RL_FAILED);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        AdxSourceConnectionService service = context.getProperty(ADX_SOURCE_SERVICE).asControllerService(AdxSourceConnectionService.class);
+        executionClient = service.getKustoQueryClient();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile outgoingFlowFile;
+        String databaseName = context.getProperty(DB_NAME).getValue();
+        String adxQuery;
+        KustoOperationResult kustoOperationResult;
+
+        //checks if this processor has any preceding connection, if yes retrieve
+        if ( context.hasIncomingConnection() ) {
+            FlowFile incomingFlowFile = session.get();
+            //incoming connection exists but the incoming flowfile is null
+            if ( incomingFlowFile == null && context.hasNonLoopConnection() ) {
+                getLogger().error("Incoming connection exists but Incoming flow file is null");
+                return;
+            }
+            //incoming connection exists and retrieve adxQuery from context
+            if ( incomingFlowFile != null && incomingFlowFile.getSize() == 0 ) {
+                if ( context.getProperty(ADX_QUERY).isSet() ) {
+                    adxQuery = context.getProperty(ADX_QUERY).evaluateAttributeExpressions(incomingFlowFile).getValue();
+                } else {
+                    String message = "FlowFile query is empty and no scheduled query is set";
+                    getLogger().error(message);
+                    incomingFlowFile = session.putAttribute(incomingFlowFile, ADX_QUERY_ERROR_MESSAGE, message);
+                    session.transfer(incomingFlowFile, RL_FAILED);
+                    return;
+                }
+            } else {
+                try {
+                    adxQuery = getQuery(session, incomingFlowFile);
+                } catch(IOException ioe) {
+                    getLogger().error("Exception while reading from FlowFile " + ioe.getLocalizedMessage(), ioe);
+                    throw new ProcessException(ioe);
+                }
+            }
+            outgoingFlowFile = incomingFlowFile;
+        } else {
+            outgoingFlowFile = session.create();
+            adxQuery = context.getProperty(ADX_QUERY).evaluateAttributeExpressions(outgoingFlowFile).getValue();
+        }
+
+        try {
+            //execute Query
+            kustoOperationResult = executeQuery(databaseName,adxQuery);
+            List<List<Object>> tableData = kustoOperationResult.getPrimaryResults().getData();
+            String json = tableData.size() == 1 ? objectMapper.writeValueAsString(tableData.get(0)) : objectMapper.writeValueAsString(tableData);
+            if ( getLogger().isDebugEnabled() ) {
+                getLogger().debug("Query result {} ", tableData);
+            }

Review Comment:
   The conditional wrapper is not necessary since logging a message with a placeholder does not involve significant additional work. In addition, entire data content should never be logged since it is available through standard provenance features. Recommend removing this debugging.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/pom.xml:
##########
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-nar-bundles</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx-bundle</artifactId>
+    <version>2.0.0-SNAPSHOT</version>
+    <packaging>pom</packaging>
+
+    <properties>
+        <nifi-release-version>2.0.0-SNAPSHOT</nifi-release-version>
+        <azure-kusto-java-sdk-version>4.0.4</azure-kusto-java-sdk-version>
+    </properties>
+
+
+    <modules>
+        <module>nifi-adx-processors</module>
+        <module>nifi-adx-nar</module>
+        <module>nifi-adx</module>
+        <module>nifi-adx-api</module>
+    </modules>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/main/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessor.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.microsoft.azure.kusto.data.Client;
+import com.microsoft.azure.kusto.data.KustoOperationResult;
+import com.microsoft.azure.kusto.data.exceptions.DataClientException;
+import com.microsoft.azure.kusto.data.exceptions.DataServiceException;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.nifi.adx.AdxSourceConnectionService;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.adx.enums.AzureAdxSourceProcessorParamsEnum;
+import org.apache.nifi.processors.adx.enums.RelationshipStatusEnum;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"azure", "adx", "microsoft", "data", "explorer", "source"})
+@CapabilityDescription("This Processor acts as a ADX source connector which queries data from Azure Data Explorer."+
+        "This connector can act only as a start of the data pipeline getting data from ADX."+
+        "The queries which can be used further details can be found here https://learn.microsoft.com/en-us/azure/data-explorer/kusto/concepts/querylimits")
+@WritesAttributes({
+        @WritesAttribute(attribute = "ADX_QUERY_ERROR_MESSAGE", description = "Azure Data Explorer error message."),
+        @WritesAttribute(attribute = "ADX_EXECUTED_QUERY", description = "Azure Data Explorer executed query.")
+})
+public class AzureAdxSourceProcessor extends AbstractProcessor {
+    public static final String ADX_QUERY_ERROR_MESSAGE = "adx.query.error.message";
+    public static final String ADX_EXECUTED_QUERY = "adx.executed.query";
+    public static final Relationship RL_SUCCEEDED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_SUCCEEDED.name())
+            .description(RelationshipStatusEnum.RL_SUCCEEDED.getDescription())
+            .build();
+    public static final Relationship RL_FAILED = new Relationship.Builder()
+            .name(RelationshipStatusEnum.RL_FAILED.name())
+            .description(RelationshipStatusEnum.RL_FAILED.getDescription())
+            .build();
+    public static final PropertyDescriptor DB_NAME = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.DB_NAME.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.DB_NAME.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_QUERY = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_QUERY.getParamDescription())
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor ADX_SOURCE_SERVICE = new PropertyDescriptor
+            .Builder().name(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.name())
+            .displayName(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDisplayName())
+            .description(AzureAdxSourceProcessorParamsEnum.ADX_SOURCE_SERVICE.getParamDescription())
+            .required(true)
+            .identifiesControllerService(AdxSourceConnectionService.class)
+            .build();
+    private final ObjectMapper objectMapper = new ObjectMapper();
+    private Set<Relationship> relationships;
+    private List<PropertyDescriptor> descriptors;
+    private Client executionClient;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        this.descriptors = List.of(ADX_SOURCE_SERVICE,DB_NAME,ADX_QUERY);
+        this.relationships = Set.of(RL_SUCCEEDED,RL_FAILED);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        AdxSourceConnectionService service = context.getProperty(ADX_SOURCE_SERVICE).asControllerService(AdxSourceConnectionService.class);
+        executionClient = service.getKustoQueryClient();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile outgoingFlowFile;
+        String databaseName = context.getProperty(DB_NAME).getValue();
+        String adxQuery;
+        KustoOperationResult kustoOperationResult;
+
+        //checks if this processor has any preceding connection, if yes retrieve
+        if ( context.hasIncomingConnection() ) {
+            FlowFile incomingFlowFile = session.get();
+            //incoming connection exists but the incoming flowfile is null
+            if ( incomingFlowFile == null && context.hasNonLoopConnection() ) {
+                getLogger().error("Incoming connection exists but Incoming flow file is null");

Review Comment:
   This status should not be logged as an error, it is a framework concern, so recommend changing the log to an info or removing the log.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx-processors/src/test/java/org/apache/nifi/processors/adx/AzureAdxSourceProcessorE2ETest.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.adx;
+
+import org.apache.nifi.adx.AzureAdxSourceConnectionService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.Assumptions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+
+/**
+ * These are end to end tests for ADX Source Processor, which are disabled/skipped by default
+ * during maven test phase. These parameters need to be provided during maven build
+ * -DexecuteE2ETests=(set to true)
+ * -DappId=(appId)
+ * -DappKey=(appKey)
+ * -DappTenant=(appTenant)
+ * -DclusterUrl=(clusterUrl)
+ * -DdatabaseName=(databaseName)
+ * -DadxQuery=(query-to-be-executed-in-source-ADX)
+ * -DadxQueryWhichExceedsLimit=(query-who-execution-will-exceed-the-kusto-query-limits)
+ **/
+class AzureAdxSourceProcessorE2ETest {
+
+    private TestRunner testRunner;
+
+    @BeforeEach
+    public void init() throws InitializationException {
+        Assumptions.assumeTrue("true".equalsIgnoreCase(System.getProperty("executeE2ETests")));

Review Comment:
   This approach should be changed to use the `EnabledIfSystemProperty` annotation



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/pom.xml:
##########
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.15.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.14.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.9.0</version>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/pom.xml:
##########
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.15.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.14.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>log4j-over-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-api</artifactId>
+            <version>5.9.0</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <version>${nifi-release-version}</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>com.microsoft.azure.kusto</groupId>
+            <artifactId>kusto-ingest</artifactId>
+            <version>${azure-kusto-java-sdk-version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>${nifi-release-version}</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.jacoco</groupId>
+                <artifactId>jacoco-maven-plugin</artifactId>
+                <version>0.8.6</version>
+                <executions>
+                    <execution>
+                        <id>prepare-agent</id>
+                        <goals>
+                            <goal>prepare-agent</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>report</id>
+                        <phase>test</phase>
+                        <goals>
+                            <goal>report</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/pom.xml:
##########
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.15.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.14.0</version>

Review Comment:
   This version is managed at the root level and should be removed.



##########
nifi-nar-bundles/nifi-adx-bundle/nifi-adx/pom.xml:
##########
@@ -0,0 +1,154 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-adx-bundle</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-adx</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-adx-api</artifactId>
+            <version>${nifi-release-version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>

Review Comment:
   The `nifi-processor-utils` library is no longer part of the project, so this dependency needs to be replaced with the specific dependency required.



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