You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ij...@apache.org on 2018/06/15 06:38:44 UTC

[1/2] nifi git commit: NIFI-5214 Added REST LookupService

Repository: nifi
Updated Branches:
  refs/heads/master be31c1efd -> be63378a1


http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupService.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupService.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupService.groovy
new file mode 100644
index 0000000..0de379b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupService.groovy
@@ -0,0 +1,144 @@
+/*
+ * 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.lookup
+
+import okhttp3.*
+import org.apache.nifi.lookup.rest.MockRestLookupService
+import org.apache.nifi.serialization.SimpleRecordSchema
+import org.apache.nifi.serialization.record.MapRecord
+import org.apache.nifi.serialization.record.MockRecordParser
+import org.apache.nifi.serialization.record.RecordField
+import org.apache.nifi.serialization.record.RecordFieldType
+import org.apache.nifi.serialization.record.RecordSchema
+import org.apache.nifi.util.TestRunner
+import org.apache.nifi.util.TestRunners
+import org.junit.Assert
+import org.junit.Before
+import org.junit.Test
+
+import static groovy.json.JsonOutput.toJson
+
+class TestRestLookupService {
+    TestRunner runner
+    MockRecordParser recordReader
+    MockRestLookupService lookupService
+
+    static final String JSON_TYPE = "application/json"
+
+    @Before
+    void setup() {
+        recordReader = new MockRecordParser()
+        lookupService = new MockRestLookupService()
+        runner = TestRunners.newTestRunner(TestRestLookupServiceProcessor.class)
+        runner.addControllerService("lookupService", lookupService)
+        runner.addControllerService("recordReader", recordReader)
+        runner.setProperty(lookupService, RestLookupService.RECORD_READER, "recordReader")
+        runner.setProperty("Lookup Service", "lookupService")
+        runner.setProperty(lookupService, RestLookupService.URL, "http://localhost:8080")
+        runner.enableControllerService(lookupService)
+        runner.enableControllerService(recordReader)
+        runner.assertValid()
+    }
+
+    @Test
+    void testSimpleLookup() {
+        recordReader.addSchemaField("name", RecordFieldType.STRING)
+        recordReader.addSchemaField("age", RecordFieldType.INT)
+        recordReader.addSchemaField("sport", RecordFieldType.STRING)
+
+        recordReader.addRecord("John Doe", 48, "Soccer")
+        recordReader.addRecord("Jane Doe", 47, "Tennis")
+        recordReader.addRecord("Sally Doe", 47, "Curling")
+
+        lookupService.response = buildResponse(toJson([ simpleTest: true]), JSON_TYPE)
+        def result = lookupService.lookup(getCoordinates(JSON_TYPE, "get"))
+        Assert.assertTrue(result.isPresent())
+        def record = result.get()
+        Assert.assertEquals("John Doe", record.getAsString("name"))
+        Assert.assertEquals(48, record.getAsInt("age"))
+        Assert.assertEquals("Soccer", record.getAsString("sport"))
+    }
+    
+    @Test
+    void testNestedLookup() {
+        runner.disableControllerService(lookupService)
+        runner.setProperty(lookupService, RestLookupService.RECORD_PATH, "/person")
+        runner.enableControllerService(lookupService)
+        runner.assertValid()
+
+        recordReader.addSchemaField("id", RecordFieldType.INT)
+        final List<RecordField> personFields = new ArrayList<>()
+        final RecordField nameField = new RecordField("name", RecordFieldType.STRING.getDataType())
+        final RecordField ageField = new RecordField("age", RecordFieldType.INT.getDataType())
+        final RecordField sportField = new RecordField("sport", RecordFieldType.STRING.getDataType())
+        personFields.add(nameField)
+        personFields.add(ageField)
+        personFields.add(sportField)
+        final RecordSchema personSchema = new SimpleRecordSchema(personFields)
+        recordReader.addSchemaField("person", RecordFieldType.RECORD)
+        recordReader.addRecord(1, new MapRecord(personSchema, new HashMap<String,Object>() {{
+            put("name", "John Doe")
+            put("age", 48)
+            put("sport", "Soccer")
+        }}))
+
+        lookupService.response = buildResponse(toJson([ simpleTest: true]), JSON_TYPE)
+        def result = lookupService.lookup(getCoordinates(JSON_TYPE, "get"))
+        Assert.assertTrue(result.isPresent())
+        def record = result.get()
+
+        Assert.assertEquals("John Doe", record.getAsString("name"))
+        Assert.assertEquals(48, record.getAsInt("age"))
+        Assert.assertEquals("Soccer", record.getAsString("sport"))
+
+        /*
+         * Test deep lookup
+         */
+
+        runner.disableControllerService(lookupService)
+        runner.setProperty(lookupService, RestLookupService.RECORD_PATH, "/person/sport")
+        runner.enableControllerService(lookupService)
+        runner.assertValid()
+
+        result = lookupService.lookup(getCoordinates(JSON_TYPE, "get"))
+        Assert.assertTrue(result.isPresent())
+        record = result.get()
+        Assert.assertNotNull(record.getAsString("sport"))
+        Assert.assertEquals("Soccer", record.getAsString("sport"))
+    }
+
+    private Map<String, Object> getCoordinates(String mimeType, String method) {
+        def retVal = [:]
+        retVal[RestLookupService.MIME_TYPE_KEY] = mimeType
+        retVal[RestLookupService.METHOD_KEY] = method
+
+        retVal
+    }
+
+    private Response buildResponse(String resp, String mimeType) {
+        return new Response.Builder()
+            .code(200)
+            .body(
+                ResponseBody.create(MediaType.parse(mimeType), resp)
+            )
+            .message("Test")
+            .protocol(Protocol.HTTP_1_1)
+            .request(new Request.Builder().url("http://localhost:8080").get().build())
+            .build()
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupServiceProcessor.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupServiceProcessor.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupServiceProcessor.groovy
new file mode 100644
index 0000000..9ca2442
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/TestRestLookupServiceProcessor.groovy
@@ -0,0 +1,45 @@
+/*
+ * 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.lookup
+
+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
+
+class TestRestLookupServiceProcessor extends AbstractProcessor {
+    static final PropertyDescriptor CLIENT_SERVICE = new PropertyDescriptor.Builder()
+            .name("Lookup Service")
+            .description("RestLookupService")
+            .identifiesControllerService(RestLookupService.class)
+            .required(true)
+            .build()
+
+    @Override
+    void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+    
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        List<PropertyDescriptor> propDescs = new ArrayList<>()
+        propDescs.add(CLIENT_SERVICE)
+        return propDescs
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/MockRestLookupService.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/MockRestLookupService.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/MockRestLookupService.groovy
new file mode 100644
index 0000000..210f329
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/MockRestLookupService.groovy
@@ -0,0 +1,31 @@
+/*
+ * 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.lookup.rest
+
+import okhttp3.Request
+import okhttp3.Response
+import org.apache.nifi.lookup.RestLookupService
+
+class MockRestLookupService extends RestLookupService {
+    Response response
+
+    @Override
+    protected Response executeRequest(Request request) {
+        return response
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/SchemaUtil.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/SchemaUtil.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/SchemaUtil.groovy
new file mode 100644
index 0000000..8ac4e4b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/SchemaUtil.groovy
@@ -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.lookup.rest
+
+class SchemaUtil {
+    static final String SIMPLE = SchemaUtil.class.getResourceAsStream("/simple.avsc").text
+
+    static final String COMPLEX = SchemaUtil.class.getResourceAsStream("/complex.avsc").text
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/BasicAuth.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/BasicAuth.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/BasicAuth.groovy
new file mode 100644
index 0000000..eb0c6f0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/BasicAuth.groovy
@@ -0,0 +1,55 @@
+/*
+ * 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.lookup.rest.handlers
+
+import org.eclipse.jetty.server.Request
+import org.eclipse.jetty.server.handler.AbstractHandler
+
+import javax.servlet.ServletException
+import javax.servlet.http.HttpServletRequest
+import javax.servlet.http.HttpServletResponse
+
+import static groovy.json.JsonOutput.prettyPrint
+import static groovy.json.JsonOutput.toJson
+
+class BasicAuth extends AbstractHandler {
+
+    @Override
+    void handle(String target, Request baseRequest, HttpServletRequest request, HttpServletResponse response) throws IOException, ServletException {
+        baseRequest.handled = true
+        def authString = request.getHeader("Authorization")
+        def headers = []
+        request.headerNames.each { headers << it }
+
+        if (!authString || authString != "Basic am9obi5zbWl0aDp0ZXN0aW5nMTIzNA==") {
+            response.status = 401
+            response.setHeader("WWW-Authenticate", "Basic realm=\"Jetty\"")
+            response.setHeader("response.phrase", "Unauthorized")
+            response.contentType = "text/plain"
+            response.writer.println("Get off my lawn!")
+            return
+        }
+
+        response.writer.println(prettyPrint(
+            toJson([
+                username: "john.smith",
+                password: "testing1234"
+            ])
+        ))
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/ComplexJson.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/ComplexJson.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/ComplexJson.groovy
new file mode 100644
index 0000000..c8790f8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/ComplexJson.groovy
@@ -0,0 +1,45 @@
+/*
+ * 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.lookup.rest.handlers
+
+import javax.servlet.http.HttpServlet
+import javax.servlet.http.HttpServletRequest
+import javax.servlet.http.HttpServletResponse
+
+import static groovy.json.JsonOutput.prettyPrint
+import static groovy.json.JsonOutput.toJson
+
+class ComplexJson extends HttpServlet {
+    @Override
+    void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+        response.contentType = "application/json"
+        response.outputStream.write(prettyPrint(
+            toJson([
+                top: [
+                    middle: [
+                        inner: [
+                            "username": "jane.doe",
+                            "password": "testing7890",
+                            "email": "jane.doe@test-example.com"
+                        ]
+                    ]
+                ]
+            ])
+        ).bytes)
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/NoRecord.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/NoRecord.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/NoRecord.groovy
new file mode 100644
index 0000000..cf80b64
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/NoRecord.groovy
@@ -0,0 +1,39 @@
+/*
+ * 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.lookup.rest.handlers
+
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import static groovy.json.JsonOutput.*
+
+import javax.servlet.http.HttpServlet
+import javax.servlet.http.HttpServletRequest
+import javax.servlet.http.HttpServletResponse
+
+class NoRecord extends HttpServlet {
+    Logger logger = LoggerFactory.getLogger(NoRecord.class)
+    @Override
+    void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+
+        response.contentType = "application/json"
+        response.outputStream.write(prettyPrint(
+            toJson([:])
+        ).bytes)
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJson.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJson.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJson.groovy
new file mode 100644
index 0000000..9fcc9d2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJson.groovy
@@ -0,0 +1,44 @@
+/*
+ * 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.lookup.rest.handlers
+
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import static groovy.json.JsonOutput.*
+
+import javax.servlet.http.HttpServlet
+import javax.servlet.http.HttpServletRequest
+import javax.servlet.http.HttpServletResponse
+
+class SimpleJson extends HttpServlet {
+    Logger logger = LoggerFactory.getLogger(SimpleJson.class)
+    @Override
+    void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+        String u = request.getHeader("X-USER")
+        String p = request.getHeader("X-PASS")
+
+        response.contentType = "application/json"
+        response.outputStream.write(prettyPrint(
+            toJson([
+                username: u ?: "john.smith",
+                password: p ?: "testing1234"
+            ])
+        ).bytes)
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJsonArray.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJsonArray.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJsonArray.groovy
new file mode 100644
index 0000000..6b63dd6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/SimpleJsonArray.groovy
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.lookup.rest.handlers
+
+import javax.servlet.http.HttpServlet
+import javax.servlet.http.HttpServletRequest
+import javax.servlet.http.HttpServletResponse
+
+import static groovy.json.JsonOutput.prettyPrint
+import static groovy.json.JsonOutput.toJson
+
+class SimpleJsonArray extends HttpServlet {
+    @Override
+    void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+        response.contentType = "application/json"
+        response.outputStream.write(prettyPrint(
+            toJson([[
+                username: "john.smith",
+                password: "testing1234"
+            ],
+                [
+                    username: "jane.doe",
+                    password: "testing7890"
+            ]])
+        ).bytes)
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/VerbTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/VerbTest.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/VerbTest.groovy
new file mode 100644
index 0000000..ad6f232
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/VerbTest.groovy
@@ -0,0 +1,69 @@
+/*
+ * 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.lookup.rest.handlers
+
+import org.apache.nifi.util.StringUtils
+import org.junit.Assert
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.servlet.http.HttpServlet
+import javax.servlet.http.HttpServletRequest
+import javax.servlet.http.HttpServletResponse
+
+import static groovy.json.JsonOutput.prettyPrint
+import static groovy.json.JsonOutput.toJson
+
+class VerbTest extends HttpServlet {
+    Logger logger = LoggerFactory.getLogger(VerbTest.class)
+
+    void doDelete(HttpServletRequest request, HttpServletResponse response) {
+        validateBody(request)
+        sendResponse(response)
+    }
+
+    void doPost(HttpServletRequest request, HttpServletResponse response) {
+        validateBody(request)
+        sendResponse(response)
+    }
+
+    void doPut(HttpServletRequest request, HttpServletResponse response) {
+        validateBody(request)
+        sendResponse(response)
+    }
+
+    void sendResponse(HttpServletResponse response) {
+        response.contentType = "application/json"
+        response.outputStream.write(prettyPrint(
+            toJson([
+                username: "john.smith",
+                password: "testing1234"
+            ])
+        ).bytes)
+    }
+
+    void validateBody(HttpServletRequest request) {
+        String needsBody = request.getHeader("needs-body")
+        boolean bodyRequired = !StringUtils.isBlank(needsBody)
+        String body = request.inputStream.text
+        if (bodyRequired) {
+            Assert.assertNotNull(body)
+            Assert.assertFalse(StringUtils.isBlank(body))
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/complex.avsc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/complex.avsc b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/complex.avsc
new file mode 100644
index 0000000..96baf3a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/complex.avsc
@@ -0,0 +1,36 @@
+{
+    "type": "record",
+    "name": "ComplexRecord",
+    "fields": [
+            {
+                "name": "top",
+                "type": {
+                "type": "record",
+                "name": "TopRecord",
+                "fields": [
+                        {
+                            "name": "middle",
+                            "type": {
+                            "name": "MiddleRecord",
+                            "type": "record",
+                            "fields": [
+                                    {
+                                        "name": "inner",
+                                        "type": {
+                                        "type": "record",
+                                        "name": "InnerRecord",
+                                        "fields": [
+                                                { "name": "username", "type": "string" },
+                                                { "name": "password", "type": "string" },
+                                                { "name": "email", "type": "string" }
+                                        ]
+                                    }
+                                    }
+                            ]
+                        }
+                        }
+                ]
+            }
+            }
+    ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/simple.avsc
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/simple.avsc b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/simple.avsc
new file mode 100644
index 0000000..cb1bd14
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/resources/simple.avsc
@@ -0,0 +1,14 @@
+{
+    "type": "record",
+    "name": "SimpleRecord",
+    "fields": [
+        {
+            "name": "username",
+            "type": "string"
+        },
+        {
+            "name": "password",
+            "type": "string"
+        }
+    ]
+}
\ No newline at end of file


[2/2] nifi git commit: NIFI-5214 Added REST LookupService

Posted by ij...@apache.org.
NIFI-5214 Added REST LookupService

NIFI-5214 Added support for the new ProxyConfigurationService

NIFI-5214 Integration tests added.

NIFI-5214 Added missing pom.xml and a change from a code review.

NIFI-5214 Added another tag based on code review.

NIFI-5214 Added user-defined header support.

NIFI-5214 Added Basic Auth support.

NIFI-5214 Moved documentation.

NIFI-5214 Fixed checkstyle issues; added changes requested in a review.

NIFI-5214 Added changes requested in a code review.

NIFI-5214 Added verb test and @DynamicProperties

NIFI-5214 Added templated URL support to RestLookupService.

NIFI-5214 Fixed documentation based on code review changes.

NIFI-5214 Changed RestLookupService to use a property descriptor.

NIFI-5214 Updated documentation.

NIFI-5214 Made changes requested in code review.

NIFI-5214 Renamed nifi-standard-web-utils to reflect that it is for tests.

NIFI-5214: Refactor RestLookupService.

1. Added 'Base URL' property to address environment specific part of URL.
2. Removed 'Record Path Property Name' property, because the name of
a resulted record field of a record path can be obtained by field name.
3. Lower cased HTTP method name should be used throughout.
4. Added mimeType require check when body is specified.
5. Added debug log to print HTTP response code.
6. Prepare for NIFI-5287.
7. Fixed that mime.type being used regardless of whether body is
specified or not, caused NullPointerException when 'mime.type' is not
specified when it is not required.
8. Updated documentation.

NIFI-5214 Fixed AWS processors that broke w/ change to TestServer location.

NIFI-5214 Added changes requested in a code review.

Refactored the way to evaluate EL for URL property

- Use PropertyValue instead of PreparedQuery to utilize Variable
Registry.
- Removed BASE_URL because Variable Registry can be used at URL

NIFI-5214 Rebased to use new LookupService method.

This closes #2723.

Signed-off-by: Koji Kawamura <ij...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/be63378a
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/be63378a
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/be63378a

Branch: refs/heads/master
Commit: be63378a1e2d822103427aaac74599f8125f5ebb
Parents: be31c1e
Author: Mike Thomsen <mi...@gmail.com>
Authored: Fri May 18 20:08:41 2018 -0400
Committer: Koji Kawamura <ij...@apache.org>
Committed: Fri Jun 15 15:18:03 2018 +0900

----------------------------------------------------------------------
 .../nifi-aws-bundle/nifi-aws-processors/pom.xml |   3 +-
 .../aws/wag/TestInvokeAWSGatewayApiCommon.java  |   2 +-
 ...keAmazonGatewayApiWithControllerService.java |   2 +-
 ...nvokeInvokeAmazonGatewayApiWithCredFile.java |   2 +-
 ...okeInvokeAmazonGatewayApiWithStaticAuth.java |   2 +-
 .../nifi-slack-processors/pom.xml               |   3 +-
 .../nifi/processors/slack/PutSlackTest.java     |   2 +-
 .../nifi-livy-processors/pom.xml                |   7 +
 .../livy/TestExecuteSparkInteractive.java       |   2 +-
 .../livy/TestExecuteSparkInteractiveSSL.java    |   2 +-
 .../nifi-standard-processors/pom.xml            |   6 +
 .../nifi/processors/standard/TestGetHTTP.java   |   1 +
 .../processors/standard/TestInvokeHTTP.java     |   1 +
 .../processors/standard/TestInvokeHttpSSL.java  |   1 +
 .../nifi/processors/standard/TestPostHTTP.java  |   1 +
 .../nifi/processors/standard/TestServer.java    | 163 -------
 .../standard/util/TestInvokeHttpCommon.java     |   2 +-
 .../nifi-standard-web-test-utils/pom.xml        |  39 ++
 .../org/apache/nifi/web/util/TestServer.java    | 163 +++++++
 nifi-nar-bundles/nifi-standard-bundle/pom.xml   |   1 +
 .../nifi-lookup-services/pom.xml                | 102 +++++
 .../apache/nifi/lookup/RestLookupService.java   | 449 +++++++++++++++++++
 ...org.apache.nifi.controller.ControllerService |   1 +
 .../additionalDetails.html                      |  65 +++
 .../nifi/lookup/RestLookupServiceIT.groovy      | 365 +++++++++++++++
 .../nifi/lookup/TestRestLookupService.groovy    | 144 ++++++
 .../TestRestLookupServiceProcessor.groovy       |  45 ++
 .../lookup/rest/MockRestLookupService.groovy    |  31 ++
 .../apache/nifi/lookup/rest/SchemaUtil.groovy   |  24 +
 .../nifi/lookup/rest/handlers/BasicAuth.groovy  |  55 +++
 .../lookup/rest/handlers/ComplexJson.groovy     |  45 ++
 .../nifi/lookup/rest/handlers/NoRecord.groovy   |  39 ++
 .../nifi/lookup/rest/handlers/SimpleJson.groovy |  44 ++
 .../lookup/rest/handlers/SimpleJsonArray.groovy |  42 ++
 .../nifi/lookup/rest/handlers/VerbTest.groovy   |  69 +++
 .../src/test/resources/complex.avsc             |  36 ++
 .../src/test/resources/simple.avsc              |  14 +
 37 files changed, 1800 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
index db2b989..a1ba775 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml
@@ -52,9 +52,8 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-processors</artifactId>
+            <artifactId>nifi-standard-web-test-utils</artifactId>
             <version>1.7.0-SNAPSHOT</version>
-            <type>test-jar</type>
             <scope>test</scope>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAWSGatewayApiCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAWSGatewayApiCommon.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAWSGatewayApiCommon.java
index a071b0e..f230cf1 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAWSGatewayApiCommon.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeAWSGatewayApiCommon.java
@@ -34,12 +34,12 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processors.aws.AbstractAWSProcessor;
 import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.web.util.TestServer;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Request;
 import org.eclipse.jetty.server.handler.AbstractHandler;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithControllerService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithControllerService.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithControllerService.java
index 9dd2f8e..403396b 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithControllerService.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithControllerService.java
@@ -17,8 +17,8 @@
 package org.apache.nifi.processors.aws.wag;
 
 import java.io.IOException;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.util.TestServer;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithCredFile.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithCredFile.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithCredFile.java
index e33c6b3..822ea76 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithCredFile.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithCredFile.java
@@ -17,8 +17,8 @@
 package org.apache.nifi.processors.aws.wag;
 
 import java.io.IOException;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.util.TestServer;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithStaticAuth.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithStaticAuth.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithStaticAuth.java
index 6bb5232..43dd792 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithStaticAuth.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/wag/TestInvokeInvokeAmazonGatewayApiWithStaticAuth.java
@@ -17,8 +17,8 @@
 package org.apache.nifi.processors.aws.wag;
 
 import java.io.IOException;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.util.TestServer;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml
index 8c2c493..1d8c679 100644
--- a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml
@@ -89,10 +89,9 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-processors</artifactId>
+            <artifactId>nifi-standard-web-test-utils</artifactId>
             <version>1.7.0-SNAPSHOT</version>
             <scope>test</scope>
-            <type>test-jar</type>
         </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/test/java/org/apache/nifi/processors/slack/PutSlackTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/test/java/org/apache/nifi/processors/slack/PutSlackTest.java b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/test/java/org/apache/nifi/processors/slack/PutSlackTest.java
index a6b330d..f6b09dc 100644
--- a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/test/java/org/apache/nifi/processors/slack/PutSlackTest.java
+++ b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/test/java/org/apache/nifi/processors/slack/PutSlackTest.java
@@ -19,9 +19,9 @@ package org.apache.nifi.processors.slack;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.FlowFile;
 import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.util.TestServer;
 import org.eclipse.jetty.servlet.ServletHandler;
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/pom.xml b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/pom.xml
index 56e115d..70f47f2 100644
--- a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/pom.xml
@@ -89,6 +89,13 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-kerberos-credentials-service-api</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-web-test-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
             <scope>test</scope>
         </dependency>
     </dependencies>

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractive.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractive.java b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractive.java
index fc454ab..1be718a 100644
--- a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractive.java
+++ b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractive.java
@@ -17,10 +17,10 @@
 package org.apache.nifi.processors.livy;
 
 import org.apache.nifi.controller.livy.LivySessionController;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.util.TestServer;
 import org.eclipse.jetty.server.Handler;
 import org.junit.After;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractiveSSL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractiveSSL.java b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractiveSSL.java
index 133c773..3e84cba 100644
--- a/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractiveSSL.java
+++ b/nifi-nar-bundles/nifi-spark-bundle/nifi-livy-processors/src/test/java/org/apache/nifi/processors/livy/TestExecuteSparkInteractiveSSL.java
@@ -17,11 +17,11 @@
 package org.apache.nifi.processors.livy;
 
 import org.apache.nifi.controller.livy.LivySessionController;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.ssl.StandardSSLContextService;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.web.util.TestServer;
 import org.eclipse.jetty.server.Handler;
 import org.junit.After;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index babc438..67c5112 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -321,6 +321,12 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-schema-registry-service-api</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-web-test-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
index 3e33948..e1d76e0 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestGetHTTP.java
@@ -18,6 +18,7 @@ package org.apache.nifi.processors.standard;
 
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.web.util.TestServer;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.ssl.StandardSSLContextService;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
index bb4f7ee..70a183a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHTTP.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processors.standard;
 
 import org.apache.nifi.processors.standard.util.TestInvokeHttpCommon;
+import org.apache.nifi.web.util.TestServer;
 import org.apache.nifi.ssl.StandardSSLContextService;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunners;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
index 7f0bcbb..43380ab 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestInvokeHttpSSL.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.processors.standard;
 
 import org.apache.nifi.processors.standard.util.TestInvokeHttpCommon;
+import org.apache.nifi.web.util.TestServer;
 import org.apache.nifi.ssl.StandardSSLContextService;
 import org.apache.nifi.util.TestRunners;
 import org.junit.After;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
index abb1951..ef34487 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPostHTTP.java
@@ -31,6 +31,7 @@ import java.util.Set;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.web.util.TestServer;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.ssl.StandardSSLContextService;
 import org.apache.nifi.util.FlowFileUnpackagerV3;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java
deleted file mode 100644
index 4410ca5..0000000
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestServer.java
+++ /dev/null
@@ -1,163 +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.standard;
-
-import java.util.Map;
-import org.apache.nifi.ssl.StandardSSLContextService;
-import org.eclipse.jetty.server.Handler;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.handler.HandlerCollection;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-
-/**
- * Test server to assist with unit tests that requires a server to be stood up.
- */
-public class TestServer {
-
-    public static final String NEED_CLIENT_AUTH = "clientAuth";
-
-    private Server jetty;
-    private boolean secure = false;
-
-    /**
-     * Creates the test server.
-     */
-    public TestServer() {
-        createServer(null);
-    }
-
-    /**
-     * Creates the test server.
-     *
-     * @param sslProperties SSLProps to be used in the secure connection. The keys should should use the StandardSSLContextService properties.
-     */
-    public TestServer(final Map<String, String> sslProperties) {
-        createServer(sslProperties);
-    }
-
-    private void createServer(final Map<String, String> sslProperties) {
-        jetty = new Server();
-
-        // create the unsecure connector
-        createConnector();
-
-        // create the secure connector if sslProperties are specified
-        if (sslProperties != null) {
-            createSecureConnector(sslProperties);
-        }
-
-        jetty.setHandler(new HandlerCollection(true));
-    }
-
-    /**
-     * Creates the http connection
-     */
-    private void createConnector() {
-        final ServerConnector http = new ServerConnector(jetty);
-        http.setPort(0);
-        // Severely taxed environments may have significant delays when executing.
-        http.setIdleTimeout(30000L);
-        jetty.addConnector(http);
-    }
-
-    private void createSecureConnector(final Map<String, String> sslProperties) {
-        SslContextFactory ssl = new SslContextFactory();
-
-        if (sslProperties.get(StandardSSLContextService.KEYSTORE.getName()) != null) {
-            ssl.setKeyStorePath(sslProperties.get(StandardSSLContextService.KEYSTORE.getName()));
-            ssl.setKeyStorePassword(sslProperties.get(StandardSSLContextService.KEYSTORE_PASSWORD.getName()));
-            ssl.setKeyStoreType(sslProperties.get(StandardSSLContextService.KEYSTORE_TYPE.getName()));
-        }
-
-        if (sslProperties.get(StandardSSLContextService.TRUSTSTORE.getName()) != null) {
-            ssl.setTrustStorePath(sslProperties.get(StandardSSLContextService.TRUSTSTORE.getName()));
-            ssl.setTrustStorePassword(sslProperties.get(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName()));
-            ssl.setTrustStoreType(sslProperties.get(StandardSSLContextService.TRUSTSTORE_TYPE.getName()));
-        }
-
-        final String clientAuth = sslProperties.get(NEED_CLIENT_AUTH);
-        if (clientAuth == null) {
-            ssl.setNeedClientAuth(true);
-        } else {
-            ssl.setNeedClientAuth(Boolean.parseBoolean(clientAuth));
-        }
-
-        // build the connector
-        final ServerConnector https = new ServerConnector(jetty, ssl);
-
-        // set host and port
-        https.setPort(0);
-        // Severely taxed environments may have significant delays when executing.
-        https.setIdleTimeout(30000L);
-
-        // add the connector
-        jetty.addConnector(https);
-
-        // mark secure as enabled
-        secure = true;
-    }
-
-    public void clearHandlers() {
-        HandlerCollection hc = (HandlerCollection) jetty.getHandler();
-        Handler[] ha = hc.getHandlers();
-        if (ha != null) {
-            for (Handler h : ha) {
-                hc.removeHandler(h);
-            }
-        }
-    }
-
-    public void addHandler(Handler handler) {
-        ((HandlerCollection) jetty.getHandler()).addHandler(handler);
-    }
-
-    public void startServer() throws Exception {
-        jetty.start();
-    }
-
-    public void shutdownServer() throws Exception {
-        jetty.stop();
-        jetty.destroy();
-    }
-
-    private int getPort() {
-        if (!jetty.isStarted()) {
-            throw new IllegalStateException("Jetty server not started");
-        }
-        return ((ServerConnector) jetty.getConnectors()[0]).getLocalPort();
-    }
-
-    private int getSecurePort() {
-        if (!jetty.isStarted()) {
-            throw new IllegalStateException("Jetty server not started");
-        }
-        return ((ServerConnector) jetty.getConnectors()[1]).getLocalPort();
-    }
-
-    public String getUrl() {
-        return "http://localhost:" + getPort();
-    }
-
-    public String getSecureUrl() {
-        String url = null;
-        if (secure) {
-            url = "https://localhost:" + getSecurePort();
-        }
-        return url;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
index 5242b8f..3304e62 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/TestInvokeHttpCommon.java
@@ -19,11 +19,11 @@ package org.apache.nifi.processors.standard.util;
 
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.processors.standard.InvokeHTTP;
-import org.apache.nifi.processors.standard.TestServer;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.web.util.TestServer;
 import org.eclipse.jetty.security.ConstraintSecurityHandler;
 import org.eclipse.jetty.security.DefaultIdentityService;
 import org.eclipse.jetty.security.HashLoginService;

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/pom.xml
new file mode 100644
index 0000000..95703e0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/pom.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- Licensed to the Apache Software Foundation (ASF) under one or more contributor
+license agreements. See the NOTICE file distributed with this work for additional
+information regarding copyright ownership. The ASF licenses this file to
+You under the Apache License, Version 2.0 (the "License"); you may not use
+this file except in compliance with the License. You may obtain a copy of
+the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+by applicable law or agreed to in writing, software distributed under the
+License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+OF ANY KIND, either express or implied. See the License for the specific
+language governing permissions and limitations under the License. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi-standard-bundle</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.7.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>nifi-standard-web-test-utils</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java
new file mode 100644
index 0000000..5fc74a5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-test-utils/src/main/java/org/apache/nifi/web/util/TestServer.java
@@ -0,0 +1,163 @@
+/*
+ * 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.web.util;
+
+import java.util.Map;
+import org.apache.nifi.ssl.StandardSSLContextService;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.handler.HandlerCollection;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+
+/**
+ * Test server to assist with unit tests that requires a server to be stood up.
+ */
+public class TestServer {
+
+    public static final String NEED_CLIENT_AUTH = "clientAuth";
+
+    private Server jetty;
+    private boolean secure = false;
+
+    /**
+     * Creates the test server.
+     */
+    public TestServer() {
+        createServer(null);
+    }
+
+    /**
+     * Creates the test server.
+     *
+     * @param sslProperties SSLProps to be used in the secure connection. The keys should should use the StandardSSLContextService properties.
+     */
+    public TestServer(final Map<String, String> sslProperties) {
+        createServer(sslProperties);
+    }
+
+    private void createServer(final Map<String, String> sslProperties) {
+        jetty = new Server();
+
+        // create the unsecure connector
+        createConnector();
+
+        // create the secure connector if sslProperties are specified
+        if (sslProperties != null) {
+            createSecureConnector(sslProperties);
+        }
+
+        jetty.setHandler(new HandlerCollection(true));
+    }
+
+    /**
+     * Creates the http connection
+     */
+    private void createConnector() {
+        final ServerConnector http = new ServerConnector(jetty);
+        http.setPort(0);
+        // Severely taxed environments may have significant delays when executing.
+        http.setIdleTimeout(30000L);
+        jetty.addConnector(http);
+    }
+
+    private void createSecureConnector(final Map<String, String> sslProperties) {
+        SslContextFactory ssl = new SslContextFactory();
+
+        if (sslProperties.get(StandardSSLContextService.KEYSTORE.getName()) != null) {
+            ssl.setKeyStorePath(sslProperties.get(StandardSSLContextService.KEYSTORE.getName()));
+            ssl.setKeyStorePassword(sslProperties.get(StandardSSLContextService.KEYSTORE_PASSWORD.getName()));
+            ssl.setKeyStoreType(sslProperties.get(StandardSSLContextService.KEYSTORE_TYPE.getName()));
+        }
+
+        if (sslProperties.get(StandardSSLContextService.TRUSTSTORE.getName()) != null) {
+            ssl.setTrustStorePath(sslProperties.get(StandardSSLContextService.TRUSTSTORE.getName()));
+            ssl.setTrustStorePassword(sslProperties.get(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName()));
+            ssl.setTrustStoreType(sslProperties.get(StandardSSLContextService.TRUSTSTORE_TYPE.getName()));
+        }
+
+        final String clientAuth = sslProperties.get(NEED_CLIENT_AUTH);
+        if (clientAuth == null) {
+            ssl.setNeedClientAuth(true);
+        } else {
+            ssl.setNeedClientAuth(Boolean.parseBoolean(clientAuth));
+        }
+
+        // build the connector
+        final ServerConnector https = new ServerConnector(jetty, ssl);
+
+        // set host and port
+        https.setPort(0);
+        // Severely taxed environments may have significant delays when executing.
+        https.setIdleTimeout(30000L);
+
+        // add the connector
+        jetty.addConnector(https);
+
+        // mark secure as enabled
+        secure = true;
+    }
+
+    public void clearHandlers() {
+        HandlerCollection hc = (HandlerCollection) jetty.getHandler();
+        Handler[] ha = hc.getHandlers();
+        if (ha != null) {
+            for (Handler h : ha) {
+                hc.removeHandler(h);
+            }
+        }
+    }
+
+    public void addHandler(Handler handler) {
+        ((HandlerCollection) jetty.getHandler()).addHandler(handler);
+    }
+
+    public void startServer() throws Exception {
+        jetty.start();
+    }
+
+    public void shutdownServer() throws Exception {
+        jetty.stop();
+        jetty.destroy();
+    }
+
+    public int getPort() {
+        if (!jetty.isStarted()) {
+            throw new IllegalStateException("Jetty server not started");
+        }
+        return ((ServerConnector) jetty.getConnectors()[0]).getLocalPort();
+    }
+
+    public int getSecurePort() {
+        if (!jetty.isStarted()) {
+            throw new IllegalStateException("Jetty server not started");
+        }
+        return ((ServerConnector) jetty.getConnectors()[1]).getLocalPort();
+    }
+
+    public String getUrl() {
+        return "http://localhost:" + getPort();
+    }
+
+    public String getSecureUrl() {
+        String url = null;
+        if (secure) {
+            url = "https://localhost:" + getSecurePort();
+        }
+        return url;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/pom.xml
index 0cadde3..b80e4c9 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/pom.xml
@@ -30,6 +30,7 @@
         <module>nifi-standard-nar</module>
         <module>nifi-jolt-transform-json-ui</module>
         <module>nifi-standard-utils</module>
+        <module>nifi-standard-web-test-utils</module>
     </modules>
     <properties>
         <jackson.version>2.9.5</jackson.version>

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/pom.xml
index dbcfa8d..29b6435 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/pom.xml
@@ -42,6 +42,11 @@
             <artifactId>nifi-record</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-expression-language</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.commons</groupId>
             <artifactId>commons-configuration2</artifactId>
             <version>2.1.1</version>
@@ -68,6 +73,11 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>okhttp</artifactId>
+            <version>3.10.0</version>
+        </dependency>
+        <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
             <version>1.7.0-SNAPSHOT</version>
@@ -83,6 +93,65 @@
             <artifactId>junit</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-service-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-ssl-context-service-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-path</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock-record-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-proxy-configuration-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-record-serialization-services</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-registry-service-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-standard-web-test-utils</artifactId>
+            <version>1.7.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-servlet</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.burgstaller</groupId>
+            <artifactId>okhttp-digest</artifactId>
+            <version>1.13</version>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>
@@ -91,6 +160,8 @@
                   <artifactId>apache-rat-plugin</artifactId>
                   <configuration>
                       <excludes combine.children="append">
+                          <exclude>src/test/resources/complex.avsc</exclude>
+                          <exclude>src/test/resources/simple.avsc</exclude>
                           <exclude>src/test/resources/test.csv</exclude>
                           <exclude>src/test/resources/test_Windows-31J.csv</exclude>
                           <exclude>src/test/resources/test.properties</exclude>
@@ -98,6 +169,37 @@
                       </excludes>
                   </configuration>
               </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>compile</goal>
+                            <goal>testCompile</goal>
+                        </goals>
+                        <configuration>
+                            <compilerId>groovy-eclipse-compiler</compilerId>
+                        </configuration>
+                    </execution>
+                </executions>
+                <configuration>
+                    <source>1.8</source>
+                    <target>1.8</target>
+                </configuration>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.codehaus.groovy</groupId>
+                        <artifactId>groovy-eclipse-compiler</artifactId>
+                        <version>2.9.2-01</version>
+                    </dependency>
+                    <dependency>
+                        <groupId>org.codehaus.groovy</groupId>
+                        <artifactId>groovy-eclipse-batch</artifactId>
+                        <version>2.4.3-01</version>
+                    </dependency>
+                </dependencies>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java
new file mode 100644
index 0000000..656ad5f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java
@@ -0,0 +1,449 @@
+/*
+ * 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.lookup;
+
+import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
+import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
+import com.burgstaller.okhttp.digest.CachingAuthenticator;
+import com.burgstaller.okhttp.digest.DigestAuthenticator;
+import okhttp3.Credentials;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import okhttp3.ResponseBody;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+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.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.Validator;
+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.proxy.ProxyConfiguration;
+import org.apache.nifi.proxy.ProxyConfigurationService;
+import org.apache.nifi.proxy.ProxySpec;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.SimpleRecordSchema;
+import org.apache.nifi.serialization.record.MapRecord;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.ssl.SSLContextService;
+import org.apache.nifi.util.StringUtils;
+
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Proxy;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.lang3.StringUtils.trimToEmpty;
+
+@Tags({ "rest", "lookup", "json", "xml", "http" })
+@CapabilityDescription("Use a REST service to look up values.")
+@DynamicProperties({
+    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
+            "as the header name and the value as the header value.")
+})
+public class RestLookupService extends AbstractControllerService implements RecordLookupService {
+    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
+        .name("rest-lookup-url")
+        .displayName("URL")
+        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
+                "not flowfile attributes.")
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .required(true)
+        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+        .build();
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+        .name("rest-lookup-record-reader")
+        .displayName("Record Reader")
+        .description("The record reader to use for loading the payload and handling it as a record set.")
+        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+        .identifiesControllerService(RecordReaderFactory.class)
+        .required(true)
+        .build();
+
+    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
+        .name("rest-lookup-record-path")
+        .displayName("Record Path")
+        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
+                "into the record set to be enriched. See documentation for examples of when this might be useful.")
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(new RecordPathValidator())
+        .required(false)
+        .build();
+
+    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+        .name("rest-lookup-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 PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
+        .name("rest-lookup-basic-auth-username")
+        .displayName("Basic Authentication Username")
+        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
+        .required(false)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
+        .build();
+
+    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
+        .name("rest-lookup-basic-auth-password")
+        .displayName("Basic Authentication Password")
+        .description("The password to be used by the client to authenticate against the Remote URL.")
+        .required(false)
+        .sensitive(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
+        .build();
+    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
+        .name("rest-lookup-digest-auth")
+        .displayName("Use Digest Authentication")
+        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
+                + "for authentication.")
+        .required(false)
+        .defaultValue("false")
+        .allowableValues("true", "false")
+        .build();
+
+    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
+    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
+            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
+
+    static final String MIME_TYPE_KEY = "mime.type";
+    static final String BODY_KEY = "request.body";
+    static final String METHOD_KEY = "request.method";
+
+    static final List<PropertyDescriptor> DESCRIPTORS;
+    static final Set<String> KEYS;
+
+    static final List VALID_VERBS = Arrays.asList("delete", "get", "post", "put");
+
+    static {
+        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+            URL,
+            RECORD_READER,
+            RECORD_PATH,
+            SSL_CONTEXT_SERVICE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROP_BASIC_AUTH_USERNAME,
+            PROP_BASIC_AUTH_PASSWORD,
+            PROP_DIGEST_AUTH
+        ));
+        KEYS = Collections.emptySet();
+    }
+
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    private volatile ProxyConfigurationService proxyConfigurationService;
+    private volatile RecordReaderFactory readerFactory;
+    private volatile RecordPath recordPath;
+    private volatile OkHttpClient client;
+    private volatile Map<String, String> headers;
+    private volatile PropertyValue urlTemplate;
+    private volatile String basicUser;
+    private volatile String basicPass;
+    private volatile boolean isDigest;
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
+        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
+                .asControllerService(ProxyConfigurationService.class);
+
+        OkHttpClient.Builder builder = new OkHttpClient.Builder();
+
+        setAuthenticator(builder, context);
+
+        if (proxyConfigurationService != null) {
+            setProxy(builder);
+        }
+
+        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
+        if (sslService != null) {
+            builder.sslSocketFactory(sslContext.getSocketFactory());
+        }
+
+        client = builder.build();
+
+        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
+        if (!StringUtils.isBlank(path)) {
+            recordPath = RecordPath.compile(path);
+        }
+
+        buildHeaders(context);
+
+        urlTemplate = context.getProperty(URL);
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        this.recordPath = null;
+        this.urlTemplate = null;
+    }
+
+    private void buildHeaders(ConfigurationContext context) {
+        headers = new HashMap<>();
+        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
+            if (descriptor.isDynamic()) {
+                headers.put(
+                    descriptor.getDisplayName(),
+                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
+                );
+            }
+        }
+    }
+
+    private void setProxy(OkHttpClient.Builder builder) {
+        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
+        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
+            final Proxy proxy = config.createProxy();
+            builder.proxy(proxy);
+
+            if (config.hasCredential()){
+                builder.proxyAuthenticator((route, response) -> {
+                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
+                    return response.request().newBuilder()
+                            .header("Proxy-Authorization", credential)
+                            .build();
+                });
+            }
+        }
+    }
+
+
+    @Override
+    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
+        return lookup(coordinates, null);
+    }
+
+    @Override
+    public Optional<Record> lookup(Map<String, Object> coordinates, Map<String, String> context) throws LookupFailureException {
+        final String endpoint = determineEndpoint(coordinates);
+        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
+        final String method   = ((String)coordinates.getOrDefault(METHOD_KEY, "get")).trim().toLowerCase();
+        final String body     = (String)coordinates.get(BODY_KEY);
+
+        validateVerb(method);
+
+        if (StringUtils.isBlank(body)) {
+            if (method.equals("post") || method.equals("put")) {
+                throw new LookupFailureException(
+                        String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
+                );
+            }
+        } else {
+            if (StringUtils.isBlank(mimeType)) {
+                throw new LookupFailureException(
+                        String.format("Request body is specified without its %s.", MIME_TYPE_KEY)
+                );
+            }
+        }
+
+        Request request = buildRequest(mimeType, method, body, endpoint);
+        try {
+            Response response = executeRequest(request);
+
+            if (getLogger().isDebugEnabled()) {
+                getLogger().debug("Response code {} was returned for coordinate {}",
+                        new Object[]{response.code(), coordinates});
+            }
+
+            final ResponseBody responseBody = response.body();
+            if (responseBody == null) {
+                return Optional.empty();
+            }
+
+            InputStream is = responseBody.byteStream();
+            Record record = handleResponse(is, context);
+
+            return Optional.ofNullable(record);
+        } catch (Exception e) {
+            getLogger().error("Could not execute lookup.", e);
+            throw new LookupFailureException(e);
+        }
+    }
+
+    protected void validateVerb(String method) throws LookupFailureException {
+        if (!VALID_VERBS.contains(method)) {
+            throw new LookupFailureException(String.format("%s is not a supported HTTP verb.", method));
+        }
+    }
+
+    protected String determineEndpoint(Map<String, Object> coordinates) {
+        Map<String, String> converted = coordinates.entrySet().stream()
+            .filter(e -> e.getValue() != null)
+            .collect(Collectors.toMap(
+                e -> e.getKey(),
+                e -> e.getValue().toString()
+            ));
+        return urlTemplate.evaluateAttributeExpressions(converted).getValue();
+    }
+
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+            .name(propertyDescriptorName)
+            .displayName(propertyDescriptorName)
+            .addValidator(Validator.VALID)
+            .dynamic(true)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+    }
+
+    protected Response executeRequest(Request request) throws IOException {
+        return client.newCall(request).execute();
+    }
+
+    private Record handleResponse(InputStream is, Map<String, String> context) throws SchemaNotFoundException, MalformedRecordException, IOException {
+
+        try (RecordReader reader = readerFactory.createRecordReader(context, is, getLogger())) {
+
+            Record record = reader.nextRecord();
+
+            if (recordPath != null) {
+                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
+                if (fv.isPresent()) {
+                    FieldValue fieldValue = fv.get();
+                    RecordSchema schema = new SimpleRecordSchema(Collections.singletonList(fieldValue.getField()));
+
+                    Record temp;
+                    Object value = fieldValue.getValue();
+                    if (value instanceof Record) {
+                        temp = (Record) value;
+                    } else if (value instanceof Map) {
+                        temp = new MapRecord(schema, (Map<String, Object>) value);
+                    } else {
+                        Map<String, Object> val = new HashMap<>();
+                        val.put(fieldValue.getField().getFieldName(), value);
+                        temp = new MapRecord(schema, val);
+                    }
+
+                    record = temp;
+                } else {
+                    record = null;
+                }
+            }
+
+            return record;
+        } catch (Exception ex) {
+            is.close();
+            throw ex;
+        }
+    }
+
+    private Request buildRequest(final String mimeType, final String method, final String body, final String endpoint) {
+        RequestBody requestBody = null;
+        if (body != null) {
+            final MediaType mt = MediaType.parse(mimeType);
+            requestBody = RequestBody.create(mt, body);
+        }
+        Request.Builder request = new Request.Builder()
+                .url(endpoint);
+        switch(method) {
+            case "delete":
+                request = body != null ? request.delete(requestBody) : request.delete();
+                break;
+            case "get":
+                request = request.get();
+                break;
+            case "post":
+                request = request.post(requestBody);
+                break;
+            case "put":
+                request = request.put(requestBody);
+                break;
+        }
+
+        if (headers != null) {
+            for (Map.Entry<String, String> header : headers.entrySet()) {
+                request = request.addHeader(header.getKey(), header.getValue());
+            }
+        }
+
+        if (!basicUser.isEmpty() && !isDigest) {
+            String credential = Credentials.basic(basicUser, basicPass);
+            request = request.header("Authorization", credential);
+        }
+
+        return request.build();
+    }
+
+    private void setAuthenticator(OkHttpClient.Builder okHttpClientBuilder, ConfigurationContext context) {
+        final String authUser = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_USERNAME).evaluateAttributeExpressions().getValue());
+        this.basicUser = authUser;
+
+
+        isDigest = context.getProperty(PROP_DIGEST_AUTH).asBoolean();
+        final String authPass = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_PASSWORD).evaluateAttributeExpressions().getValue());
+        this.basicPass = authPass;
+        // If the username/password properties are set then check if digest auth is being used
+        if (!authUser.isEmpty() && isDigest) {
+
+            /*
+             * OkHttp doesn't have built-in Digest Auth Support. A ticket for adding it is here[1] but they authors decided instead to rely on a 3rd party lib.
+             *
+             * [1] https://github.com/square/okhttp/issues/205#issuecomment-154047052
+             */
+            final Map<String, CachingAuthenticator> authCache = new ConcurrentHashMap<>();
+            com.burgstaller.okhttp.digest.Credentials credentials = new com.burgstaller.okhttp.digest.Credentials(authUser, authPass);
+            final DigestAuthenticator digestAuthenticator = new DigestAuthenticator(credentials);
+
+            okHttpClientBuilder.interceptors().add(new AuthenticationCacheInterceptor(authCache));
+            okHttpClientBuilder.authenticator(new CachingAuthenticatorDecorator(digestAuthenticator, authCache));
+        }
+    }
+
+    @Override
+    public Class<?> getValueType() {
+        return Record.class;
+    }
+
+    @Override
+    public Set<String> getRequiredKeys() {
+        return KEYS;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
index 544ec91..34395e8 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -15,6 +15,7 @@
 org.apache.nifi.lookup.maxmind.IPLookupService
 org.apache.nifi.lookup.CSVRecordLookupService
 org.apache.nifi.lookup.PropertiesFileLookupService
+org.apache.nifi.lookup.RestLookupService
 org.apache.nifi.lookup.SimpleKeyValueLookupService
 org.apache.nifi.lookup.SimpleCsvFileLookupService
 org.apache.nifi.lookup.XMLFileLookupService

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs/org.apache.nifi.lookup.RestLookupService/additionalDetails.html
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs/org.apache.nifi.lookup.RestLookupService/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs/org.apache.nifi.lookup.RestLookupService/additionalDetails.html
new file mode 100644
index 0000000..b38b919
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs/org.apache.nifi.lookup.RestLookupService/additionalDetails.html
@@ -0,0 +1,65 @@
+<!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>RestLookupService</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
+</head>
+<body>
+    <h2>General</h2>
+    <p>This lookup service has the following optional lookup coordinate keys:</p>
+    <ul>
+        <li>request.method; defaults to 'get', valid values:
+            <ul>
+                <li>delete</li>
+                <li>get</li>
+                <li>post</li>
+                <li>put</li>
+            </ul>
+        </li>
+        <li>body; contains a string representing JSON, XML, etc. to be sent with any
+            of those methods except for "get".</li>
+        <li>mime.type; specifies media type of the request body, required when 'body' is passed.</li>
+        <li>*; any other keys can be configured to pass variables to resolve target URLs. See 'Dynamic URLs' section below.</li>
+    </ul>
+    <p>The record reader is used to consume the response of the REST service call and turn it into one or more records. The record path property
+    is provided to allow for a lookup path to either a nested record or a single point deep in the REST response. Note: a valid schema must be
+    built that encapsulates the REST response accurately in order for this service to work.</p>
+    <h2>Headers</h2>
+    <p>Headers are supported using dynamic properties. Just add a dynamic property and the name will be the header name and the value will be the value for the header. Expression language
+    powered by input from the variable registry is supported.</p>
+    <h2>Dynamic URLs</h2>
+    <p>The URL property supports expression language through the lookup key/value pairs configured on the component using this lookup service (e.g. LookupRecord processor). The configuration specified by the user will be passed
+    through to the expression language engine for evaluation. Note: flowfile attributes will be disregarded here for this property.</p>
+    <p>Ex. URL: <em>http://example.com/service/${user.name}/friend/${friend.id}</em>, combined with example record paths at LookupRecord processor:</p>
+    <ul>
+        <li>user.name => "/example/username"</li>
+        <li>friend.id => "/example/first_friend"</li>
+    </ul>
+    <p>Would dynamically produce an endpoint of <em>http://example.com/service/john.smith/friend/12345</em></p>
+
+    <h3>Using Variable Registry with URLs</h3>
+
+    <p>In addition to the lookup key/value pairs, Variable Registry can be referred from expression languages configured at the URL property.</p>
+    <p>Ex. URL: <em>http://${apiServerHostname}:${apiServerPort}/service/${user.name}/friend/${friend.id}</em>, combined with the previous example record paths, and variable registry:</p>
+    <ul>
+        <li>apiServerHostname => "test.example.com"</li>
+        <li>apiServerPort => "8080"</li>
+    </ul>
+    <p>Would dynamically produce an endpoint of <em>http://test.example.com:8080/service/john.smith/friend/12345</em></p>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/be63378a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy
new file mode 100644
index 0000000..e2f9a61
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy
@@ -0,0 +1,365 @@
+/*
+ * 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.lookup
+
+import org.apache.avro.Schema
+import org.apache.nifi.avro.AvroTypeUtil
+import org.apache.nifi.json.JsonTreeReader
+import org.apache.nifi.lookup.rest.SchemaUtil
+import org.apache.nifi.lookup.rest.handlers.BasicAuth
+import org.apache.nifi.lookup.rest.handlers.ComplexJson
+import org.apache.nifi.lookup.rest.handlers.NoRecord
+import org.apache.nifi.lookup.rest.handlers.SimpleJson
+import org.apache.nifi.lookup.rest.handlers.SimpleJsonArray
+import org.apache.nifi.lookup.rest.handlers.VerbTest
+import org.apache.nifi.schema.access.SchemaAccessUtils
+import org.apache.nifi.serialization.record.MockSchemaRegistry
+import org.apache.nifi.serialization.record.Record
+import org.apache.nifi.serialization.record.RecordSchema
+import org.apache.nifi.util.TestRunner
+import org.apache.nifi.util.TestRunners
+import org.apache.nifi.web.util.TestServer
+import org.eclipse.jetty.servlet.ServletHandler
+import org.junit.Assert
+import org.junit.Before
+import org.junit.Test
+
+import static groovy.json.JsonOutput.prettyPrint
+import static groovy.json.JsonOutput.toJson
+
+class RestLookupServiceIT {
+    static final JsonTreeReader reader
+    static final MockSchemaRegistry registry = new MockSchemaRegistry()
+    static final RecordSchema simpleSchema
+    static final RecordSchema nestedSchema
+
+    TestServer server
+    TestRunner runner
+    RestLookupService lookupService
+
+    static {
+        simpleSchema = AvroTypeUtil.createSchema(new Schema.Parser().parse(SchemaUtil.SIMPLE))
+        nestedSchema = AvroTypeUtil.createSchema(new Schema.Parser().parse(SchemaUtil.COMPLEX))
+        registry.addSchema("simple", simpleSchema)
+        registry.addSchema("complex", nestedSchema)
+
+        reader = new JsonTreeReader()
+    }
+
+    @Before
+    void setup() {
+        lookupService = new RestLookupService()
+
+        runner = TestRunners.newTestRunner(TestRestLookupServiceProcessor.class)
+        runner.addControllerService("jsonReader", reader)
+        runner.addControllerService("registry", registry)
+        runner.addControllerService("lookupService", lookupService)
+        runner.setProperty(reader, SchemaAccessUtils.SCHEMA_REGISTRY, "registry")
+        runner.setProperty(lookupService, SchemaAccessUtils.SCHEMA_REGISTRY, "registry")
+        runner.setProperty(lookupService, RestLookupService.RECORD_READER, "jsonReader")
+        runner.setProperty(TestRestLookupServiceProcessor.CLIENT_SERVICE, "lookupService")
+        runner.enableControllerService(registry)
+        runner.enableControllerService(reader)
+    }
+
+    @Test
+    void basicAuth() {
+        runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_USERNAME, "john.smith")
+        runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_PASSWORD, "testing1234")
+
+        TestServer server = new TestServer()
+        server.addHandler(new BasicAuth())
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, "/simple")
+
+            def coordinates = [
+                "mime.type": "application/json",
+                "request.method": "get"
+            ]
+
+            def context = [ "schema.name": "simple" ]
+
+            Optional<Record> response = lookupService.lookup(coordinates, context)
+            Assert.assertTrue(response.isPresent())
+            def record = response.get()
+            Assert.assertEquals("john.smith", record.getAsString("username"))
+            Assert.assertEquals("testing1234", record.getAsString("password"))
+
+            Throwable t
+            try {
+                runner.disableControllerService(lookupService)
+                runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_USERNAME, "john.smith2")
+                runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_PASSWORD, ":wetadfasdfadf")
+                runner.enableControllerService(lookupService)
+
+                lookupService.lookup(coordinates)
+            } catch (Throwable lfe) {
+                t = lfe
+            }
+
+            Assert.assertNotNull(t)
+            Assert.assertTrue(t.getClass().getCanonicalName(), t instanceof LookupFailureException)
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    @Test
+    void simpleJson() {
+        TestServer server = new TestServer()
+        ServletHandler handler = new ServletHandler()
+        handler.addServletWithMapping(SimpleJson.class, "/simple")
+        server.addHandler(handler)
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, "/simple")
+
+            def coordinates = [
+                "mime.type": "application/json",
+                "request.method": "get"
+            ]
+
+            def context = [ "schema.name": "simple" ]
+
+            Optional<Record> response = lookupService.lookup(coordinates, context)
+            Assert.assertTrue(response.isPresent())
+            def record = response.get()
+            Assert.assertEquals("john.smith", record.getAsString("username"))
+            Assert.assertEquals("testing1234", record.getAsString("password"))
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    @Test
+    void noRecord() {
+        TestServer server = new TestServer()
+        ServletHandler handler = new ServletHandler()
+        handler.addServletWithMapping(NoRecord.class, "/simple")
+        server.addHandler(handler)
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, "/simple")
+
+            def coordinates = [
+                "mime.type": "application/json",
+                "request.method": "get"
+            ]
+
+            def context = [ "schema.name": "simple" ]
+
+            Optional<Record> response = lookupService.lookup(coordinates, context)
+            Assert.assertTrue(response.isPresent())
+            def record = response.get()
+            Assert.assertNull(record.getAsString("username"))
+            Assert.assertNull(record.getAsString("password"))
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    @Test
+    void simpleJsonArray() {
+        TestServer server = new TestServer()
+        ServletHandler handler = new ServletHandler()
+        handler.addServletWithMapping(SimpleJsonArray.class, "/simple_array")
+        server.addHandler(handler)
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, "/simple_array")
+
+            def coordinates = [
+                "mime.type": "application/json",
+                "request.method": "get"
+            ]
+
+            def context = [ "schema.name": "simple" ]
+
+            Optional<Record> response = lookupService.lookup(coordinates, context)
+            Assert.assertTrue(response.isPresent())
+            def record = response.get()
+            Assert.assertEquals("john.smith", record.getAsString("username"))
+            Assert.assertEquals("testing1234", record.getAsString("password"))
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    @Test
+    void testHeaders() {
+        runner.setProperty(lookupService, "X-USER", "jane.doe")
+        runner.setProperty(lookupService, "X-PASS", "testing7890")
+
+        TestServer server = new TestServer()
+        ServletHandler handler = new ServletHandler()
+        handler.addServletWithMapping(SimpleJson.class, "/simple")
+        server.addHandler(handler)
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, "/simple")
+
+            def coordinates = [
+                "mime.type": "application/json",
+                "request.method": "get"
+            ]
+
+            def context = [ "schema.name": "simple" ]
+
+            Optional<Record> response = lookupService.lookup(coordinates, context)
+            Assert.assertTrue(response.isPresent())
+            def record = response.get()
+            Assert.assertEquals("jane.doe", record.getAsString("username"))
+            Assert.assertEquals("testing7890", record.getAsString("password"))
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    @Test
+    void complexJson() {
+        runner.setProperty(lookupService, RestLookupService.RECORD_PATH, "/top/middle/inner")
+
+        TestServer server = new TestServer()
+        ServletHandler handler = new ServletHandler()
+        handler.addServletWithMapping(ComplexJson.class, "/complex")
+        server.addHandler(handler)
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, "/complex")
+
+            def coordinates = [
+                "mime.type": "application/json",
+                "request.method": "get"
+            ]
+
+            def context = [ "schema.name": "complex" ]
+
+            Optional<Record> response = lookupService.lookup(coordinates, context)
+            Assert.assertTrue(response.isPresent())
+            def record = response.get()
+            Assert.assertEquals("jane.doe", record.getAsString("username"))
+            Assert.assertEquals("testing7890", record.getAsString("password"))
+            Assert.assertEquals("jane.doe@test-example.com", record.getAsString("email"))
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    @Test
+    void testOtherVerbs() {
+        TestServer server = new TestServer()
+        ServletHandler handler = new ServletHandler()
+        handler.addServletWithMapping(VerbTest.class, "/simple")
+        server.addHandler(handler)
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, "/simple")
+
+            def validation = { String verb, boolean addBody, boolean addMimeType, boolean valid ->
+                def coordinates = [
+                    "mime.type"     : addMimeType ? "application/json" : null,
+                    "request.method": verb
+                ]
+
+                def context = [ "schema.name": "simple" ]
+
+                if (addBody) {
+                    coordinates["request.body"] = prettyPrint(toJson([ msg: "Hello, world" ]))
+                }
+
+                try {
+                    Optional<Record> response = lookupService.lookup(coordinates, context)
+                    if (!valid) {
+                        Assert.fail("Validation should fail.")
+                    }
+
+                    Assert.assertTrue(response.isPresent())
+                    def record = response.get()
+                    Assert.assertEquals("john.smith", record.getAsString("username"))
+                    Assert.assertEquals("testing1234", record.getAsString("password"))
+
+                } catch (LookupFailureException e) {
+                    if (valid) {
+                        Assert.fail("Validation should be successful.")
+                    }
+                }
+            }
+
+            // Delete does not require body nor mimeType.
+            validation("delete", false, false, true)
+
+            // Post and Put require body and mimeType.
+            ["post", "put"].each { verb ->
+                validation(verb, false, false, false)
+                validation(verb, true, false, false)
+                validation(verb, true, true, true)
+            }
+
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    @Test
+    void testTemplateMode() {
+        TestServer server = new TestServer()
+        ServletHandler handler = new ServletHandler()
+        handler.addServletWithMapping(SimpleJson.class, "/simple/john.smith/friends/12345")
+        server.addHandler(handler)
+        try {
+            server.startServer()
+
+            setEndpoint(server.port, '/simple/${user.name}/friends/${friend.id}')
+
+            def coordinates = [
+                "mime.type": "application/json",
+                "request.method": "get",
+                "user.name": "john.smith",
+                "friend.id": 12345,
+                "endpoint.template": true
+            ]
+
+            def context = [ "schema.name": "simple" ]
+
+            Optional<Record> response = lookupService.lookup(coordinates, context)
+            Assert.assertTrue(response.isPresent())
+            def record = response.get()
+            Assert.assertEquals("john.smith", record.getAsString("username"))
+            Assert.assertEquals("testing1234", record.getAsString("password"))
+        } finally {
+            server.shutdownServer()
+        }
+    }
+
+    void setEndpoint(Integer serverPort, String endpoint) {
+        // Resolve environmental part of the URL via variable registry.
+        runner.setVariable("serverPort", String.valueOf(serverPort))
+        runner.setProperty(lookupService, RestLookupService.URL, "http://localhost:${serverPort}" + endpoint)
+        runner.enableControllerService(lookupService)
+
+        runner.assertValid()
+    }
+}