You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2019/06/20 17:44:34 UTC

[nifi] 02/02: NIFI-5537 Refactored to be named Cypher Bundle instead. NIFI-5537 Added skeletons of services, apis, etc. NIFI-5537 Renamed classes to reflect that they are for cypher and not neo4j specific NIFI-5537 Added start of client API for doing streaming execution. NIFI-5537 Completed refactor of unit and integration tests. NIFI-5537 Added OpenCypherClientService. NIFI-5537 Updated to gremlin-core 3.3.5 to get better SSL configuration options. NIFI-5537 Added SSL support to OpenCypher client servi [...]

This is an automated email from the ASF dual-hosted git repository.

mattyb149 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 82f177c63245b2d14204ff3b45e09860af8ede47
Author: Mike Thomsen <mi...@gmail.com>
AuthorDate: Sat Feb 2 21:11:31 2019 -0500

    NIFI-5537 Refactored to be named Cypher Bundle instead.
    NIFI-5537 Added skeletons of services, apis, etc.
    NIFI-5537 Renamed classes to reflect that they are for cypher and not neo4j specific
    NIFI-5537 Added start of client API for doing streaming execution.
    NIFI-5537 Completed refactor of unit and integration tests.
    NIFI-5537 Added OpenCypherClientService.
    NIFI-5537 Updated to gremlin-core 3.3.5 to get better SSL configuration options.
    NIFI-5537 Added SSL support to OpenCypher client service.
    NIFI-5537 Updated L&N.
    
    NIFI-5537 Updated to 1.10.0-SNAPSHOT.
    
    NIFI-5537 Removed Mac-specific documentation in int test.
    
    NIFI-5537 Updated a few properties to add EL.
    
    NIFI-5537 Added GremlinClientService to support gremlin as well.
    
    NIFI-5537 refactored everything to be a graph bundle.
    
    NIFI-5537 Updated documentation.
    
    NIFI-5537 Fixed assembly.
    
    NIFI-5537 Updated Jackson.
    
    NIFI-5537 Moved MIT section per code review guidance.
    
    NIFI-5537 Changed provenance event and made it more explicit.
    
    NIFI-5537 Updated processor config file.
    
    NIFI-5537 Made changes requested in a code review.
    
    NIFI-5537 Added documentation about driver configuration.
    
    NIFI-5537 Remove output flowfile if an exception is thrown.
    
    NIFI-5537 Updated configuration properties.
    
    NIFI-5537 Made port and path configurable in the tinkerpop family of components.
    NIFI-5537 Upgraded to OpenCypher cypher-gremlin-neo4j-driver 0.9.13 because it's the most recent version that JanusGraph supports.
    
    NIFI-5537 Added warning about Janus 0.3.X to int test.
    
    NIFI-5537 Added TinkerPopClientService interface.
    
    NIFI-5537 Changed to mockito-core from mockito-all.
    
    NIFI-5537 Added documentation about Gremlin for first time users w/ NiFi.
---
 nifi-assembly/pom.xml                              |  22 +-
 .../nifi-graph-client-service-api-nar}/pom.xml     |  21 +-
 .../nifi-graph-client-service-api}/pom.xml         |  26 +-
 .../org/apache/nifi/graph/GraphClientService.java  |  35 ++
 .../nifi/graph/GraphQueryResultCallback.java       |  24 ++
 .../apache/nifi/graph/TinkerPopClientService.java  |  25 ++
 .../nifi-graph-nar}/pom.xml                        |  12 +-
 .../src/main/resources/META-INF/LICENSE            |   0
 .../src/main/resources/META-INF/NOTICE             |  44 ++
 .../nifi-graph-processors}/pom.xml                 |  26 +-
 .../processors/graph/AbstractGraphExecutor.java    |  98 +++++
 .../nifi/processors/graph/ExecuteGraphQuery.java   | 185 +++++++++
 .../services/org.apache.nifi.processor.Processor   |   2 +-
 .../additionalDetails.html                         |  37 ++
 .../processors/graph/MockCypherClientService.java  |  56 +++
 .../processors/graph/TestExecuteGraphQuery.java    | 101 +++++
 .../nifi-neo4j-cypher-service-nar/pom.xml          |  37 ++
 .../src/main/resources/META-INF/LICENSE            |  63 +++
 .../src/main/resources/META-INF/NOTICE             | 152 +++++++
 .../nifi-neo4j-cypher-service/pom.xml              | 104 +++++
 .../nifi/graph/Neo4JCypherClientService.java}      | 260 ++++++------
 .../org.apache.nifi.controller.ControllerService}  |   3 +-
 .../apache/nifi/graph/ITNeo4JCypherExecutor.java   | 152 +++++++
 .../java/org/apache/nifi/graph/MockProcessor.java  |  47 +++
 .../nifi-other-graph-services-nar/pom.xml          |  37 ++
 .../src/main/resources/META-INF/LICENSE            |  52 +++
 .../src/main/resources/META-INF/NOTICE             | 452 +++++++++++++++++++++
 .../nifi-other-graph-services/pom.xml              | 112 +++++
 .../nifi/graph/AbstractTinkerpopClientService.java | 109 +++++
 .../apache/nifi/graph/GremlinClientService.java    |  93 +++++
 .../apache/nifi/graph/OpenCypherClientService.java | 103 +++++
 .../org.apache.nifi.controller.ControllerService}  |   4 +-
 .../additionalDetails.html                         |  54 +++
 .../apache/nifi/graph/GremlinClientServiceIT.java  |  75 ++++
 .../java/org/apache/nifi/graph/MockProcessor.java  |  52 +++
 .../nifi/graph/OpenCypherClientServiceIT.java      |  96 +++++
 .../nifi/graph/TestableGremlinClientService.java   |  26 ++
 .../src/test/resources/setup.gremlin               |  41 ++
 .../src/test/resources/teardown.gremlin            |  19 +
 .../pom.xml                                        |  20 +-
 .../src/main/resources/META-INF/NOTICE             |  31 --
 .../nifi/processors/neo4j/Neo4JCypherExecutor.java | 207 ----------
 .../processors/neo4j/ITNeo4JCypherExecutor.java    | 219 ----------
 .../processors/neo4j/TestNeo4JCypherExecutor.java  | 272 -------------
 nifi-nar-bundles/pom.xml                           |   2 +-
 45 files changed, 2700 insertions(+), 908 deletions(-)

diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index abec31c..4aea8df 100755
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -399,8 +399,26 @@ language governing permissions and limitations under the License. -->
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-neo4j-nar</artifactId>
-            <version>1.8.0-SNAPSHOT</version>
+            <artifactId>nifi-graph-client-service-api-nar</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-neo4j-cypher-service-nar</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-other-graph-services-nar</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-graph-nar</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml
similarity index 77%
copy from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml
copy to nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml
index 16ee10f..9e9cf11 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml
@@ -18,28 +18,25 @@
 
     <parent>
         <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-neo4j-bundle</artifactId>
-        <version>1.8.0-SNAPSHOT</version>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
     </parent>
 
-    <artifactId>nifi-neo4j-nar</artifactId>
+    <artifactId>nifi-graph-client-service-api-nar</artifactId>
     <packaging>nar</packaging>
-    <properties>
-        <maven.javadoc.skip>true</maven.javadoc.skip>
-        <source.skip>true</source.skip>
-    </properties>
 
     <dependencies>
-        <dependency>
+	    <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.8.0-SNAPSHOT</version>
-            <type>nar</type>
+            <version>1.10.0-SNAPSHOT</version>
+	        <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-neo4j-processors</artifactId>
+            <artifactId>nifi-graph-client-service-api</artifactId>
+	        <version>1.10.0-SNAPSHOT</version>
+            <scope>compile</scope>
         </dependency>
     </dependencies>
-
 </project>
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml
similarity index 71%
copy from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml
copy to nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml
index 16ee10f..2ebe33a 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml
@@ -18,28 +18,28 @@
 
     <parent>
         <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-neo4j-bundle</artifactId>
-        <version>1.8.0-SNAPSHOT</version>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
     </parent>
 
-    <artifactId>nifi-neo4j-nar</artifactId>
-    <packaging>nar</packaging>
-    <properties>
-        <maven.javadoc.skip>true</maven.javadoc.skip>
-        <source.skip>true</source.skip>
-    </properties>
+    <artifactId>nifi-graph-client-service-api</artifactId>
+    <packaging>jar</packaging>
 
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.8.0-SNAPSHOT</version>
-            <type>nar</type>
+            <artifactId>nifi-api</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-neo4j-processors</artifactId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <scope>compile</scope>
         </dependency>
     </dependencies>
-
 </project>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphClientService.java
new file mode 100644
index 0000000..035a6d7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphClientService.java
@@ -0,0 +1,35 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.controller.ControllerService;
+
+import java.util.Map;
+
+public interface GraphClientService extends ControllerService {
+    String NODES_CREATED= "graph.nodes.created";
+    String RELATIONS_CREATED = "graph.relations.created";
+    String LABELS_ADDED = "graph.labels.added";
+    String NODES_DELETED = "graph.nodes.deleted";
+    String RELATIONS_DELETED = "graph.relations.deleted";
+    String PROPERTIES_SET = "graph.properties.set";
+    String ROWS_RETURNED = "graph.rows.returned";
+
+    Map<String, String> executeQuery(String query, Map<String, Object> parameters, GraphQueryResultCallback handler);
+    String getTransitUrl();
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphQueryResultCallback.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphQueryResultCallback.java
new file mode 100644
index 0000000..46e18f0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphQueryResultCallback.java
@@ -0,0 +1,24 @@
+/*
+ * 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.graph;
+
+import java.util.Map;
+
+public interface GraphQueryResultCallback {
+    void process(Map<String, Object> record, boolean isMore);
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/TinkerPopClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/TinkerPopClientService.java
new file mode 100644
index 0000000..7cd0eb4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/TinkerPopClientService.java
@@ -0,0 +1,25 @@
+/*
+ * 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.graph;
+
+/**
+ * This interface is meant to make it easier for anyone who wants to write components to target TinkerPop/OpenCypher
+ * client services only.
+ */
+public interface TinkerPopClientService extends GraphClientService {
+}
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml
similarity index 83%
rename from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml
rename to nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml
index 16ee10f..18e4e1a 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml
@@ -18,11 +18,11 @@
 
     <parent>
         <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-neo4j-bundle</artifactId>
-        <version>1.8.0-SNAPSHOT</version>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
     </parent>
 
-    <artifactId>nifi-neo4j-nar</artifactId>
+    <artifactId>nifi-graph-nar</artifactId>
     <packaging>nar</packaging>
     <properties>
         <maven.javadoc.skip>true</maven.javadoc.skip>
@@ -32,13 +32,13 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-services-api-nar</artifactId>
-            <version>1.8.0-SNAPSHOT</version>
+            <artifactId>nifi-graph-client-service-api-nar</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-neo4j-processors</artifactId>
+            <artifactId>nifi-graph-processors</artifactId>
         </dependency>
     </dependencies>
 
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/src/main/resources/META-INF/LICENSE
similarity index 100%
copy from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE
copy to nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/src/main/resources/META-INF/LICENSE
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..d8a9e95
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,44 @@
+nifi-cypher-nar
+Copyright 2016 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+===========================================
+Apache Software License v2
+===========================================
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml
similarity index 84%
rename from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/pom.xml
rename to nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml
index 475a5a7..50094e6 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml
@@ -18,20 +18,14 @@
 
     <parent>
         <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-neo4j-bundle</artifactId>
-        <version>1.8.0-SNAPSHOT</version>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
     </parent>
 
-    <artifactId>nifi-neo4j-processors</artifactId>
+    <artifactId>nifi-graph-processors</artifactId>
     <packaging>jar</packaging>
 
     <dependencies>
-         <dependency>
-             <groupId>org.neo4j.driver</groupId>
-             <artifactId>neo4j-java-driver</artifactId>
-             <version>1.6.2</version>
-             <scope>compile</scope>
-        </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
@@ -43,7 +37,7 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <version>1.8.0-SNAPSHOT</version>
+            <version>1.10.0-SNAPSHOT</version>
         </dependency>
         <dependency>
             <groupId>commons-io</groupId>
@@ -58,12 +52,12 @@
         <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-databind</artifactId>
-            <version>2.9.5</version>
+            <version>2.9.8</version>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
-            <version>1.8.0-SNAPSHOT</version>
+            <version>1.10.0-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
         <dependency>
@@ -84,8 +78,14 @@
         </dependency>
 		<dependency>
 			<groupId>org.mockito</groupId>
-			<artifactId>mockito-all</artifactId>
+			<artifactId>mockito-core</artifactId>
 			<scope>test</scope>
 		</dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-graph-client-service-api</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/AbstractGraphExecutor.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/AbstractGraphExecutor.java
new file mode 100644
index 0000000..b8bc7de
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/AbstractGraphExecutor.java
@@ -0,0 +1,98 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Abstract base class for ExecuteGraphQuery processors
+ */
+abstract class AbstractGraphExecutor extends AbstractProcessor {
+    public static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+        .name("graph-client-service")
+        .displayName("Client Service")
+        .description("The graph client service for connecting to the graph database.")
+        .required(true)
+        .identifiesControllerService(GraphClientService.class)
+        .build();
+    public static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
+        .name("graph-query")
+        .displayName("Graph Query")
+        .description("Specifies the graph query. If it is left blank, the processor will attempt " +
+                "to get the query from body.")
+        .required(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+        .build();
+
+    static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successful FlowFiles are routed to this relationship").build();
+
+    static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
+            .description("Failed FlowFiles are routed to this relationship").build();
+
+    static final Relationship REL_ORIGINAL = new Relationship.Builder().name("original")
+            .description("If there is an input flowfile, the original input flowfile will be " +
+                    "written to this relationship if the operation succeeds.")
+            .build();
+
+    public static final String ERROR_MESSAGE = "graph.error.message";
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String name) {
+        return new PropertyDescriptor.Builder()
+            .name(name)
+            .displayName(name)
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    }
+
+    protected List<PropertyDescriptor> queryParameters;
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        queryParameters = context.getProperties()
+            .keySet().stream()
+            .filter(prop -> prop.isDynamic())
+            .collect(Collectors.toList());
+    }
+
+    protected Map<String, Object> getParameters(ProcessContext context, FlowFile input) {
+        Map<String, Object> params = new HashMap<>();
+        for (PropertyDescriptor descriptor : queryParameters) {
+            String value = context.getProperty(descriptor).evaluateAttributeExpressions(input).getValue();
+            params.put(descriptor.getName(), value);
+        }
+
+        return params;
+    }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQuery.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQuery.java
new file mode 100644
index 0000000..b909e2d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/java/org/apache/nifi/processors/graph/ExecuteGraphQuery.java
@@ -0,0 +1,185 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+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.attributes.CoreAttributes;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.StringUtils;
+
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
+@EventDriven
+@SupportsBatching
+@Tags({"cypher", "neo4j", "graph", "network", "insert", "update", "delete", "put", "get",
+        "node", "relationship", "connection", "executor", "gremlin", "tinkerpop"})
+@CapabilityDescription("This processor is designed to execute queries in either the Cypher query language or the Tinkerpop " +
+        "Gremlin DSL. It delegates most of the logic to a configured client service that handles the interaction with the " +
+        "remote data source. All of the output is written out as JSON data.")
+@WritesAttributes({
+    @WritesAttribute(attribute = AbstractGraphExecutor.ERROR_MESSAGE, description = "GraphDB error message"),
+    @WritesAttribute(attribute = GraphClientService.LABELS_ADDED, description = "Number of labels added"),
+    @WritesAttribute(attribute = GraphClientService.NODES_CREATED, description = "Number of nodes created"),
+    @WritesAttribute(attribute = GraphClientService.NODES_DELETED, description = "Number of nodes deleted"),
+    @WritesAttribute(attribute = GraphClientService.PROPERTIES_SET, description = "Number of properties set"),
+    @WritesAttribute(attribute = GraphClientService.RELATIONS_CREATED, description = "Number of relationships created"),
+    @WritesAttribute(attribute = GraphClientService.RELATIONS_DELETED, description = "Number of relationships deleted"),
+    @WritesAttribute(attribute = GraphClientService.ROWS_RETURNED, description = "Number of rows returned"),
+    @WritesAttribute(attribute = ExecuteGraphQuery.EXECUTION_TIME, description = "The amount of time in milliseconds that the query" +
+            "took to execute.")
+    })
+public class ExecuteGraphQuery extends AbstractGraphExecutor {
+
+    private static final Set<Relationship> relationships;
+    private static final List<PropertyDescriptor> propertyDescriptors;
+
+    public static final String EXECUTION_TIME = "query.took";
+
+    static {
+        final Set<Relationship> tempRelationships = new HashSet<>();
+        tempRelationships.add(REL_SUCCESS);
+        tempRelationships.add(REL_ORIGINAL);
+        tempRelationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(tempRelationships);
+
+        final List<PropertyDescriptor> tempDescriptors = new ArrayList<>();
+        tempDescriptors.add(CLIENT_SERVICE);
+        tempDescriptors.add(QUERY);
+
+        propertyDescriptors = Collections.unmodifiableList(tempDescriptors);
+    }
+
+    protected ObjectMapper mapper = new ObjectMapper();
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return propertyDescriptors;
+    }
+
+    private volatile GraphClientService clientService;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        super.onScheduled(context);
+        clientService = context.getProperty(CLIENT_SERVICE).asControllerService(GraphClientService.class);
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+
+        FlowFile output = flowFile != null ? session.create(flowFile) : session.create();
+        try (OutputStream os = session.write(output)) {
+            String query = getQuery(context, session, flowFile);
+            long startTimeMillis = System.currentTimeMillis();
+
+            os.write("[".getBytes());
+            Map<String, String> resultAttrs = clientService.executeQuery(query, getParameters(context, output), (record, hasMore) -> {
+                try {
+                    String obj = mapper.writeValueAsString(record);
+                    os.write(obj.getBytes());
+                    if (hasMore) {
+                        os.write(",".getBytes());
+                    }
+                } catch (Exception ex) {
+                    throw new ProcessException(ex);
+                }
+            });
+            os.write("]".getBytes());
+            os.close();
+
+            final long endTimeMillis = System.currentTimeMillis();
+
+            String executionTime = String.valueOf((endTimeMillis - startTimeMillis));
+            resultAttrs.put(EXECUTION_TIME, executionTime);
+            resultAttrs.put(CoreAttributes.MIME_TYPE.key(), "application/json");
+            output = session.putAllAttributes(output, resultAttrs);
+            session.transfer(output, REL_SUCCESS);
+            session.getProvenanceReporter().invokeRemoteProcess(output, clientService.getTransitUrl(),
+                String.format("The following query was executed in %s milliseconds: \"%s\"", executionTime, query)
+            );
+            if (flowFile != null) {
+                session.transfer(flowFile, REL_ORIGINAL);
+            }
+
+        } catch (Exception exception) {
+            getLogger().error("Failed to execute graph statement due to {}",
+                    new Object[]{exception.getLocalizedMessage()}, exception);
+            session.remove(output);
+            if (flowFile != null) {
+                flowFile = session.putAttribute(flowFile, ERROR_MESSAGE, String.valueOf(exception.getMessage()));
+                session.transfer(flowFile, REL_FAILURE);
+            }
+            context.yield();
+        }
+    }
+
+    protected String getQuery(ProcessContext context, ProcessSession session, FlowFile input) {
+        String query = context.getProperty(QUERY).evaluateAttributeExpressions(input).getValue();
+        if (StringUtils.isEmpty(query) && input != null) {
+            try {
+                if (input.getSize() > (64 * 1024)) {
+                    throw new Exception("Input bigger than 64kb. Cannot assume this is a valid query for Gremlin Server " +
+                            "or Neo4J.");
+                    /*
+                     * Note: Gremlin Server compiles the query down to Java byte code, and queries 64kb and above often
+                     * bounce because they violate Java method size limits. We might want to revist this, but as a starting
+                     * point, it is a sane default to assume that if a flowfile is bigger than about 64kb it is either not a
+                     * query or it is a query likely to be a very poor fit here.
+                     */
+                }
+
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                session.exportTo(input, out);
+                out.close();
+
+
+                query = new String(out.toByteArray());
+            } catch (Exception ex) {
+                throw new ProcessException(ex);
+            }
+        }
+        return query;
+    }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
similarity index 93%
copy from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
copy to nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index bacbcfb..615c785 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -12,4 +12,4 @@
 # 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.
-org.apache.nifi.processors.neo4j.Neo4JCypherExecutor
+org.apache.nifi.processors.graph.ExecuteGraphQuery
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/resources/docs/org.apache.nifi.processors.graph.ExecuteGraphQuery/additionalDetails.html b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/resources/docs/org.apache.nifi.processors.graph.ExecuteGraphQuery/additionalDetails.html
new file mode 100644
index 0000000..fd2433f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/main/resources/docs/org.apache.nifi.processors.graph.ExecuteGraphQuery/additionalDetails.html
@@ -0,0 +1,37 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>ExecuteGraphQuery</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+
+<body>
+<!-- Processor Documentation ================================================== -->
+<h2>Description:</h2>
+<p>
+    This processor is designed to work with Gremlin and Cypher queries. The query is specified in the configuration parameter labeled
+    Query, and parameters can be configured using dynamic properties on the processor. All Gremlin and Cypher CRUD operations are
+    supported by this processor. It will stream the entire result set into a single flowfile as a JSON array.
+</p>
+<h2>More Information</h2>
+<ul>
+    <li><a href="https://neo4j.com/developer/cypher-query-language/">Cypher Query Language introduction</a>.</li>
+    <li><a href="http://tinkerpop.apache.org/docs/current/reference/#connecting-gremlin">Gremlin Query DSL documentation</a>.</li>
+</ul>
+</body>
+</html>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/test/java/org/apache/nifi/processors/graph/MockCypherClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/test/java/org/apache/nifi/processors/graph/MockCypherClientService.java
new file mode 100644
index 0000000..81df34a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/test/java/org/apache/nifi/processors/graph/MockCypherClientService.java
@@ -0,0 +1,56 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.graph.GraphQueryResultCallback;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class MockCypherClientService extends AbstractControllerService implements GraphClientService {
+
+    @Override
+    public Map<String, String> executeQuery(String query, Map<String, Object> parameters, GraphQueryResultCallback handler) {
+        handler.process(new HashMap<String, Object>(){{
+            put("name", "John Smith");
+            put("age", 40);
+        }}, true);
+        handler.process(new HashMap<String, Object>(){{
+            put("name", "John Smith");
+            put("age", 40);
+        }}, false);
+
+        Map<String,String> resultAttributes = new HashMap<>();
+        resultAttributes.put(NODES_CREATED,String.valueOf(1));
+        resultAttributes.put(RELATIONS_CREATED,String.valueOf(1));
+        resultAttributes.put(LABELS_ADDED,String.valueOf(1));
+        resultAttributes.put(NODES_DELETED,String.valueOf(1));
+        resultAttributes.put(RELATIONS_DELETED,String.valueOf(1));
+        resultAttributes.put(PROPERTIES_SET, String.valueOf(1));
+        resultAttributes.put(ROWS_RETURNED, String.valueOf(1));
+
+        return resultAttributes;
+    }
+
+    @Override
+    public String getTransitUrl() {
+        return "mock://localhost:12345/fake_database";
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/test/java/org/apache/nifi/processors/graph/TestExecuteGraphQuery.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/test/java/org/apache/nifi/processors/graph/TestExecuteGraphQuery.java
new file mode 100644
index 0000000..cdcd112
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/test/java/org/apache/nifi/processors/graph/TestExecuteGraphQuery.java
@@ -0,0 +1,101 @@
+/*
+ * 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.graph;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.graph.GraphClientService;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Cypher unit tests.
+ */
+public class TestExecuteGraphQuery {
+    protected TestRunner runner;
+
+    @Rule public MockitoRule mockitoRule = MockitoJUnit.rule();
+
+    @Before
+    public void setUp() throws Exception {
+        MockCypherClientService service = new MockCypherClientService();
+        runner = TestRunners.newTestRunner(ExecuteGraphQuery.class);
+        runner.addControllerService("clientService", service);
+        runner.enableControllerService(service);
+        runner.setProperty(AbstractGraphExecutor.CLIENT_SERVICE, "clientService");
+        runner.setProperty(AbstractGraphExecutor.QUERY, "match (n) return n");
+    }
+
+    @Test
+    public void testExecuteFromParameter() throws Exception {
+        runner.setProperty(AbstractGraphExecutor.QUERY, "MATCH (p:person) RETURN p");
+        testExecute(1, 0, 0);
+    }
+
+    @Test
+    public void testExecuteFromBody() throws Exception {
+        runner.enqueue("MATCH (p:person) RETURN p");
+        testExecute(1, 0, 1);
+    }
+
+    @Test
+    public void testExecuteFromParameterWithEL() throws Exception {
+        runner.setProperty(AbstractGraphExecutor.QUERY, "${query}");
+        runner.enqueue("test-data", new HashMap<String, String>(){{
+            put("query", "MATCH (p:person) RETURN p");
+        }});
+        testExecute(1, 0, 1);
+    }
+
+    private void testExecute(int success, int failure, int original) throws Exception {
+        runner.run(1,true,true);
+        runner.assertTransferCount(ExecuteGraphQuery.REL_SUCCESS, success);
+        runner.assertTransferCount(ExecuteGraphQuery.REL_FAILURE, failure);
+        runner.assertTransferCount(ExecuteGraphQuery.REL_ORIGINAL, original);
+        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ExecuteGraphQuery.REL_SUCCESS);
+        assertEquals("1",flowFiles.get(0).getAttribute(GraphClientService.LABELS_ADDED));
+        assertEquals("1",flowFiles.get(0).getAttribute(GraphClientService.NODES_CREATED));
+        assertEquals("1",flowFiles.get(0).getAttribute(GraphClientService.NODES_DELETED));
+        assertEquals("1",flowFiles.get(0).getAttribute(GraphClientService.RELATIONS_CREATED));
+        assertEquals("1",flowFiles.get(0).getAttribute(GraphClientService.RELATIONS_DELETED));
+        assertEquals("1",flowFiles.get(0).getAttribute(GraphClientService.PROPERTIES_SET));
+        assertEquals("1",flowFiles.get(0).getAttribute(GraphClientService.ROWS_RETURNED));
+        byte[] raw = runner.getContentAsByteArray(flowFiles.get(0));
+        String str = new String(raw);
+        List<Map<String, Object>> parsed = new ObjectMapper().readValue(str, List.class);
+        assertNotNull(parsed);
+        assertEquals(2, parsed.size());
+        for (Map<String, Object> result : parsed) {
+            assertEquals(2, result.size());
+            assertTrue(result.containsKey("name"));
+            assertTrue(result.containsKey("age"));
+        }
+    }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/pom.xml
new file mode 100644
index 0000000..c89b8b2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/pom.xml
@@ -0,0 +1,37 @@
+<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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-neo4j-cypher-service-nar</artifactId>
+    <packaging>nar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-graph-client-service-api-nar</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-neo4j-cypher-service</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/src/main/resources/META-INF/LICENSE
similarity index 78%
copy from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE
copy to nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/src/main/resources/META-INF/LICENSE
index d645695..3f5fc79 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/src/main/resources/META-INF/LICENSE
@@ -200,3 +200,66 @@
    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.
+
+
+************
+MIT
+************
+
+  	This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+  	a temporary self-signed X.509 certificate when the JVM does not provide the
+  	equivalent functionality.  It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.bouncycastle.txt (MIT License)
+  	  * HOMEPAGE:
+  	    * http://www.bouncycastle.org/
+
+This product bundles 'jBCrypt' which is available under an MIT license.
+For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE
+
+    Copyright (c) 2006 Damien Miller <dj...@mindrot.org>
+
+    Permission to use, copy, modify, and distribute this software for any
+    purpose with or without fee is hereby granted, provided that the above
+    copyright notice and this permission notice appear in all copies.
+
+    THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
+    WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
+    MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
+    ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
+    WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
+    ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
+    OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+
+
+
+  The binary distribution of this product bundles 'ParaNamer' and 'Paranamer Core'
+  which is available under a BSD style license.
+
+    Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc
+     All rights reserved.
+
+     Redistribution and use in source and binary forms, with or without
+     modification, are permitted provided that the following conditions
+     are met:
+     1. Redistributions of source code must retain the above copyright
+        notice, this list of conditions and the following disclaimer.
+     2. Redistributions in binary form must reproduce the above copyright
+        notice, this list of conditions and the following disclaimer in the
+        documentation and/or other materials provided with the distribution.
+     3. Neither the name of the copyright holders nor the names of its
+        contributors may be used to endorse or promote products derived from
+        this software without specific prior written permission.
+
+     THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+     AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+     IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+     ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+     LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+     CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+     SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+     INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+     CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+     ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+     THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..bb2b72c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,152 @@
+nifi-cypher-services-nar
+Copyright 2017-2018 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache Commons Compress
+    The following NOTICE information applies:
+      Apache Commons Compress
+      Copyright 2002-2017 The Apache Software Foundation
+
+      The files in the package org.apache.commons.compress.archivers.sevenz
+      were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/),
+      which has been placed in the public domain:
+
+      "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
+
+  (ASLv2) Apache Commons Configuration
+    The following NOTICE information applies:
+      Apache Commons Configuration
+      Copyright 2001-2017 The Apache Software Foundation
+
+      This product includes software developed at
+      The Apache Software Foundation (http://www.apache.org/).
+
+  (ASLv2) Apache Commons CSV
+    The following NOTICE information applies:
+      Apache Commons CSV
+      Copyright 2005-2016 The Apache Software Foundation
+
+      This product includes software developed at
+      The Apache Software Foundation (http://www.apache.org/).
+
+  (ASLv2) Apache Commons BeanUtils
+    The following NOTICE information applies:
+      Apache Commons BeanUtils
+      Copyright 2000-2016 The Apache Software Foundation
+
+      This product includes software developed at
+      The Apache Software Foundation (http://www.apache.org/).
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+      This product includes software from the Spring Framework,
+      under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+  (ASLv2) Apache Commons Codec
+    The following NOTICE information applies:
+      Apache Commons Codec
+      Copyright 2002-2014 The Apache Software Foundation
+
+      src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
+      contains test data from http://aspell.net/test/orig/batch0.tab.
+      Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+
+      ===============================================================================
+
+      The content of package org.apache.commons.codec.language.bm has been translated
+      from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+      with permission from the original authors.
+      Original source copyright:
+      Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+  (ASLv2) Apache Commons Logging
+    The following NOTICE information applies:
+      Apache Commons Logging
+      Copyright 2003-2013 The Apache Software Foundation
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Text
+    The following NOTICE information applies:
+      Apache Commons Text
+      Copyright 2001-2018 The Apache Software Foundation
+
+  (ASLv2) Caffeine
+    The following NOTICE information applies:
+      Caffeine (caching library)
+      Copyright Ben Manes
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
+
+  (ASL2 License)
+    (ASLv2) Neo4j Java Driver
+      Neo4j
+      Copyright © 20022018 Neo4j Sweden AB (referred to in this notice as "Neo4j")
+      [http://neo4j.com]
+
+
+  (ASLv2) Snappy Java
+    The following NOTICE information applies:
+      This product includes software developed by Google
+       Snappy: http://code.google.com/p/snappy/ (New BSD License)
+
+      This product includes software developed by Apache
+       PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/
+       (Apache 2.0 license)
+
+      This library containd statically linked libstdc++. This inclusion is allowed by
+      "GCC RUntime Library Exception"
+      http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
+
+
+*****************
+Public Domain
+*****************
+
+The following binary components are provided to the 'Public Domain'.  See project link for details.
+
+    (Public Domain) XZ for Java (org.tukaani:xz:jar:1.5 - http://tukaani.org/xz/java.html
+
+
+************************
+Creative Commons Attribution-ShareAlike 3.0
+************************
+
+The following binary components are provided under the Creative Commons Attribution-ShareAlike 3.0.  See project link for details.
+
+	(CCAS 3.0) MaxMind DB (https://github.com/maxmind/MaxMind-DB)
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/pom.xml
new file mode 100644
index 0000000..90b640d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/pom.xml
@@ -0,0 +1,104 @@
+<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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!-- 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. -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-neo4j-cypher-service</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-lookup-service-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-graph-client-service-api</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-avro-record-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-registry-service-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-json-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.neo4j.driver</groupId>
+            <artifactId>neo4j-java-driver</artifactId>
+            <version>1.6.2</version>
+            <scope>compile</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes combine.children="append">
+                        <exclude>src/test/resources/simple.avsc</exclude>
+                        <exclude>src/test/resources/test.csv</exclude>
+                        <exclude>src/test/resources/test.properties</exclude>
+                        <exclude>src/test/resources/test.xml</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/java/org/apache/nifi/processors/neo4j/AbstractNeo4JCypherExecutor.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/main/java/org/apache/nifi/graph/Neo4JCypherClientService.java
similarity index 59%
rename from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/java/org/apache/nifi/processors/neo4j/AbstractNeo4JCypherExecutor.java
rename to nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/main/java/org/apache/nifi/graph/Neo4JCypherClientService.java
index 6e551a3..643bf29 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/java/org/apache/nifi/processors/neo4j/AbstractNeo4JCypherExecutor.java
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/main/java/org/apache/nifi/graph/Neo4JCypherClientService.java
@@ -14,48 +14,46 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.processors.neo4j;
 
-import java.io.File;
-import java.util.concurrent.TimeUnit;
-
-import javax.net.ssl.SSLContext;
+package org.apache.nifi.graph;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
-import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.ssl.SSLContextService;
+import org.neo4j.driver.internal.InternalNode;
 import org.neo4j.driver.v1.AuthTokens;
 import org.neo4j.driver.v1.Config;
-import org.neo4j.driver.v1.Config.ConfigBuilder;
-import org.neo4j.driver.v1.Config.LoadBalancingStrategy;
-import org.neo4j.driver.v1.Config.TrustStrategy;
 import org.neo4j.driver.v1.Driver;
 import org.neo4j.driver.v1.GraphDatabase;
+import org.neo4j.driver.v1.Record;
+import org.neo4j.driver.v1.Session;
+import org.neo4j.driver.v1.StatementResult;
+import org.neo4j.driver.v1.summary.ResultSummary;
+import org.neo4j.driver.v1.summary.SummaryCounters;
 
-/**
- * Abstract base class for Neo4JCypherExecutor processors
- */
-abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
-
-    protected static final PropertyDescriptor QUERY = new PropertyDescriptor.Builder()
-            .name("neo4J-query")
-            .displayName("Neo4J Query")
-            .description("Specifies the Neo4j Query.")
-            .required(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
-            .build();
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
+@Tags({ "graph", "neo4j", "cypher" })
+@CapabilityDescription("Provides a client service for managing connections to a Neo4J database. Configuration information for " +
+        "the Neo4J driver that corresponds to most of the settings for this service can be found here: " +
+        "https://neo4j.com/docs/driver-manual/current/client-applications/#driver-configuration")
+public class Neo4JCypherClientService extends AbstractControllerService implements GraphClientService {
     public static final PropertyDescriptor CONNECTION_URL = new PropertyDescriptor.Builder()
             .name("neo4j-connection-url")
             .displayName("Neo4j Connection URL")
@@ -83,10 +81,9 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             .sensitive(true)
             .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
             .build();
+    public static AllowableValue LOAD_BALANCING_STRATEGY_ROUND_ROBIN = new AllowableValue(Config.LoadBalancingStrategy.ROUND_ROBIN.name(), "Round Robin", "Round Robin Strategy");
 
-    public static AllowableValue LOAD_BALANCING_STRATEGY_ROUND_ROBIN = new AllowableValue(LoadBalancingStrategy.ROUND_ROBIN.name(), "Round Robin", "Round Robin Strategy");
-
-    public static AllowableValue LOAD_BALANCING_STRATEGY_LEAST_CONNECTED = new AllowableValue(LoadBalancingStrategy.LEAST_CONNECTED.name(), "Least Connected", "Least Connected Strategy");
+    public static AllowableValue LOAD_BALANCING_STRATEGY_LEAST_CONNECTED = new AllowableValue(Config.LoadBalancingStrategy.LEAST_CONNECTED.name(), "Least Connected", "Least Connected Strategy");
 
     protected static final PropertyDescriptor LOAD_BALANCING_STRATEGY = new PropertyDescriptor.Builder()
             .name("neo4j-load-balancing-strategy")
@@ -104,6 +101,7 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             .defaultValue("5 seconds")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(false)
             .build();
 
@@ -114,6 +112,7 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             .defaultValue("100")
             .required(true)
             .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(false)
             .build();
 
@@ -124,6 +123,7 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             .defaultValue("60 second")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(false)
             .build();
 
@@ -134,6 +134,7 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             .defaultValue("60 seconds")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(false)
             .build();
 
@@ -144,6 +145,7 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             .defaultValue("3600 seconds")
             .required(true)
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .sensitive(false)
             .build();
 
@@ -166,98 +168,55 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             .identifiesControllerService(SSLContextService.class)
             .build();
 
-    
-//    public static AllowableValue TRUST_SYSTEM_CA_SIGNED_CERTIFICATES =
-//        new AllowableValue(TrustStrategy.Strategy.TRUST_SYSTEM_CA_SIGNED_CERTIFICATES.name(),
-//            "Trust System CA Signed Certificates", "Trust system specified CA signed certificates");
-//
-//    public static AllowableValue TRUST_CUSTOM_CA_SIGNED_CERTIFICATES =
-//        new AllowableValue(TrustStrategy.Strategy.TRUST_CUSTOM_CA_SIGNED_CERTIFICATES.name(),
-//            "Trust Custom CA Signed Certificates", "Trust custom CA signed certificates defined in the file");
-//
-//    public static AllowableValue TRUST_ALL_CERTIFICATES =
-//        new AllowableValue(TrustStrategy.Strategy.TRUST_ALL_CERTIFICATES.name(),
-//            "Trust All Certificates", "Trust all certificate");
-//
-//    protected static final PropertyDescriptor TRUST_STRATEGY = new PropertyDescriptor.Builder()
-//            .name("neo4j-trust-strategy")
-//            .displayName("Trust Strategy")
-//            .description("Trust Strategy (Trust All Certificates, System CA Signed Certificates or Custom CA Signed Certificates)")
-//            .required(false)
-//            .defaultValue(TRUST_ALL_CERTIFICATES.getValue())
-//            .allowableValues(TRUST_ALL_CERTIFICATES, TRUST_SYSTEM_CA_SIGNED_CERTIFICATES, TRUST_CUSTOM_CA_SIGNED_CERTIFICATES)
-//            .build();
-//
-//    protected static final PropertyDescriptor TRUST_CUSTOM_CA_SIGNED_CERTIFICATES_FILE = new PropertyDescriptor.Builder()
-//            .name("neo4j-custom-ca-strategy-certificates-file")
-//            .displayName("Custom Trust CA Signed Certificates File")
-//            .description("Custom file containing CA signed certificates to be trusted.")
-//            .required(false)
-//            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
-//            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
-//            .build();
-
-    static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
-            .description("Sucessful FlowFiles are routed to this relationship").build();
-
-    static final Relationship REL_FAILURE = new Relationship.Builder().name("failure")
-            .description("Failed FlowFiles are routed to this relationship").build();
-
-    public static final String ERROR_MESSAGE = "neo4j.error.message";
-    public static final String NODES_CREATED= "neo4j.nodes.created";
-    public static final String RELATIONS_CREATED = "neo4j.relations.created";
-    public static final String LABELS_ADDED = "neo4j.labels.added";
-    public static final String NODES_DELETED = "neo4j.nodes.deleted";
-    public static final String RELATIONS_DELETED = "neo4j.relations.deleted";
-    public static final String PROPERTIES_SET = "neo4j.properties.set";
-    public static final String ROWS_RETURNED = "neo4j.rows.returned";
-
     protected Driver neo4JDriver;
     protected String username;
     protected String password;
     protected String connectionUrl;
-    protected Integer port;
-    protected LoadBalancingStrategy loadBalancingStrategy;
 
-    /**
-     * Helper method to help testability
-     * @return Driver instance
-     */
-    protected Driver getNeo4JDriver() {
-        return neo4JDriver;
+    private static final List<PropertyDescriptor> DESCRIPTORS;
+    static {
+        List<PropertyDescriptor> _temp = new ArrayList<>();
+        _temp.add(CONNECTION_URL);
+        _temp.add(USERNAME);
+        _temp.add(PASSWORD);
+        _temp.add(LOAD_BALANCING_STRATEGY);
+        _temp.add(CONNECTION_TIMEOUT);
+        _temp.add(MAX_CONNECTION_POOL_SIZE);
+        _temp.add(MAX_CONNECTION_ACQUISITION_TIMEOUT);
+        _temp.add(IDLE_TIME_BEFORE_CONNECTION_TEST);
+        _temp.add(MAX_CONNECTION_LIFETIME);
+        _temp.add(ENCRYPTION);
+        _temp.add(SSL_CONTEXT_SERVICE);
+
+        DESCRIPTORS = Collections.unmodifiableList(_temp);
     }
 
-    @OnScheduled
-    public void onScheduled(final ProcessContext context) {
-        try {
-            neo4JDriver = getDriver(context);
-        } catch(Exception e) {
-            getLogger().error("Error while getting connection " + e.getLocalizedMessage(),e);
-            throw new ProcessException("Error while getting connection" + e.getLocalizedMessage(),e);
-        }
-        getLogger().info("Neo4JCypherExecutor connection created for url {}",
-                new Object[] {connectionUrl});
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
     }
 
-    protected Driver getDriver(ProcessContext context) {
+    protected Driver getDriver(ConfigurationContext context) {
         connectionUrl = context.getProperty(CONNECTION_URL).evaluateAttributeExpressions().getValue();
         username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
         password = context.getProperty(PASSWORD).getValue();
 
-        ConfigBuilder configBuilder = Config.build();
+        Config.ConfigBuilder configBuilder = Config.build();
         String loadBalancingStrategyValue = context.getProperty(LOAD_BALANCING_STRATEGY).getValue();
         if ( ! StringUtils.isBlank(loadBalancingStrategyValue) ) {
-             configBuilder = configBuilder.withLoadBalancingStrategy(
-                     LoadBalancingStrategy.valueOf(loadBalancingStrategyValue));
+            configBuilder = configBuilder.withLoadBalancingStrategy(
+                    Config.LoadBalancingStrategy.valueOf(loadBalancingStrategyValue));
         }
 
-        configBuilder.withMaxConnectionPoolSize(context.getProperty(MAX_CONNECTION_POOL_SIZE).asInteger());
+        configBuilder.withMaxConnectionPoolSize(context.getProperty(MAX_CONNECTION_POOL_SIZE).evaluateAttributeExpressions().asInteger());
+
+        configBuilder.withConnectionTimeout(context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS), TimeUnit.SECONDS);
 
-        configBuilder.withConnectionAcquisitionTimeout(context.getProperty(CONNECTION_TIMEOUT).asTimePeriod(TimeUnit.SECONDS), TimeUnit.SECONDS);
+        configBuilder.withConnectionAcquisitionTimeout(context.getProperty(MAX_CONNECTION_ACQUISITION_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS), TimeUnit.SECONDS);
 
-        configBuilder.withMaxConnectionLifetime(context.getProperty(MAX_CONNECTION_ACQUISITION_TIMEOUT).asTimePeriod(TimeUnit.SECONDS), TimeUnit.SECONDS);
+        configBuilder.withMaxConnectionLifetime(context.getProperty(MAX_CONNECTION_LIFETIME).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS), TimeUnit.SECONDS);
 
-        configBuilder.withConnectionLivenessCheckTimeout(context.getProperty(IDLE_TIME_BEFORE_CONNECTION_TEST).asTimePeriod(TimeUnit.SECONDS), TimeUnit.SECONDS);
+        configBuilder.withConnectionLivenessCheckTimeout(context.getProperty(IDLE_TIME_BEFORE_CONNECTION_TEST).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS), TimeUnit.SECONDS);
 
         if ( context.getProperty(ENCRYPTION).asBoolean() ) {
             configBuilder.withEncryption();
@@ -267,30 +226,39 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
 
         final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
         if (sslService != null) {
-          if ( sslService.isTrustStoreConfigured()) {
-              configBuilder.withTrustStrategy(TrustStrategy.trustCustomCertificateSignedBy(new File(
-            		  sslService.getTrustStoreFile())));
-          } else {
-              configBuilder.withTrustStrategy(TrustStrategy.trustSystemCertificates());
-          } 
+            if ( sslService.isTrustStoreConfigured()) {
+                configBuilder.withTrustStrategy(Config.TrustStrategy.trustCustomCertificateSignedBy(new File(
+                        sslService.getTrustStoreFile())));
+            } else {
+                configBuilder.withTrustStrategy(Config.TrustStrategy.trustSystemCertificates());
+            }
         }
-//        PropertyValue trustStrategy = context.getProperty(TRUST_STRATEGY);
-//        if ( trustStrategy.isSet() ) {
-//            if ( trustStrategy.getValue().equals(TRUST_CUSTOM_CA_SIGNED_CERTIFICATES.getValue())) {
-//                configBuilder.withTrustStrategy(TrustStrategy.trustCustomCertificateSignedBy(new File(
-//                    context.getProperty(TRUST_CUSTOM_CA_SIGNED_CERTIFICATES_FILE).getValue())));
-//            } else if ( trustStrategy.getValue().equals(TRUST_SYSTEM_CA_SIGNED_CERTIFICATES.getValue())) {
-//                configBuilder.withTrustStrategy(TrustStrategy.trustSystemCertificates());
-//            } else if ( trustStrategy.getValue().equals(TRUST_ALL_CERTIFICATES.getValue())) {
-//                configBuilder.withTrustStrategy(TrustStrategy.trustAllCertificates());
-//            }
-//        }
 
         return GraphDatabase.driver( connectionUrl, AuthTokens.basic( username, password),
-                 configBuilder.toConfig());
+                configBuilder.toConfig());
     }
 
-    @OnStopped
+    /**
+     * Helper method to help testability
+     * @return Driver instance
+     */
+    protected Driver getNeo4JDriver() {
+        return neo4JDriver;
+    }
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        try {
+            neo4JDriver = getDriver(context);
+        } catch(Exception e) {
+            getLogger().error("Error while getting connection " + e.getLocalizedMessage(),e);
+            throw new ProcessException("Error while getting connection" + e.getLocalizedMessage(),e);
+        }
+        getLogger().info("Neo4JCypherExecutor connection created for url {}",
+                new Object[] {connectionUrl});
+    }
+
+    @OnDisabled
     public void close() {
         getLogger().info("Closing driver");
         if ( neo4JDriver != null ) {
@@ -298,4 +266,52 @@ abstract class AbstractNeo4JCypherExecutor extends AbstractProcessor {
             neo4JDriver = null;
         }
     }
-}
\ No newline at end of file
+
+    private Map<String, Object> handleInternalNode(Map<String, Object> recordMap) {
+        if (recordMap.size() == 1) {
+            String key = recordMap.keySet().iterator().next();
+            Object value = recordMap.get(key);
+            if (value instanceof InternalNode) {
+                return ((InternalNode)value).asMap();
+            }
+        }
+
+        return recordMap;
+    }
+
+    @Override
+    public Map<String, String> executeQuery(String query, Map<String, Object> parameters, GraphQueryResultCallback handler) {
+        try (Session session = neo4JDriver.session()) {
+            StatementResult result = session.run(query, parameters);
+            long count = 0;
+            while (result.hasNext()) {
+                Record record = result.next();
+                Map<String, Object> asMap = handleInternalNode(record.asMap());
+                handler.process(asMap, result.hasNext());
+                count++;
+            }
+
+            ResultSummary summary = result.summary();
+            SummaryCounters counters = summary.counters();
+
+            Map<String,String> resultAttributes = new HashMap<>();
+            resultAttributes.put(NODES_CREATED,String.valueOf(counters.nodesCreated()));
+            resultAttributes.put(RELATIONS_CREATED,String.valueOf(counters.relationshipsCreated()));
+            resultAttributes.put(LABELS_ADDED,String.valueOf(counters.labelsAdded()));
+            resultAttributes.put(NODES_DELETED,String.valueOf(counters.nodesDeleted()));
+            resultAttributes.put(RELATIONS_DELETED,String.valueOf(counters.relationshipsDeleted()));
+            resultAttributes.put(PROPERTIES_SET, String.valueOf(counters.propertiesSet()));
+            resultAttributes.put(ROWS_RETURNED, String.valueOf(count));
+
+            return resultAttributes;
+        } catch (Exception ex) {
+            getLogger().error("", ex);
+            throw new ProcessException(ex);
+        }
+    }
+
+    @Override
+    public String getTransitUrl() {
+        return null;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
similarity index 93%
copy from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
copy to nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
index bacbcfb..d5132cf 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -12,4 +12,5 @@
 # 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.
-org.apache.nifi.processors.neo4j.Neo4JCypherExecutor
+
+org.apache.nifi.graph.Neo4JCypherClientService
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/test/java/org/apache/nifi/graph/ITNeo4JCypherExecutor.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/test/java/org/apache/nifi/graph/ITNeo4JCypherExecutor.java
new file mode 100644
index 0000000..b2fa2c9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/test/java/org/apache/nifi/graph/ITNeo4JCypherExecutor.java
@@ -0,0 +1,152 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.neo4j.driver.v1.AuthTokens;
+import org.neo4j.driver.v1.Driver;
+import org.neo4j.driver.v1.GraphDatabase;
+import org.neo4j.driver.v1.Session;
+import org.neo4j.driver.v1.StatementResult;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Neo4J Cypher integration tests.  Please set the neo4j url, user and password according to your setup.
+ */
+public class ITNeo4JCypherExecutor {
+    protected TestRunner runner;
+    protected Driver driver;
+    protected String neo4jUrl = "bolt://localhost:7687";
+    protected String user = "neo4j";
+    protected String password = "testing1234";
+
+    private GraphClientService clientService;
+    private GraphQueryResultCallback EMPTY_CALLBACK = (record, hasMore) -> {};
+
+    @Before
+    public void setUp() throws Exception {
+        clientService = new Neo4JCypherClientService();
+        runner = TestRunners.newTestRunner(MockProcessor.class);
+        runner.addControllerService("clientService", clientService);
+        runner.setProperty(clientService, Neo4JCypherClientService.USERNAME, user);
+        runner.setProperty(clientService, Neo4JCypherClientService.PASSWORD, password);
+        runner.enableControllerService(clientService);
+        runner.setProperty(MockProcessor.CLIENT, "clientService");
+        driver = GraphDatabase.driver(neo4jUrl, AuthTokens.basic(user, password));
+        executeSession("match (n) detach delete n");
+
+        StatementResult result = executeSession("match (n) return n");
+
+        assertEquals("nodes should be equal", 0, result.list().size());
+    }
+
+    protected StatementResult executeSession(String statement) {
+        try (Session session = driver.session()) {
+            return session.run(statement);
+        }
+    }
+
+    @After
+    public void tearDown() {
+        runner = null;
+        if (driver != null) {
+            driver.close();
+        }
+        driver = null;
+    }
+
+    @Test
+    public void testCreateNodeNoReturn() {
+        String query = "create (n:nodereturn { name: \"Testing\"})";
+
+        Map<String, String> attributes = clientService.executeQuery(query, new HashMap<>(), EMPTY_CALLBACK);
+        assertEquals("1",attributes.get(GraphClientService.LABELS_ADDED));
+        assertEquals("1",attributes.get(GraphClientService.NODES_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.NODES_DELETED));
+        assertEquals("0",attributes.get(GraphClientService.RELATIONS_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.RELATIONS_DELETED));
+        assertEquals("1",attributes.get(GraphClientService.PROPERTIES_SET));
+        assertEquals("0",attributes.get(GraphClientService.ROWS_RETURNED));
+    }
+
+    @Test
+    public void testCreateNodeOnePropertyWithReturn() {
+        String query = "create (n { name:'abc' }) return n.name";
+
+        final List<Map<String, Object>> result = new ArrayList<>();
+        Map<String, String> attributes = clientService.executeQuery(query, new HashMap<>(), (record, hasMore) -> result.add(record));
+        assertEquals("0",attributes.get(GraphClientService.LABELS_ADDED));
+        assertEquals("1",attributes.get(GraphClientService.NODES_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.NODES_DELETED));
+        assertEquals("0",attributes.get(GraphClientService.RELATIONS_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.RELATIONS_DELETED));
+        assertEquals("1",attributes.get(GraphClientService.PROPERTIES_SET));
+        assertEquals("1",attributes.get(GraphClientService.ROWS_RETURNED));
+        assertEquals(1, result.size());
+        assertEquals("abc", result.get(0).get("n.name"));
+    }
+
+    @Test
+    public void testCreateNodeTwoPropertyOneLabelWithReturn() {
+        String query = "create (n:Person { name:'abc', age : 1 }) return n.name, n.age";
+
+        final List<Map<String, Object>> result = new ArrayList<>();
+        Map<String, String> attributes = clientService.executeQuery(query, new HashMap<>(), (record, hasMore) -> {
+            result.add(record);
+        });
+
+        assertEquals("1",attributes.get(GraphClientService.LABELS_ADDED));
+        assertEquals("1",attributes.get(GraphClientService.NODES_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.NODES_DELETED));
+        assertEquals("0",attributes.get(GraphClientService.RELATIONS_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.RELATIONS_DELETED));
+        assertEquals("2",attributes.get(GraphClientService.PROPERTIES_SET));
+        assertEquals("1",attributes.get(GraphClientService.ROWS_RETURNED));
+        assertEquals(1, result.size());
+        assertEquals("abc", result.get(0).get("n.name"));
+        assertEquals(1l, result.get(0).get("n.age"));
+    }
+
+    @Test
+    public void testCreateTwoNodeTwoPropertyOneRelationshipWithReturn() {
+        String query = "create (m:Person { name:'abc', age : 1 }) create (n:Person { name : 'pqr'}) create (m)-[r:hello]->(n) return m.name, n.name, type(r)";
+
+        List<Map<String, Object>> result = new ArrayList<>();
+        Map<String, String> attributes = clientService.executeQuery(query, new HashMap<>(), ((record, isMore) -> result.add(record)));
+        assertEquals("2",attributes.get(GraphClientService.LABELS_ADDED));
+        assertEquals("2",attributes.get(GraphClientService.NODES_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.NODES_DELETED));
+        assertEquals("1",attributes.get(GraphClientService.RELATIONS_CREATED));
+        assertEquals("0",attributes.get(GraphClientService.RELATIONS_DELETED));
+        assertEquals("3",attributes.get(GraphClientService.PROPERTIES_SET));
+        assertEquals("1",attributes.get(GraphClientService.ROWS_RETURNED));
+        assertEquals(1, result.size());
+        assertEquals("abc", result.get(0).get("m.name"));
+        assertEquals("pqr", result.get(0).get("n.name"));
+        assertEquals("hello", result.get(0).get("type(r)"));
+    }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/test/java/org/apache/nifi/graph/MockProcessor.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/test/java/org/apache/nifi/graph/MockProcessor.java
new file mode 100644
index 0000000..5ae4844
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-neo4j-cypher-service/src/test/java/org/apache/nifi/graph/MockProcessor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class MockProcessor extends AbstractProcessor {
+    public static final PropertyDescriptor CLIENT = new PropertyDescriptor.Builder()
+        .name("client")
+        .required(true)
+        .identifiesControllerService(GraphClientService.class)
+        .build();
+
+    public static final List<PropertyDescriptor> PROPS = Arrays.asList(CLIENT);
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPS;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/pom.xml
new file mode 100644
index 0000000..5597fec
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/pom.xml
@@ -0,0 +1,37 @@
+<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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-other-graph-services-nar</artifactId>
+    <packaging>nar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-graph-client-service-api-nar</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-other-graph-services</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/src/main/resources/META-INF/LICENSE
similarity index 80%
rename from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE
rename to nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/src/main/resources/META-INF/LICENSE
index d645695..666da22 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/src/main/resources/META-INF/LICENSE
@@ -200,3 +200,55 @@
    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.
+
+
+
+
+This product bundles 'jBCrypt' which is available under an MIT license.
+For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE
+
+    Copyright (c) 2006 Damien Miller <dj...@mindrot.org>
+
+    Permission to use, copy, modify, and distribute this software for any
+    purpose with or without fee is hereby granted, provided that the above
+    copyright notice and this permission notice appear in all copies.
+
+    THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
+    WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
+    MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
+    ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
+    WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
+    ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
+    OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
+
+
+
+  The binary distribution of this product bundles 'ParaNamer' and 'Paranamer Core'
+  which is available under a BSD style license.
+
+    Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc
+     All rights reserved.
+
+     Redistribution and use in source and binary forms, with or without
+     modification, are permitted provided that the following conditions
+     are met:
+     1. Redistributions of source code must retain the above copyright
+        notice, this list of conditions and the following disclaimer.
+     2. Redistributions in binary form must reproduce the above copyright
+        notice, this list of conditions and the following disclaimer in the
+        documentation and/or other materials provided with the distribution.
+     3. Neither the name of the copyright holders nor the names of its
+        contributors may be used to endorse or promote products derived from
+        this software without specific prior written permission.
+
+     THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+     AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+     IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+     ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+     LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+     CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+     SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+     INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+     CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+     ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+     THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000..8856f8d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,452 @@
+nifi-cypher-services-nar
+Copyright 2017-2018 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+  (ASLv2) Apache Commons Compress
+    The following NOTICE information applies:
+      Apache Commons Compress
+      Copyright 2002-2017 The Apache Software Foundation
+
+      The files in the package org.apache.commons.compress.archivers.sevenz
+      were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/),
+      which has been placed in the public domain:
+
+      "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html)
+
+  (ASLv2) Apache Commons Collections
+    The following NOTICE information applies:
+      Apache Commons Collections
+      Copyright 2001-2015 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Configuration
+    The following NOTICE information applies:
+      Apache Commons Configuration
+      Copyright 2001-2013 The Apache Software Foundation
+
+  (ASLv2) Apache Commons CSV
+    The following NOTICE information applies:
+      Apache Commons CSV
+      Copyright 2005-2016 The Apache Software Foundation
+
+      This product includes software developed at
+      The Apache Software Foundation (http://www.apache.org/).
+
+  (ASLv2) Apache Commons BeanUtils
+    The following NOTICE information applies:
+      Apache Commons BeanUtils
+      Copyright 2000-2016 The Apache Software Foundation
+
+      This product includes software developed at
+      The Apache Software Foundation (http://www.apache.org/).
+
+  (ASLv2) Apache Commons Lang
+    The following NOTICE information applies:
+      Apache Commons Lang
+      Copyright 2001-2015 The Apache Software Foundation
+
+      This product includes software from the Spring Framework,
+      under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+  (ASLv2) Apache Commons Codec
+    The following NOTICE information applies:
+      Apache Commons Codec
+      Copyright 2002-2014 The Apache Software Foundation
+
+      src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java
+      contains test data from http://aspell.net/test/orig/batch0.tab.
+      Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org)
+
+      ===============================================================================
+
+      The content of package org.apache.commons.codec.language.bm has been translated
+      from the original php source code available at http://stevemorse.org/phoneticinfo.htm
+      with permission from the original authors.
+      Original source copyright:
+      Copyright (c) 2008 Alexander Beider & Stephen P. Morse.
+
+  (ASLv2) Apache Commons Logging
+    The following NOTICE information applies:
+      Apache Commons Logging
+      Copyright 2003-2013 The Apache Software Foundation
+
+  (ASLv2) Apache Commons IO
+    The following NOTICE information applies:
+      Apache Commons IO
+      Copyright 2002-2016 The Apache Software Foundation
+
+  (ASLv2) Apache Commons Text
+    The following NOTICE information applies:
+      Apache Commons Text
+      Copyright 2001-2018 The Apache Software Foundation
+
+  (ASLv2) Apache Tinkerpop Gremlin
+    The following NOTICE information applies:
+      Gremlin-Core
+      Gremlin-Driver
+      Gremlin-Shaded
+      Copyright The Apache Software Foundation
+
+  (ASLv2) Caffeine
+    The following NOTICE information applies:
+      Caffeine (caching library)
+      Copyright Ben Manes
+
+    (ASLv2) Carrotsearch HPPC
+      The following NOTICE information applies:
+      HPPC borrowed code, ideas or both from:
+
+       * Apache Lucene, http://lucene.apache.org/
+         (Apache license)
+       * Fastutil, http://fastutil.di.unimi.it/
+         (Apache license)
+       * Koloboke, https://github.com/OpenHFT/Koloboke
+         (Apache license)
+
+  (ASLv2) Exp4j
+    Copyright 2017
+
+  (ASLv2) Groovy
+    The following NOTICE information applies:
+      Groovy Language
+         Copyright 2003-2015 The respective authors and developers
+         Developers and Contributors are listed in the project POM file
+         and Gradle build file
+
+         This product includes software developed by
+         The Groovy community (http://groovy.codehaus.org/).
+
+  (ASLv2) Jackson JSON processor
+    The following NOTICE information applies:
+      # Jackson JSON processor
+
+      Jackson is a high-performance, Free/Open Source JSON processing library.
+      It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+      been in development since 2007.
+      It is currently developed by a community of developers, as well as supported
+      commercially by FasterXML.com.
+
+      ## Licensing
+
+      Jackson core and extension components may licensed under different licenses.
+      To find the details that apply to this artifact see the accompanying LICENSE file.
+      For more information, including possible other licensing options, contact
+      FasterXML.com (http://fasterxml.com).
+
+      ## Credits
+
+      A list of contributors may be found from CREDITS file, which is included
+      in some artifacts (usually source distributions); but is always available
+      from the source code management (SCM) system project uses.
+
+  (ASLv2) JavaTuples
+     Copyright © 2012 The JAVATUPLES team. All Rights Reserved. 
+
+  (ASLv2) JCabi Components
+    Copyright (c) 2012-2017, jcabi.com
+
+  (ASLv2) JavaPoet
+    Copyright 2015 Square, Inc.
+
+
+    (ASLv2) Neo4j Java Driver
+      Neo4j
+      Copyright © 20022018 Neo4j Sweden AB (referred to in this notice as "Neo4j")
+      [http://neo4j.com]
+
+  (ASLv2) OpenCypher Cypher-for-Gremlin
+    Cypher for Gremlin is a toolkit for users of Apache TinkerPop™ that allows querying Gremlin
+    databases with Cypher, the industry's most widely used property graph query language defined
+    and maintained by the openCypher project.
+
+    Copyright 2018 Neo4J, Inc.
+
+
+  (ASLv2) Snappy Java
+    The following NOTICE information applies:
+      This product includes software developed by Google
+       Snappy: http://code.google.com/p/snappy/ (New BSD License)
+
+      This product includes software developed by Apache
+       PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/
+       (Apache 2.0 license)
+
+      This library containd statically linked libstdc++. This inclusion is allowed by
+      "GCC RUntime Library Exception"
+      http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html
+
+
+  (ASLv2) The Netty Project
+        The following NOTICE information applies:
+
+    	                            The Netty Project
+    	                            =================
+
+    	Please visit the Netty web site for more information:
+
+    	  * http://netty.io/
+
+    	Copyright 2014 The Netty Project
+
+    	The Netty Project 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.
+
+    	Also, please refer to each LICENSE.<component>.txt file, which is located in
+    	the 'license' directory of the distribution file, for the license terms of the
+    	components that this product depends on.
+
+    	-------------------------------------------------------------------------------
+    	This product contains the extensions to Java Collections Framework which has
+    	been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jsr166y.txt (Public Domain)
+    	  * HOMEPAGE:
+    	    * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
+    	    * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
+
+    	This product contains a modified version of Robert Harder's Public Domain
+    	Base64 Encoder and Decoder, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.base64.txt (Public Domain)
+    	  * HOMEPAGE:
+    	    * http://iharder.sourceforge.net/current/java/base64/
+
+    	This product contains a modified portion of 'Webbit', an event based
+    	WebSocket and HTTP server, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.webbit.txt (BSD License)
+    	  * HOMEPAGE:
+    	    * https://github.com/joewalnes/webbit
+
+    	This product contains a modified portion of 'SLF4J', a simple logging
+    	facade for Java, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.slf4j.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * http://www.slf4j.org/
+
+    	This product contains a modified portion of 'Apache Harmony', an open source
+    	Java SE, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.harmony.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://archive.apache.org/dist/harmony/
+
+    	This product contains a modified portion of 'jbzip2', a Java bzip2 compression
+    	and decompression library written by Matthew J. Francis. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jbzip2.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * https://code.google.com/p/jbzip2/
+
+    	This product contains a modified portion of 'libdivsufsort', a C API library to construct
+    	the suffix array and the Burrows-Wheeler transformed string for any input string of
+    	a constant-size alphabet written by Yuta Mori. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.libdivsufsort.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * https://github.com/y-256/libdivsufsort
+
+    	This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
+    	 which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jctools.txt (ASL2 License)
+    	  * HOMEPAGE:
+    	    * https://github.com/JCTools/JCTools
+
+    	This product optionally depends on 'JZlib', a re-implementation of zlib in
+    	pure Java, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jzlib.txt (BSD style License)
+    	  * HOMEPAGE:
+    	    * http://www.jcraft.com/jzlib/
+
+    	This product optionally depends on 'Compress-LZF', a Java library for encoding and
+    	decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.compress-lzf.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/ning/compress
+
+    	This product optionally depends on 'lz4', a LZ4 Java compression
+    	and decompression library written by Adrien Grand. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.lz4.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/jpountz/lz4-java
+
+    	This product optionally depends on 'lzma-java', a LZMA Java compression
+    	and decompression library, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.lzma-java.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/jponge/lzma-java
+
+    	This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
+    	and decompression library written by William Kinney. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jfastlz.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * https://code.google.com/p/jfastlz/
+
+    	This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
+    	interchange format, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.protobuf.txt (New BSD License)
+    	  * HOMEPAGE:
+    	    * https://github.com/google/protobuf
+
+    	This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+    	a temporary self-signed X.509 certificate when the JVM does not provide the
+    	equivalent functionality.  It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.bouncycastle.txt (MIT License)
+    	  * HOMEPAGE:
+    	    * http://www.bouncycastle.org/
+
+    	This product optionally depends on 'Snappy', a compression library produced
+    	by Google Inc, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.snappy.txt (New BSD License)
+    	  * HOMEPAGE:
+    	    * https://github.com/google/snappy
+
+    	This product optionally depends on 'JBoss Marshalling', an alternative Java
+    	serialization API, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1)
+    	  * HOMEPAGE:
+    	    * http://www.jboss.org/jbossmarshalling
+
+    	This product optionally depends on 'Caliper', Google's micro-
+    	benchmarking framework, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.caliper.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/google/caliper
+
+    	This product optionally depends on 'Apache Commons Logging', a logging
+    	framework, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.commons-logging.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://commons.apache.org/logging/
+
+    	This product optionally depends on 'Apache Log4J', a logging framework, which
+    	can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.log4j.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://logging.apache.org/log4j/
+
+    	This product optionally depends on 'Aalto XML', an ultra-high performance
+    	non-blocking XML processor, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.aalto-xml.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * http://wiki.fasterxml.com/AaltoHome
+
+    	This product contains a modified version of 'HPACK', a Java implementation of
+    	the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.hpack.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+    	    * https://github.com/twitter/hpack
+
+    	This product contains a modified portion of 'Apache Commons Lang', a Java library
+    	provides utilities for the java.lang API, which can be obtained at:
+
+    	  * LICENSE:
+    	    * license/LICENSE.commons-lang.txt (Apache License 2.0)
+    	  * HOMEPAGE:
+        * https://commons.apache.org/proper/commons-lang/
+
+        This product contains a forked and modified version of Tomcat Native
+
+          * LICENSE:
+            * ASL2
+          * HOMEPAGE:
+            * http://tomcat.apache.org/native-doc/
+            * https://svn.apache.org/repos/asf/tomcat/native/
+
+  (ASLv2) Parboiled
+    This product NOTICE information applies Parboiled components.
+     Copyright Mathias sirthias
+
+
+  (ASLv2) Scala Libraries
+    This product NOTICE information applies to several components of the Scala Programming Language.
+      Copyright (c) 2002-2019 EPFL
+      Copyright (c) 2011-2019 Lightbend, Inc.
+
+  (ASLv2) SnakeYAML
+
+
+
+************
+MIT
+************
+
+  	This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
+  	a temporary self-signed X.509 certificate when the JVM does not provide the
+  	equivalent functionality.  It can be obtained at:
+
+  	  * LICENSE:
+  	    * license/LICENSE.bouncycastle.txt (MIT License)
+  	  * HOMEPAGE:
+  	    * http://www.bouncycastle.org/
+
+*****************
+Public Domain
+*****************
+
+The following binary components are provided to the 'Public Domain'.  See project link for details.
+
+    (Public Domain) XZ for Java (org.tukaani:xz:jar:1.5 - http://tukaani.org/xz/java.html
+
+
+************************
+Creative Commons Attribution-ShareAlike 3.0
+************************
+
+The following binary components are provided under the Creative Commons Attribution-ShareAlike 3.0.  See project link for details.
+
+	(CCAS 3.0) MaxMind DB (https://github.com/maxmind/MaxMind-DB)
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/pom.xml
new file mode 100644
index 0000000..b26d0d4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/pom.xml
@@ -0,0 +1,112 @@
+<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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!-- 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. -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-graph-bundle</artifactId>
+        <version>1.10.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-other-graph-services</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-lookup-service-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-graph-client-service-api</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-avro-record-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-registry-service-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-json-utils</artifactId>
+            <version>1.10.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.opencypher.gremlin</groupId>
+            <artifactId>cypher-gremlin-neo4j-driver</artifactId>
+            <version>0.9.13</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tinkerpop</groupId>
+            <artifactId>gremlin-core</artifactId>
+            <version>3.3.5</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tinkerpop</groupId>
+            <artifactId>gremlin-driver</artifactId>
+            <version>3.3.5</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes combine.children="append">
+                        <exclude>src/test/resources/simple.avsc</exclude>
+                        <exclude>src/test/resources/test.csv</exclude>
+                        <exclude>src/test/resources/test.properties</exclude>
+                        <exclude>src/test/resources/test.xml</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/AbstractTinkerpopClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/AbstractTinkerpopClientService.java
new file mode 100644
index 0000000..4bf7bec
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/AbstractTinkerpopClientService.java
@@ -0,0 +1,109 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.tinkerpop.gremlin.driver.Cluster;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public abstract class AbstractTinkerpopClientService extends AbstractControllerService {
+    public static final PropertyDescriptor CONTACT_POINTS = new PropertyDescriptor.Builder()
+            .name("tinkerpop-contact-points")
+            .displayName("Contact Points")
+            .description("A comma-separated list of hostnames or IP addresses where an OpenCypher-enabled server can be found.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("tinkerpop-port")
+            .displayName("Port")
+            .description("The port where Gremlin Server is running on each host listed as a contact point.")
+            .required(true)
+            .defaultValue("8182")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+    public static final PropertyDescriptor PATH = new PropertyDescriptor.Builder()
+            .name("tinkerpop-path")
+            .displayName("Path")
+            .description("The URL path where Gremlin Server is running on each host listed as a contact point.")
+            .required(true)
+            .defaultValue("/gremlin")
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("tinkerpop-ssl-context-service")
+            .displayName("SSL Context Service")
+            .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
+                    + "connections.")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            CONTACT_POINTS, PORT, PATH, SSL_CONTEXT_SERVICE
+    ));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    protected Cluster.Builder setupSSL(ConfigurationContext context, Cluster.Builder builder) {
+        if (context.getProperty(SSL_CONTEXT_SERVICE).isSet()) {
+            SSLContextService service = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+            builder
+                    .enableSsl(true)
+                    .keyStore(service.getKeyStoreFile())
+                    .keyStorePassword(service.getKeyStorePassword())
+                    .keyStoreType(service.getKeyStoreType())
+                    .trustStore(service.getTrustStoreFile())
+                    .trustStorePassword(service.getTrustStorePassword());
+        }
+
+        return builder;
+    }
+
+    protected Cluster buildCluster(ConfigurationContext context) {
+        String contactProp = context.getProperty(CONTACT_POINTS).evaluateAttributeExpressions().getValue();
+        int port = context.getProperty(PORT).evaluateAttributeExpressions().asInteger();
+        String path = context.getProperty(PATH).evaluateAttributeExpressions().getValue();
+        String[] contactPoints = contactProp.split(",[\\s]*");
+        Cluster.Builder builder = Cluster.build();
+        for (String contactPoint : contactPoints) {
+            builder.addContactPoint(contactPoint.trim());
+        }
+
+        builder.port(port).path(path);
+
+        builder = setupSSL(context, builder);
+
+        return builder.create();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java
new file mode 100644
index 0000000..4b51623
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java
@@ -0,0 +1,93 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.tinkerpop.gremlin.driver.Client;
+import org.apache.tinkerpop.gremlin.driver.Cluster;
+import org.apache.tinkerpop.gremlin.driver.Result;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+@CapabilityDescription("A client service that connects to a graph database that can accept queries in the Tinkerpop Gremlin DSL.")
+@Tags({ "graph", "database", "gremlin", "tinkerpop", })
+public class GremlinClientService extends AbstractTinkerpopClientService implements TinkerPopClientService {
+    private Cluster cluster;
+    protected Client client;
+    public static final String NOT_SUPPORTED = "NOT_SUPPORTED";
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        cluster = buildCluster(context);
+        client = cluster.connect();
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        client.close();
+        cluster.close();
+        client = null;
+        cluster = null;
+    }
+
+    @Override
+    public Map<String, String> executeQuery(String query, Map<String, Object> parameters, GraphQueryResultCallback handler) {
+        try {
+            Iterator<Result> iterator = client.submit(query, parameters).iterator();
+            long count = 0;
+            while (iterator.hasNext()) {
+                Result result = iterator.next();
+                Object obj = result.getObject();
+                if (obj instanceof Map) {
+                    handler.process((Map)obj, iterator.hasNext());
+                } else {
+                    handler.process(new HashMap<String, Object>(){{
+                        put("result", obj);
+                    }}, iterator.hasNext());
+                }
+                count++;
+            }
+
+            Map<String, String> resultAttributes = new HashMap<>();
+            resultAttributes.put(NODES_CREATED, NOT_SUPPORTED);
+            resultAttributes.put(RELATIONS_CREATED, NOT_SUPPORTED);
+            resultAttributes.put(LABELS_ADDED, NOT_SUPPORTED);
+            resultAttributes.put(NODES_DELETED, NOT_SUPPORTED);
+            resultAttributes.put(RELATIONS_DELETED, NOT_SUPPORTED);
+            resultAttributes.put(PROPERTIES_SET, NOT_SUPPORTED);
+            resultAttributes.put(ROWS_RETURNED, String.valueOf(count));
+
+            return resultAttributes;
+
+        } catch (Exception ex) {
+            throw new ProcessException(ex);
+        }
+    }
+
+    @Override
+    public String getTransitUrl() {
+        return null;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/OpenCypherClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/OpenCypherClientService.java
new file mode 100644
index 0000000..4fb4699
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/OpenCypherClientService.java
@@ -0,0 +1,103 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.tinkerpop.gremlin.driver.Cluster;
+import org.neo4j.driver.internal.InternalNode;
+import org.neo4j.driver.v1.Driver;
+import org.neo4j.driver.v1.Record;
+import org.neo4j.driver.v1.Session;
+import org.neo4j.driver.v1.StatementResult;
+import org.opencypher.gremlin.neo4j.driver.GremlinDatabase;
+
+import java.util.HashMap;
+import java.util.Map;
+
+@CapabilityDescription("A client service that uses the OpenCypher implementation of the Cypher query language to connect to " +
+        "databases other than Neo4J that are on the supported list of OpenCypher-compatible products. For more information, see: " +
+        "http://www.opencypher.org/")
+@Tags({ "cypher", "opencypher", "graph", "database", "janus" })
+public class OpenCypherClientService extends AbstractTinkerpopClientService implements GraphClientService {
+    private volatile Driver gremlinDriver;
+    private volatile String transitUrl;
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        Cluster cluster = buildCluster(context);
+
+        gremlinDriver = GremlinDatabase.driver(cluster);
+        transitUrl = String.format("gremlin://%s", context.getProperty(CONTACT_POINTS).getValue());
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        gremlinDriver.close();
+    }
+
+    public static final String NOT_SUPPORTED = "NOT_SUPPORTED";
+
+    private Map<String, Object> handleInternalNode(Map<String, Object> recordMap) {
+        if (recordMap.size() == 1) {
+            String key = recordMap.keySet().iterator().next();
+            Object value = recordMap.get(key);
+            if (value instanceof InternalNode) {
+                return ((InternalNode)value).asMap();
+            }
+        }
+
+        return recordMap;
+    }
+
+    @Override
+    public Map<String, String> executeQuery(String query, Map<String, Object> parameters, GraphQueryResultCallback handler) {
+        try (Session session = gremlinDriver.session()) {
+            StatementResult result = session.run(query, parameters);
+            long count = 0;
+            while (result.hasNext()) {
+                Record record = result.next();
+                Map<String, Object> asMap = handleInternalNode(record.asMap());
+                handler.process(asMap, result.hasNext());
+                count++;
+            }
+
+            Map<String,String> resultAttributes = new HashMap<>();
+            resultAttributes.put(NODES_CREATED, NOT_SUPPORTED);
+            resultAttributes.put(RELATIONS_CREATED, NOT_SUPPORTED);
+            resultAttributes.put(LABELS_ADDED, NOT_SUPPORTED);
+            resultAttributes.put(NODES_DELETED, NOT_SUPPORTED);
+            resultAttributes.put(RELATIONS_DELETED, NOT_SUPPORTED);
+            resultAttributes.put(PROPERTIES_SET, NOT_SUPPORTED);
+            resultAttributes.put(ROWS_RETURNED, String.valueOf(count));
+
+            return resultAttributes;
+        } catch (Exception ex) {
+            throw new ProcessException(ex);
+        }
+    }
+
+    @Override
+    public String getTransitUrl() {
+        return transitUrl;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
similarity index 89%
rename from nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
rename to nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
index bacbcfb..a85b146 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -12,4 +12,6 @@
 # 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.
-org.apache.nifi.processors.neo4j.Neo4JCypherExecutor
+
+org.apache.nifi.graph.GremlinClientService
+org.apache.nifi.graph.OpenCypherClientService
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/resources/docs/org.apache.nifi.graph.GremlinClientService/additionalDetails.html b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/resources/docs/org.apache.nifi.graph.GremlinClientService/additionalDetails.html
new file mode 100644
index 0000000..33bca23
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/resources/docs/org.apache.nifi.graph.GremlinClientService/additionalDetails.html
@@ -0,0 +1,54 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+  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.
+-->
+<head>
+    <meta charset="utf-8" />
+    <title>GremlinClientService</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+
+<body>
+<!-- Processor Documentation ================================================== -->
+<h2>Description:</h2>
+<p>
+    This client service configures a connection to a Gremlin Server and allows Gremlin queries to be executed against
+    the Gremlin Server. For more information on Gremlin and Gremlin Server, see the <a href="http://tinkerpop.apache.org/">Apache Tinkerpop</a> project.
+</p>
+<h2>Warning for New Users</h2>
+<p>
+    A common issue when creating Gremlin scripts for first time users is to accidentally return an unserializable object. Gremlin
+    is a Groovy DSL and so it behaves like compiled Groovy including returning the last statement in the script. This is an example
+    of a Gremlin script that could cause unexpected failures:
+</p>
+<pre>
+    g.V().hasLabel("person").has("name", "John Smith").valueMap()
+</pre>
+<p>
+    The <em>valueMap()</em> step is not directly serializable and will fail. To fix that you have two potential options:
+</p>
+<pre>
+    //Return a Map
+    g.V().hasLabel("person").has("name", "John Smith").valueMap().next()
+</pre>
+<p>
+    Alternative:
+</p>
+<pre>
+    g.V().hasLabel("person").has("name", "John Smith").valueMap()
+    true //Return boolean literal
+</pre>
+</body>
+</html>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/GremlinClientServiceIT.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/GremlinClientServiceIT.java
new file mode 100644
index 0000000..ee8dc60
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/GremlinClientServiceIT.java
@@ -0,0 +1,75 @@
+/*
+ * 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.graph;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/*
+ * As of JanusGraph 0.3.X these tests can be a little inconsistent for a few runs at first.
+ */
+public class GremlinClientServiceIT {
+    private TestRunner runner;
+    private TestableGremlinClientService clientService;
+
+    @Before
+    public void setup() throws Exception {
+        clientService = new TestableGremlinClientService();
+        runner = TestRunners.newTestRunner(MockProcessor.class);
+        runner.addControllerService("gremlinService", clientService);
+        runner.setProperty(clientService, AbstractTinkerpopClientService.CONTACT_POINTS, "localhost");
+        runner.setProperty(MockProcessor.GREMLIN_CLIENT, "gremlinService");
+        runner.enableControllerService(clientService);
+        runner.assertValid();
+
+        String setup = IOUtils.toString(getClass().getResourceAsStream("/setup.gremlin"), "UTF-8");
+        clientService.getClient().submit(setup);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        String teardown = IOUtils.toString(getClass().getResourceAsStream("/teardown.gremlin"), "UTF-8");
+        clientService.getClient().submit(teardown);
+    }
+
+    @Test
+    public void testValueMap() {
+        String gremlin = "g.V().hasLabel('dog').valueMap()";
+        AtomicInteger integer = new AtomicInteger();
+        Map<String, String> result = clientService.executeQuery(gremlin, new HashMap<>(), (record, isMore) -> integer.incrementAndGet());
+
+        Assert.assertEquals(2, integer.get());
+    }
+
+    @Test
+    public void testCount() {
+        String gremlin = "g.V().hasLabel('dog').count()";
+        AtomicInteger integer = new AtomicInteger();
+        Map<String, String> result = clientService.executeQuery(gremlin, new HashMap<>(), (record, isMore) -> integer.incrementAndGet());
+        Assert.assertEquals(1, integer.get());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/MockProcessor.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/MockProcessor.java
new file mode 100644
index 0000000..0504887
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/MockProcessor.java
@@ -0,0 +1,52 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class MockProcessor extends AbstractProcessor {
+    public static final PropertyDescriptor CLIENT = new PropertyDescriptor.Builder()
+        .name("client")
+        .required(false)
+        .identifiesControllerService(GraphClientService.class)
+        .build();
+    public static final PropertyDescriptor GREMLIN_CLIENT = new PropertyDescriptor.Builder()
+            .name("gremlin-client")
+            .required(false)
+            .identifiesControllerService(GremlinClientService.class)
+            .build();
+
+    public static final List<PropertyDescriptor> PROPS = Arrays.asList(CLIENT, GREMLIN_CLIENT);
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPS;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/OpenCypherClientServiceIT.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/OpenCypherClientServiceIT.java
new file mode 100644
index 0000000..b4b3faa
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/OpenCypherClientServiceIT.java
@@ -0,0 +1,96 @@
+/*
+ * 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.graph;
+
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.neo4j.driver.v1.Driver;
+import org.neo4j.driver.v1.Session;
+import org.neo4j.driver.v1.StatementResult;
+import org.opencypher.gremlin.neo4j.driver.GremlinDatabase;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/*
+ * To run this, setup JanusGraph using just the BerkeleyJE configuration for the server.
+ *
+ * 1. Documentation for configuring JanusGraph: https://docs.janusgraph.org/latest/configuration.html
+ * 2. Documentation for setting up the Janus Server including adding both REST and WebSocket support:
+ *    https://docs.janusgraph.org/latest/server.html
+ *
+ * Then follow these steps with JanusGraph to install OpenCypher support:
+ *
+ * 1. Add support to the Janus console: https://github.com/opencypher/cypher-for-gremlin/tree/master/tinkerpop/cypher-gremlin-console-plugin
+ * 2. Add support to the Janus server: https://github.com/opencypher/cypher-for-gremlin/tree/master/tinkerpop/cypher-gremlin-server-plugin
+ */
+public class OpenCypherClientServiceIT {
+    TestRunner runner;
+    GraphClientService service;
+    private Driver driver;
+
+    @Before
+    public void before() throws Exception {
+        service = new OpenCypherClientService();
+        runner = TestRunners.newTestRunner(MockProcessor.class);
+        runner.addControllerService("clientService", service);
+        runner.setProperty(service, AbstractTinkerpopClientService.CONTACT_POINTS, "localhost");
+        runner.setProperty(MockProcessor.CLIENT, "clientService");
+        runner.enableControllerService(service);
+        runner.assertValid();
+
+        driver = GremlinDatabase.driver("//localhost:8182");
+        executeSession("MATCH (n) detach delete n");
+        executeSession("CREATE (rover:dog { name: \"Rover\"})");
+        executeSession("CREATE (fido:dog { name: \"Fido\"})");
+        executeSession("MATCH (fido:dog) WHERE fido.name = \"Rover\" " +
+                "MATCH (rover:dog) WHERE rover.name = \"Rover\" " +
+                "CREATE (rover)-[:chases]->(fido)");
+    }
+
+    @After
+    public void after() {
+        executeSession("MATCH (n) DETACH DELETE n");
+    }
+
+    protected StatementResult executeSession(String statement) {
+        try (Session session = driver.session()) {
+            return session.run(statement);
+        }
+    }
+
+    @Test
+    public void testBasicQuery() {
+        String query = "MATCH (n) RETURN n";
+
+        List<Map<String, Object>> results = new ArrayList<>();
+        Map<String, String> attributes = service.executeQuery(query, new HashMap<>(), (record, hasMore) -> results.add(record));
+        assertNotNull(attributes);
+        assertEquals(7, attributes.size());
+        assertEquals(2, results.size());
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/TestableGremlinClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/TestableGremlinClientService.java
new file mode 100644
index 0000000..5006e44
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/TestableGremlinClientService.java
@@ -0,0 +1,26 @@
+/*
+ * 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.graph;
+
+import org.apache.tinkerpop.gremlin.driver.Client;
+
+public class TestableGremlinClientService extends GremlinClientService {
+    public Client getClient() {
+        return client;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/resources/setup.gremlin b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/resources/setup.gremlin
new file mode 100644
index 0000000..0777329
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/resources/setup.gremlin
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+fido = graph.addVertex("dog")
+fido.property("name", "fido")
+fido.property("age", 3)
+fido.property("is_for", "testing_only")
+
+rover = graph.addVertex("dog")
+rover.property("name", "rover")
+rover.property("age", 2)
+rover.property("is_for", "testing_only")
+
+fido_owner = graph.addVertex("dog_owner")
+fido_owner.property("name", "John Smith")
+fido_owner.property("is_for", "testing_only")
+
+rover_owner = graph.addVertex("dog_owner")
+rover_owner.property("name", "Jane Doe")
+rover_owner.property("is_for", "testing_only")
+
+fido.addEdge('chases', rover)
+rover.addEdge("runs_from", fido)
+fido_owner.addEdge('owns', fido)
+rover_owner.addEdge('owns', rover)
+
+graph.tx().commit()
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/resources/teardown.gremlin b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/resources/teardown.gremlin
new file mode 100644
index 0000000..c67449d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/resources/teardown.gremlin
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+
+g.V().has('is_for', 'testing_only').drop().iterate()
+graph.tx().commit()
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/pom.xml b/nifi-nar-bundles/nifi-graph-bundle/pom.xml
similarity index 66%
rename from nifi-nar-bundles/nifi-neo4j-bundle/pom.xml
rename to nifi-nar-bundles/nifi-graph-bundle/pom.xml
index 1b41f88..8ffc4f0 100644
--- a/nifi-nar-bundles/nifi-neo4j-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-graph-bundle/pom.xml
@@ -10,7 +10,7 @@
   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
+  See the License for the specific lang`uage 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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
@@ -19,23 +19,29 @@
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-nar-bundles</artifactId>
-        <version>1.8.0-SNAPSHOT</version>
+        <version>1.10.0-SNAPSHOT</version>
     </parent>
 
-    <artifactId>nifi-neo4j-bundle</artifactId>
+    <artifactId>nifi-graph-bundle</artifactId>
     <packaging>pom</packaging>
 
     <modules>
-        <module>nifi-neo4j-processors</module>
-        <module>nifi-neo4j-nar</module>
+        <module>nifi-graph-client-service-api</module>
+        <module>nifi-graph-client-service-api-nar</module>
+        <module>nifi-graph-processors</module>
+        <module>nifi-graph-nar</module>
+        <module>nifi-neo4j-cypher-service</module>
+        <module>nifi-neo4j-cypher-service-nar</module>
+        <module>nifi-other-graph-services</module>
+        <module>nifi-other-graph-services-nar</module>
     </modules>
 
     <dependencyManagement>
         <dependencies>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
-                <artifactId>nifi-neo4j-processors</artifactId>
-                <version>1.8.0-SNAPSHOT</version>
+                <artifactId>nifi-graph-processors</artifactId>
+                <version>1.10.0-SNAPSHOT</version>
             </dependency>
         </dependencies>
     </dependencyManagement>
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/NOTICE
deleted file mode 100644
index bfe604e..0000000
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/NOTICE
+++ /dev/null
@@ -1,31 +0,0 @@
-nifi-neo4j-nar
-Copyright 2016 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-===========================================
-Apache Software License v2
-===========================================
-
-The following binary components are provided under the Apache Software License v2
-
-  (ASL2 License) 
-    (ASLv2) Neo4j Java Driver
-      Neo4j
-      Copyright © 2002-2018 Neo4j Sweden AB (referred to in this notice as "Neo4j")
-      [http://neo4j.com]
-
-  (ASLv2) Apache Commons IO
-    The following NOTICE information applies:
-      Apache Commons IO
-      Copyright 2002-2016 The Apache Software Foundation
-
-  (ASLv2) Apache Commons Lang
-    The following NOTICE information applies:
-      Apache Commons Lang
-      Copyright 2001-2015 The Apache Software Foundation
-
-  (ASLv2) Google GSON
-    The following NOTICE information applies:
-    Copyright 2008 Google Inc.
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/java/org/apache/nifi/processors/neo4j/Neo4JCypherExecutor.java b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/java/org/apache/nifi/processors/neo4j/Neo4JCypherExecutor.java
deleted file mode 100644
index 22d661a..0000000
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/main/java/org/apache/nifi/processors/neo4j/Neo4JCypherExecutor.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * 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.neo4j;
-
-import java.io.ByteArrayInputStream;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.nifi.annotation.behavior.EventDriven;
-import org.apache.nifi.annotation.behavior.InputRequirement;
-import org.apache.nifi.annotation.behavior.SupportsBatching;
-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.annotation.lifecycle.OnStopped;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.neo4j.driver.v1.Session;
-import org.neo4j.driver.v1.StatementResult;
-import org.neo4j.driver.v1.summary.ResultSummary;
-import org.neo4j.driver.v1.summary.SummaryCounters;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
-@EventDriven
-@SupportsBatching
-@Tags({"neo4j", "graph", "network", "insert", "update", "delete", "put", "get", "node", "relationship", "connection", "executor"})
-@CapabilityDescription("This processor executes a Neo4J Query (https://www.neo4j.com/) defined in the 'Neo4j Query' property of the "
-    + "FlowFile and writes the result to the FlowFile body in JSON format. The processor has been tested with Neo4j version 3.4.5")
-@WritesAttributes({
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.ERROR_MESSAGE, description = "Neo4J error message"),
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.LABELS_ADDED, description = "Number of labels added"),
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.NODES_CREATED, description = "Number of nodes created"),
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.NODES_DELETED, description = "Number of nodes deleted"),
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.PROPERTIES_SET, description = "Number of properties set"),
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.RELATIONS_CREATED, description = "Number of relationships created"),
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.RELATIONS_DELETED, description = "Number of relationships deleted"),
-    @WritesAttribute(attribute = AbstractNeo4JCypherExecutor.ROWS_RETURNED, description = "Number of rows returned"),
-    })
-public class Neo4JCypherExecutor extends AbstractNeo4JCypherExecutor {
-
-    private static final Set<Relationship> relationships;
-    private static final List<PropertyDescriptor> propertyDescriptors;
-
-    static {
-        final Set<Relationship> tempRelationships = new HashSet<>();
-        tempRelationships.add(REL_SUCCESS);
-        tempRelationships.add(REL_FAILURE);
-        relationships = Collections.unmodifiableSet(tempRelationships);
-
-        final List<PropertyDescriptor> tempDescriptors = new ArrayList<>();
-        tempDescriptors.add(CONNECTION_URL);
-        tempDescriptors.add(USERNAME);
-        tempDescriptors.add(PASSWORD);
-        tempDescriptors.add(QUERY);
-        tempDescriptors.add(LOAD_BALANCING_STRATEGY);
-        tempDescriptors.add(CONNECTION_TIMEOUT);
-        tempDescriptors.add(MAX_CONNECTION_POOL_SIZE);
-        tempDescriptors.add(MAX_CONNECTION_ACQUISITION_TIMEOUT);
-        tempDescriptors.add(IDLE_TIME_BEFORE_CONNECTION_TEST);
-        tempDescriptors.add(MAX_CONNECTION_LIFETIME);
-        tempDescriptors.add(ENCRYPTION);
-        tempDescriptors.add(SSL_CONTEXT_SERVICE);
-
-        propertyDescriptors = Collections.unmodifiableList(tempDescriptors);
-    }
-
-    protected ObjectMapper mapper = new ObjectMapper();
-
-    @Override
-    public Set<Relationship> getRelationships() {
-        return relationships;
-    }
-
-    @Override
-    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return propertyDescriptors;
-    }
-
-    @OnScheduled
-    public void onScheduled(final ProcessContext context) {
-        super.onScheduled(context);
-    }
-
-//    @Override
-//    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
-//        List<ValidationResult> results = new ArrayList<>();
-//
-//        PropertyValue trustStrategy = validationContext.getProperty(AbstractNeo4JCypherExecutor.TRUST_STRATEGY);
-//        if (trustStrategy.isSet() && trustStrategy.getValue().equals(AbstractNeo4JCypherExecutor.TRUST_CUSTOM_CA_SIGNED_CERTIFICATES.getValue())) {
-//            if ( ! validationContext.getProperty(AbstractNeo4JCypherExecutor.TRUST_CUSTOM_CA_SIGNED_CERTIFICATES_FILE).evaluateAttributeExpressions().isSet() ) {
-//                results.add(new ValidationResult.Builder()
-//                    .subject(TRUST_STRATEGY.getDisplayName() + " with " + TRUST_CUSTOM_CA_SIGNED_CERTIFICATES.getDisplayName())
-//                    .explanation(TRUST_STRATEGY.getDisplayName() + " with " + TRUST_CUSTOM_CA_SIGNED_CERTIFICATES.getDisplayName() + " requires "
-//                         + AbstractNeo4JCypherExecutor.TRUST_CUSTOM_CA_SIGNED_CERTIFICATES_FILE.getDisplayName() + " to be set").valid(false).build());
-//            }
-//        }
-//        return results;
-//
-//    }
-
-    @Override
-    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
-        FlowFile flowFile = session.get();
-        if (flowFile == null) {
-            return;
-        }
-
-        String query = context.getProperty(QUERY).evaluateAttributeExpressions(flowFile).getValue();
-
-        try {
-            long startTimeMillis = System.currentTimeMillis();
-
-            StatementResult statementResult = executeQuery(query);
-
-            List<Map<String, Object>> returnValue = statementResult.list().stream().map(i -> i.asMap()).collect(Collectors.toList());
-
-            if ( getLogger().isDebugEnabled() ) {
-                getLogger().debug("Result of query {} is {}", new Object [] { query, returnValue });
-            }
-
-            String json = mapper.writeValueAsString(returnValue);
-
-            ByteArrayInputStream bios = new ByteArrayInputStream(json.getBytes(Charset.defaultCharset()));
-            session.importFrom(bios, flowFile);
-
-            final long endTimeMillis = System.currentTimeMillis();
-
-            if ( getLogger().isDebugEnabled() ) {
-                getLogger().debug("Executed statement with result {}", new Object[] {statementResult});
-            }
-
-            flowFile = populateAttributes(session, flowFile, statementResult, returnValue.size());
-
-            session.transfer(flowFile, REL_SUCCESS);
-            session.getProvenanceReporter().send(flowFile, connectionUrl, (endTimeMillis - startTimeMillis));
-        } catch (Exception exception) {
-            getLogger().error("Failed to execute Neo4J statement due to {}",
-                    new Object[]{exception.getLocalizedMessage()}, exception);
-            flowFile = session.putAttribute(flowFile, ERROR_MESSAGE, String.valueOf(exception.getMessage()));
-            session.transfer(flowFile, REL_FAILURE);
-            context.yield();
-        }
-    }
-
-    protected StatementResult executeQuery(String query) {
-        try ( Session session = getNeo4JDriver().session()) {
-            return session.run(query);
-        }
-    }
-
-    private FlowFile populateAttributes(final ProcessSession session, FlowFile flowFile,
-        StatementResult statementResult, int size) {
-        ResultSummary summary = statementResult.summary();
-
-        SummaryCounters counters = summary.counters();
-
-        Map<String,String> resultAttributes = new HashMap<>();
-        resultAttributes.put(NODES_CREATED,String.valueOf(counters.nodesCreated()));
-        resultAttributes.put(RELATIONS_CREATED,String.valueOf(counters.relationshipsCreated()));
-        resultAttributes.put(LABELS_ADDED,String.valueOf(counters.labelsAdded()));
-        resultAttributes.put(NODES_DELETED,String.valueOf(counters.nodesDeleted()));
-        resultAttributes.put(RELATIONS_DELETED,String.valueOf(counters.relationshipsDeleted()));
-        resultAttributes.put(PROPERTIES_SET, String.valueOf(counters.propertiesSet()));
-        resultAttributes.put(ROWS_RETURNED, String.valueOf(size));
-
-        flowFile = session.putAllAttributes(flowFile, resultAttributes);
-        return flowFile;
-    }
-
-    @OnStopped
-    public void close() {
-        super.close();
-    }
-}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/test/java/org/apache/nifi/processors/neo4j/ITNeo4JCypherExecutor.java b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/test/java/org/apache/nifi/processors/neo4j/ITNeo4JCypherExecutor.java
deleted file mode 100644
index 502aac7..0000000
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/test/java/org/apache/nifi/processors/neo4j/ITNeo4JCypherExecutor.java
+++ /dev/null
@@ -1,219 +0,0 @@
-/*
- * 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.neo4j;
-
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.neo4j.driver.v1.AuthTokens;
-import org.neo4j.driver.v1.Driver;
-import org.neo4j.driver.v1.GraphDatabase;
-import org.neo4j.driver.v1.Session;
-import org.neo4j.driver.v1.StatementResult;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.nio.charset.Charset;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Neo4J Cypher integration tests.  Please set the neo4j url, user and password according to your setup.
- * The steps to setup neo4j are
- * <ol>
- *   <li> Install Neo4J</li>
- *      <code>brew install neo4j</code>
- *   <li>Setup neo4j</li>
- * <code>neo4j start</code>
- *    <li>Log into cypher shell using default username/password - neo4j/neo4j</li>
- * <code>cypher-shell</code>
- *    <li>Changel password to admin</li>
- * <code>CALL dbms.changePassword('admin')</code>
- *    <li> Restart neo4j</li>
- * <code>neo4j restart</code>
- *    <li>Log into cypher shell using new password (admin)</li>
- * <code>cypher-shell</code>
- * </ol>
- */
-public class ITNeo4JCypherExecutor {
-    protected TestRunner runner;
-    protected Driver driver;
-    protected String neo4jUrl = "bolt://localhost:7687";
-    protected String user = "neo4j";
-    protected String password = "admin";
-
-    @Before
-    public void setUp() throws Exception {
-        runner = TestRunners.newTestRunner(Neo4JCypherExecutor.class);
-        runner.setProperty(AbstractNeo4JCypherExecutor.CONNECTION_URL, neo4jUrl);
-        runner.setProperty(AbstractNeo4JCypherExecutor.USERNAME, user);
-        runner.setProperty(AbstractNeo4JCypherExecutor.PASSWORD, password);
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "match (n) return n");
-        driver = GraphDatabase.driver(neo4jUrl, AuthTokens.basic(user, password));
-        executeSession("match (n) detach delete n");
-
-        StatementResult result = executeSession("match (n) return n");
-
-        assertEquals("nodes should be equal", 0, result.list().size());
-    }
-
-    protected StatementResult executeSession(String statement) {
-        try (Session session = driver.session()) {
-            return session.run(statement);
-        }
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        runner = null;
-        if (driver != null) {
-            driver.close();
-        }
-        driver = null;
-    }
-
-    @Test
-    public void testCreateNodeNoReturn() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "create (n)");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_SUCCESS);
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-        flowFiles.get(0).assertContentEquals("[]".getBytes(Charset.defaultCharset()));
-    }
-
-    @Test
-    public void testCreateNodeOnePropertyWithReturn() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "create (n { name:'abc' }) return n.name");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_SUCCESS);
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-        flowFiles.get(0).assertContentEquals("[{\"n.name\":\"abc\"}]".getBytes(Charset.defaultCharset()));
-    }
-
-    @Test
-    public void testCreateNodeTwoPropertyOneLabelWithReturn() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "create (n:Person { name:'abc', age : 1 }) return n.name, n.age");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_SUCCESS);
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("2",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-        flowFiles.get(0).assertContentEquals("[{\"n.name\":\"abc\",\"n.age\":1}]".getBytes(Charset.defaultCharset()));
-    }
-
-    @Test
-    public void testCreateTwoNodeTwoPropertyOneRelationshipWithReturn() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY,
-            "create (m:Person { name:'abc', age : 1 }) create (n:Person { name : 'pqr'}) create (m)-[r:hello]->(n) return m.name, n.name, type(r)");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_SUCCESS);
-        assertEquals("2",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("2",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("3",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-        flowFiles.get(0).assertContentEquals("[{\"m.name\":\"abc\",\"n.name\":\"pqr\",\"type(r)\":\"hello\"}]".getBytes(Charset.defaultCharset()));
-    }
-
-    @Test
-    public void testCreateNodeOnePropertyWithReturnWithEL() throws Exception {
-        String query = "create (n { name:'abc' }) return n.name";
-
-        byte [] bytes = new byte [] {};
-        Map<String,String> properties = new HashMap<>();
-        properties.put("query",query);
-
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "${query}");
-        runner.enqueue(bytes, properties);
-        runner.run(1,true,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_SUCCESS);
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-        flowFiles.get(0).assertContentEquals("[{\"n.name\":\"abc\"}]".getBytes(Charset.defaultCharset()));
-    }
-
-    @Test
-    public void testCreateTwoNodesWithReturn() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "create (m:Person { name:'abc'}) create (n:Person { name : 'pqr'}) return n.name, m.name");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,false,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_SUCCESS, 1);
-
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "match (n) return n.name");
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_SUCCESS);
-        assertEquals("2",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("2",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("2",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("1",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-
-        assertEquals("0",flowFiles.get(1).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("0",flowFiles.get(1).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(1).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("0",flowFiles.get(1).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(1).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("0",flowFiles.get(1).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("2",flowFiles.get(1).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-        flowFiles.get(1).assertContentEquals("[{\"n.name\":\"abc\"},{\"n.name\":\"pqr\"}]".getBytes(Charset.defaultCharset()));
-    }
-}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/test/java/org/apache/nifi/processors/neo4j/TestNeo4JCypherExecutor.java b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/test/java/org/apache/nifi/processors/neo4j/TestNeo4JCypherExecutor.java
deleted file mode 100644
index c6b1ac2..0000000
--- a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-processors/src/test/java/org/apache/nifi/processors/neo4j/TestNeo4JCypherExecutor.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * 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.neo4j;
-
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.TestRunner;
-import org.apache.nifi.util.TestRunners;
-import org.neo4j.driver.v1.Driver;
-import org.neo4j.driver.v1.StatementResult;
-import org.neo4j.driver.v1.Record;
-import org.neo4j.driver.v1.summary.ResultSummary;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.mockito.Answers;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.mockito.junit.MockitoJUnit;
-import org.mockito.junit.MockitoRule;
-
-import java.io.File;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Neo4J Cypher unit tests.
- */
-public class TestNeo4JCypherExecutor {
-    protected TestRunner runner;
-    protected Driver driver;
-    protected String neo4jUrl = "bolt://localhost:7687";
-    protected String user = "neo4j";
-    protected String password = "admin";
-
-    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
-    protected Driver mockDriver;
-
-    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
-    protected StatementResult mockStatementResult;
-
-    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
-    protected ResultSummary mockResultSummary;
-
-    @Rule public MockitoRule mockitoRule = MockitoJUnit.rule();
-
-    @Before
-    public void setUp() throws Exception {
-        Neo4JCypherExecutor mockExecutor = new Neo4JCypherExecutor() {
-
-            @Override
-            protected StatementResult executeQuery(String query) {
-                return mockStatementResult;
-            }
-
-            @Override
-            protected Driver getNeo4JDriver() {
-                return mockDriver;
-            }
-
-            @Override
-            protected Driver getDriver(ProcessContext context) {
-                connectionUrl = "testurl";
-                return mockDriver;
-            }
-        };
-
-        runner = TestRunners.newTestRunner(mockExecutor);
-        runner.setProperty(AbstractNeo4JCypherExecutor.CONNECTION_URL, neo4jUrl);
-        runner.setProperty(AbstractNeo4JCypherExecutor.USERNAME, user);
-        runner.setProperty(AbstractNeo4JCypherExecutor.PASSWORD, password);
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "match (n) return n");
-
-        Mockito.when(mockDriver.session().run(Mockito.anyString())).thenReturn(mockStatementResult);
-        Mockito.when(mockStatementResult.list()).thenReturn(new ArrayList<Record>());
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        runner = null;
-        mockDriver = null;
-        mockResultSummary = null;
-        mockStatementResult = null;
-    }
-
-    @Test
-    public void testTestLoadBalancingStrategy() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.LOAD_BALANCING_STRATEGY, AbstractNeo4JCypherExecutor.LOAD_BALANCING_STRATEGY_ROUND_ROBIN.getValue());
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.LOAD_BALANCING_STRATEGY, AbstractNeo4JCypherExecutor.LOAD_BALANCING_STRATEGY_LEAST_CONNECTED.getValue());
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.LOAD_BALANCING_STRATEGY, "BadValue");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void testTestConnectionTimeout() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.CONNECTION_TIMEOUT, "0 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.CONNECTION_TIMEOUT, "1 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.CONNECTION_TIMEOUT, "-1 seconds");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void testTestIdleTimeBeforeConnectionTest() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.IDLE_TIME_BEFORE_CONNECTION_TEST, "0 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.IDLE_TIME_BEFORE_CONNECTION_TEST, "1 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.IDLE_TIME_BEFORE_CONNECTION_TEST, "-1 seconds");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void testEncryption() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.ENCRYPTION, "true");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.ENCRYPTION, "false");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.ENCRYPTION, "bad");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void testTestMaxConnectionAcquisitionTime() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_ACQUISITION_TIMEOUT, "0 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_ACQUISITION_TIMEOUT, "1 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_ACQUISITION_TIMEOUT, "-1 seconds");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void testTestMaxConnectionLifeTime() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_LIFETIME, "0 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_LIFETIME, "1 seconds");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_LIFETIME, "-1 seconds");
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void testTestMaxConnectionPoolSize() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_POOL_SIZE, "1");
-        runner.assertValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_POOL_SIZE, "-1");
-        runner.assertNotValid();
-        runner.setProperty(AbstractNeo4JCypherExecutor.MAX_CONNECTION_POOL_SIZE, "0");
-        runner.assertNotValid();
-    }
-
-//    @Test
-//    public void testTestTrustStrategy() throws Exception {
-//        runner.setProperty(AbstractNeo4JCypherExecutor.TRUST_STRATEGY, AbstractNeo4JCypherExecutor.TRUST_ALL_CERTIFICATES.getValue());
-//        runner.assertValid();
-//        runner.setProperty(AbstractNeo4JCypherExecutor.TRUST_STRATEGY, AbstractNeo4JCypherExecutor.TRUST_CUSTOM_CA_SIGNED_CERTIFICATES.getValue());
-//        runner.assertNotValid();
-//        runner.setProperty(AbstractNeo4JCypherExecutor.TRUST_CUSTOM_CA_SIGNED_CERTIFICATES_FILE, File.createTempFile("temp", "tmp").getAbsolutePath());
-//        runner.assertValid();
-//        runner.setProperty(AbstractNeo4JCypherExecutor.TRUST_STRATEGY, "BadValue");
-//        runner.assertNotValid();
-//    }
-//
-    @Test
-    public void testCreateNodeNoReturn() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "create (n)");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_SUCCESS, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_SUCCESS);
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.LABELS_ADDED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.NODES_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_CREATED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.RELATIONS_DELETED));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.PROPERTIES_SET));
-        assertEquals("0",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ROWS_RETURNED));
-        flowFiles.get(0).assertContentEquals("[]".getBytes(Charset.defaultCharset()));
-    }
-
-    @Test(expected=AssertionError.class)
-    public void testGetDriverThrowsException() throws Exception {
-        Neo4JCypherExecutor mockExecutor = new Neo4JCypherExecutor() {
-
-            @Override
-            protected StatementResult executeQuery(String query) {
-                return mockStatementResult;
-            }
-
-            @Override
-            protected Driver getNeo4JDriver() {
-                return mockDriver;
-            }
-
-            @Override
-            protected Driver getDriver(ProcessContext context) {
-                throw new RuntimeException("RuntimeException");
-            }
-        };
-
-        runner = TestRunners.newTestRunner(mockExecutor);
-        runner.setProperty(AbstractNeo4JCypherExecutor.CONNECTION_URL, neo4jUrl);
-        runner.setProperty(AbstractNeo4JCypherExecutor.USERNAME, user);
-        runner.setProperty(AbstractNeo4JCypherExecutor.PASSWORD, password);
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "match (n) return n");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-    }
-
-    @Test
-    public void testExecuteQueryThrowsException() throws Exception {
-        Neo4JCypherExecutor mockExecutor = new Neo4JCypherExecutor() {
-
-            @Override
-            protected StatementResult executeQuery(String query) {
-                throw new RuntimeException("QueryException");
-            }
-
-            @Override
-            protected Driver getNeo4JDriver() {
-                return mockDriver;
-            }
-
-            @Override
-            protected Driver getDriver(ProcessContext context) {
-                return mockDriver;
-            }
-        };
-
-        runner = TestRunners.newTestRunner(mockExecutor);
-        runner.setProperty(AbstractNeo4JCypherExecutor.CONNECTION_URL, neo4jUrl);
-        runner.setProperty(AbstractNeo4JCypherExecutor.USERNAME, user);
-        runner.setProperty(AbstractNeo4JCypherExecutor.PASSWORD, password);
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "match (n) return n");
-
-        runner.enqueue(new byte[] {});
-        runner.run(1,true,true);
-        runner.assertAllFlowFilesTransferred(Neo4JCypherExecutor.REL_FAILURE, 1);
-        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(Neo4JCypherExecutor.REL_FAILURE);
-        assertEquals("QueryException",flowFiles.get(0).getAttribute(Neo4JCypherExecutor.ERROR_MESSAGE));
-    }
-
-    @Test
-    public void testEmptyQuery() throws Exception {
-        runner.setProperty(AbstractNeo4JCypherExecutor.QUERY, "");
-        runner.assertNotValid();
-    }
-}
\ No newline at end of file
diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml
index 1dc42d6..ba27b6c 100755
--- a/nifi-nar-bundles/pom.xml
+++ b/nifi-nar-bundles/pom.xml
@@ -75,7 +75,7 @@
         <module>nifi-ignite-bundle</module>
         <module>nifi-rethinkdb-bundle</module>
         <module>nifi-influxdb-bundle</module>
-        <module>nifi-neo4j-bundle</module>
+        <module>nifi-graph-bundle</module>
         <module>nifi-email-bundle</module>
         <module>nifi-groovyx-bundle</module>
     	<module>nifi-ranger-bundle</module>