You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2017/04/11 04:37:43 UTC

nifi git commit: NIFI-3633 This closes #1610. Adding HttpNotificationService.

Repository: nifi
Updated Branches:
  refs/heads/master 8a5398eba -> 5419891fe


NIFI-3633 This closes #1610. Adding HttpNotificationService.

Signed-off-by: joewitt <jo...@apache.org>


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

Branch: refs/heads/master
Commit: 5419891fe760746f3fd4bb5c397d9a43b79f8281
Parents: 8a5398e
Author: Joe Percivall <JP...@apache.org>
Authored: Tue Mar 21 22:50:06 2017 -0400
Committer: joewitt <jo...@apache.org>
Committed: Tue Apr 11 00:37:12 2017 -0400

----------------------------------------------------------------------
 nifi-assembly/pom.xml                           |   2 +
 nifi-bootstrap/pom.xml                          |  13 +
 .../bootstrap/NotificationServiceManager.java   |  21 +-
 .../notification/NotificationService.java       |   3 +-
 .../email/EmailNotificationService.java         |   3 +-
 .../http/HttpNotificationService.java           | 288 +++++++++++++++++++
 .../TestCustomNotificationService.java          |   2 +-
 .../http/TestHttpNotificationService.java       |  61 ++++
 .../http/TestHttpNotificationServiceCommon.java | 128 +++++++++
 .../http/TestHttpNotificationServiceSSL.java    |  87 ++++++
 .../src/test/resources/localhost-ks.jks         | Bin 0 -> 3512 bytes
 .../src/test/resources/localhost-ts.jks         | Bin 0 -> 1816 bytes
 .../src/main/asciidoc/administration-guide.adoc |  55 +++-
 .../conf/bootstrap-notification-services.xml    |   7 +
 pom.xml                                         |  11 +
 15 files changed, 663 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml
index fd5ce78..070beba 100755
--- a/nifi-assembly/pom.xml
+++ b/nifi-assembly/pom.xml
@@ -672,6 +672,8 @@ language governing permissions and limitations under the License. -->
                                                     <include>ch.qos.logback:logback-core</include>
                                                     <include>org.apache.nifi:nifi-security-utils</include>
                                                     <include>org.apache.nifi:nifi-utils</include>
+                                                    <include>com.squareup.okhttp3:okhttp</include>
+                                                    <include>com.squareup.okio:okio</include>
                                                 </includes>
                                             </dependency>
                                         </mapping>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/pom.xml b/nifi-bootstrap/pom.xml
index db0645d..e76791e 100644
--- a/nifi-bootstrap/pom.xml
+++ b/nifi-bootstrap/pom.xml
@@ -34,6 +34,10 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-utils</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+        </dependency>
+        <dependency>
             <groupId>javax.mail</groupId>
             <artifactId>mail</artifactId>
         </dependency>
@@ -41,6 +45,15 @@ language governing permissions and limitations under the License. -->
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-expression-language</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>okhttp</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>mockwebserver</artifactId>
+            <scope>test</scope>
+        </dependency>
         <!-- This needs to be here because it is relied upon by the nifi-runtime which starts NiFi.  It uses this bootstrap module
         and the libs that get laid down in lib/bootstrap to create a child classloader with these bits in it -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java
index 869658a..6203a06 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NotificationServiceManager.java
@@ -202,7 +202,7 @@ public class NotificationServiceManager {
                     if (invalidReasons.isEmpty()) {
                         final NotificationContext context = buildNotificationContext(config);
                         try {
-                            service.notify(context, subject, message);
+                            service.notify(context, type, subject, message);
                             logger.info("Successfully sent notification of type {} to {}", type, service);
                         } catch (final Throwable t) {   // keep running even if a Throwable is caught because we need to ensure that we are able to restart NiFi
                             logger.error("Failed to send notification of type {} to {} with Subject {} due to {}. Will ",
@@ -264,18 +264,21 @@ public class NotificationServiceManager {
                 final Map<String, String> configuredProps = config.getProperties();
 
                 final NotificationService service = config.getService();
-                for (final PropertyDescriptor descriptor : service.getPropertyDescriptors()) {
-                    final String configuredValue = configuredProps.get(descriptor.getName());
-                    if (configuredValue == null) {
-                        props.put(descriptor, descriptor.getDefaultValue());
-                    } else {
-                        props.put(descriptor, configuredValue);
-                    }
+                final List<PropertyDescriptor> configuredPropertyDescriptors = new ArrayList<>(service.getPropertyDescriptors());
+
+                // This is needed to capture all dynamic properties
+                configuredProps.forEach((key, value) -> {
+                    PropertyDescriptor propertyDescriptor = config.service.getPropertyDescriptor(key);
+                    props.put(config.service.getPropertyDescriptor(key), value);
+                    configuredPropertyDescriptors.remove(propertyDescriptor);
+                });
+
+                for (final PropertyDescriptor descriptor : configuredPropertyDescriptors) {
+                    props.put(descriptor, descriptor.getDefaultValue());
                 }
 
                 return props;
             }
-
         };
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationService.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationService.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationService.java
index 70b6e7e..63bd9ad 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationService.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/NotificationService.java
@@ -48,9 +48,10 @@ public interface NotificationService extends ConfigurableComponent {
      * Notifies the configured recipients of some event
      *
      * @param context the context that is relevant for this notification
+     * @param notificationType the notification type
      * @param subject the subject of the message
      * @param message the message to be provided to recipients
      */
-    void notify(NotificationContext context, String subject, String message) throws NotificationFailedException;
+    void notify(NotificationContext context, NotificationType notificationType, String subject, String message) throws NotificationFailedException;
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
index d6d876e..91451fe 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/email/EmailNotificationService.java
@@ -38,6 +38,7 @@ import javax.mail.internet.MimeMessage;
 import org.apache.nifi.bootstrap.notification.AbstractNotificationService;
 import org.apache.nifi.bootstrap.notification.NotificationContext;
 import org.apache.nifi.bootstrap.notification.NotificationFailedException;
+import org.apache.nifi.bootstrap.notification.NotificationType;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
@@ -195,7 +196,7 @@ public class EmailNotificationService extends AbstractNotificationService {
     }
 
     @Override
-    public void notify(final NotificationContext context, final String subject, final String messageText) throws NotificationFailedException {
+    public void notify(final NotificationContext context, final NotificationType notificationType, final String subject, final String messageText) throws NotificationFailedException {
         final Properties properties = getMailProperties(context);
         final Session mailSession = createMailSession(properties);
         final Message message = new MimeMessage(mailSession);

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
new file mode 100644
index 0000000..22d8bf1
--- /dev/null
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
@@ -0,0 +1,288 @@
+/*
+ * 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.bootstrap.notification.http;
+
+import okhttp3.Call;
+import okhttp3.MediaType;
+import okhttp3.OkHttpClient;
+import okhttp3.Request;
+import okhttp3.RequestBody;
+import okhttp3.Response;
+import org.apache.nifi.bootstrap.notification.AbstractNotificationService;
+import org.apache.nifi.bootstrap.notification.NotificationContext;
+import org.apache.nifi.bootstrap.notification.NotificationFailedException;
+import org.apache.nifi.bootstrap.notification.NotificationInitializationContext;
+import org.apache.nifi.bootstrap.notification.NotificationType;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.SslContextFactory;
+
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class HttpNotificationService extends AbstractNotificationService {
+
+    public static final String NOTIFICATION_TYPE_KEY = "notification.type";
+    public static final String NOTIFICATION_SUBJECT_KEY = "notification.subject";
+
+    public static final String STORE_TYPE_JKS = "JKS";
+    public static final String STORE_TYPE_PKCS12 = "PKCS12";
+
+    public static final PropertyDescriptor PROP_URL = new PropertyDescriptor.Builder()
+            .name("URL")
+            .description("The URL to send the notification to.")
+            .required(true)
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.URL_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PROP_CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("Connection timeout")
+            .description("Max wait time for connection to remote service.")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("10s")
+            .build();
+    public static final PropertyDescriptor PROP_WRITE_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("Write timeout")
+            .description("Max wait time for remote service to read the request sent.")
+            .expressionLanguageSupported(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("10s")
+            .build();
+
+    public static final PropertyDescriptor PROP_TRUSTSTORE = new PropertyDescriptor.Builder()
+            .name("Truststore Filename")
+            .description("The fully-qualified filename of the Truststore")
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor PROP_TRUSTSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("Truststore Type")
+            .description("The Type of the Truststore. Either JKS or PKCS12")
+            .allowableValues(STORE_TYPE_JKS, STORE_TYPE_PKCS12)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor PROP_TRUSTSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("Truststore Password")
+            .description("The password for the Truststore")
+            .defaultValue(null)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor PROP_KEYSTORE = new PropertyDescriptor.Builder()
+            .name("Keystore Filename")
+            .description("The fully-qualified filename of the Keystore")
+            .defaultValue(null)
+            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor PROP_KEYSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("Keystore Type")
+            .description("The Type of the Keystore")
+            .allowableValues(STORE_TYPE_JKS, STORE_TYPE_PKCS12)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor PROP_KEYSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("Keystore Password")
+            .defaultValue(null)
+            .description("The password for the Keystore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+    public static final PropertyDescriptor PROP_KEY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("Key Password")
+            .displayName("Key Password")
+            .description("The password for the key. If this is not specified, but the Keystore Filename, Password, and Type are specified, "
+                    + "then the Keystore Password will be assumed to be the same as the Key Password.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    public static final PropertyDescriptor SSL_ALGORITHM = new PropertyDescriptor.Builder()
+            .name("SSL Protocol")
+            .defaultValue("TLS")
+            .allowableValues("SSL", "TLS")
+            .description("The algorithm to use for this SSL context.")
+            .sensitive(false)
+            .build();
+
+    private final AtomicReference<OkHttpClient> httpClientReference = new AtomicReference<>();
+    private final AtomicReference<String> urlReference = new AtomicReference<>();
+
+    private static final List<PropertyDescriptor> supportedProperties;
+    static {
+        supportedProperties = new ArrayList<>();
+        supportedProperties.add(PROP_URL);
+        supportedProperties.add(PROP_CONNECTION_TIMEOUT);
+        supportedProperties.add(PROP_WRITE_TIMEOUT);
+        supportedProperties.add(PROP_TRUSTSTORE);
+        supportedProperties.add(PROP_TRUSTSTORE_PASSWORD);
+        supportedProperties.add(PROP_TRUSTSTORE_TYPE);
+        supportedProperties.add(PROP_KEYSTORE);
+        supportedProperties.add(PROP_KEYSTORE_PASSWORD);
+        supportedProperties.add(PROP_KEYSTORE_TYPE);
+        supportedProperties.add(PROP_KEY_PASSWORD);
+
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return supportedProperties;
+    }
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName){
+        return new PropertyDescriptor.Builder()
+                .required(false)
+                .name(propertyDescriptorName)
+                .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
+                .dynamic(true)
+                .expressionLanguageSupported(true)
+                .build();
+    }
+
+    @Override
+    protected void init(final NotificationInitializationContext context) {
+        final String url = context.getProperty(PROP_URL).evaluateAttributeExpressions().getValue();
+        if (url == null || url.isEmpty()) {
+            throw new IllegalArgumentException("Property, \"" + PROP_URL.getDisplayName() + "\", for the URL to POST notifications to must be set.");
+        }
+
+        urlReference.set(url);
+
+        httpClientReference.set(null);
+
+        final OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient.Builder();
+
+        Long connectTimeout = context.getProperty(PROP_CONNECTION_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+        Long writeTimeout = context.getProperty(PROP_WRITE_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        // Set timeouts
+        okHttpClientBuilder.connectTimeout(connectTimeout, TimeUnit.MILLISECONDS);
+        okHttpClientBuilder.writeTimeout(writeTimeout, TimeUnit.MILLISECONDS);
+
+        // check if the keystore is set and add the factory if so
+        if (url.toLowerCase().startsWith("https")) {
+            try {
+                SSLSocketFactory sslSocketFactory = getSslSocketFactory(context);
+                okHttpClientBuilder.sslSocketFactory(sslSocketFactory);
+            } catch (Exception e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        httpClientReference.set(okHttpClientBuilder.build());
+    }
+
+    @Override
+    public void notify(NotificationContext context, NotificationType notificationType, String subject, String message) throws NotificationFailedException {
+        try {
+            final RequestBody requestBody = RequestBody.create(MediaType.parse("text/plain"), message);
+
+            Request.Builder requestBuilder = new Request.Builder()
+                    .post(requestBody)
+                    .url(urlReference.get());
+
+            Map<PropertyDescriptor, String> configuredProperties = context.getProperties();
+
+            for(PropertyDescriptor propertyDescriptor: configuredProperties.keySet()) {
+                if (propertyDescriptor.isDynamic()) {
+                    String propertyValue = context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue();
+                    requestBuilder = requestBuilder.addHeader(propertyDescriptor.getDisplayName(), propertyValue);
+                }
+            }
+
+            final Request request = requestBuilder
+                    .addHeader(NOTIFICATION_SUBJECT_KEY, subject)
+                    .addHeader(NOTIFICATION_TYPE_KEY, notificationType.name())
+                    .build();
+
+            final OkHttpClient httpClient = httpClientReference.get();
+
+            final Call call = httpClient.newCall(request);
+            final Response response = call.execute();
+
+            if (!response.isSuccessful()) {
+                throw new NotificationFailedException("Failed to send Http Notification. Received an unsuccessful status code response '" + response.code() +"'. The message was '" +
+                        response.message() + "'");
+            }
+        } catch (NotificationFailedException e){
+            throw e;
+        } catch (Exception e) {
+            throw new NotificationFailedException("Failed to send Http Notification", e);
+        }
+    }
+
+    private static SSLSocketFactory getSslSocketFactory(NotificationInitializationContext context) throws Exception {
+
+        final String protocol = context.getProperty(SSL_ALGORITHM).getValue();
+        try {
+            final PropertyValue keyPasswdProp = context.getProperty(PROP_KEY_PASSWORD);
+            final char[] keyPassword = keyPasswdProp.isSet() ? keyPasswdProp.getValue().toCharArray() : null;
+
+            final SSLContext sslContext;
+            final String truststoreFile = context.getProperty(PROP_TRUSTSTORE).getValue();
+            final String keystoreFile = context.getProperty(PROP_KEYSTORE).getValue();
+
+            if (keystoreFile == null) {
+                // If keystore not specified, create SSL Context based only on trust store.
+                sslContext = SslContextFactory.createTrustSslContext(
+                        context.getProperty(PROP_TRUSTSTORE).getValue(),
+                        context.getProperty(PROP_TRUSTSTORE_PASSWORD).getValue().toCharArray(),
+                        context.getProperty(PROP_TRUSTSTORE_TYPE).getValue(),
+                        protocol);
+
+            } else if (truststoreFile == null) {
+                // If truststore not specified, create SSL Context based only on key store.
+                sslContext = SslContextFactory.createSslContext(
+                        context.getProperty(PROP_KEYSTORE).getValue(),
+                        context.getProperty(PROP_KEYSTORE_PASSWORD).getValue().toCharArray(),
+                        keyPassword,
+                        context.getProperty(PROP_KEYSTORE_TYPE).getValue(),
+                        protocol);
+
+            } else {
+                sslContext = SslContextFactory.createSslContext(
+                        context.getProperty(PROP_KEYSTORE).getValue(),
+                        context.getProperty(PROP_KEYSTORE_PASSWORD).getValue().toCharArray(),
+                        keyPassword,
+                        context.getProperty(PROP_KEYSTORE_TYPE).getValue(),
+                        context.getProperty(PROP_TRUSTSTORE).getValue(),
+                        context.getProperty(PROP_TRUSTSTORE_PASSWORD).getValue().toCharArray(),
+                        context.getProperty(PROP_TRUSTSTORE_TYPE).getValue(),
+                        SslContextFactory.ClientAuth.REQUIRED,
+                        protocol);
+            }
+
+            return sslContext.getSocketFactory();
+        } catch (final Exception e) {
+            throw new ProcessException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/test/groovy/org/apache/nifi/bootstrap/notification/TestCustomNotificationService.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/test/groovy/org/apache/nifi/bootstrap/notification/TestCustomNotificationService.java b/nifi-bootstrap/src/test/groovy/org/apache/nifi/bootstrap/notification/TestCustomNotificationService.java
index 3685cb1..730f55b 100644
--- a/nifi-bootstrap/src/test/groovy/org/apache/nifi/bootstrap/notification/TestCustomNotificationService.java
+++ b/nifi-bootstrap/src/test/groovy/org/apache/nifi/bootstrap/notification/TestCustomNotificationService.java
@@ -75,7 +75,7 @@ public class TestCustomNotificationService extends AbstractNotificationService {
     }
 
     @Override
-    public void notify(NotificationContext context, String subject, String message) throws NotificationFailedException {
+    public void notify(NotificationContext context, NotificationType type, String subject, String message) throws NotificationFailedException {
         logger.info(context.getProperty(CUSTOM_HOSTNAME).evaluateAttributeExpressions().getValue());
         logger.info(context.getProperty(CUSTOM_USERNAME).evaluateAttributeExpressions().getValue());
         logger.info(context.getProperty(CUSTOM_PASSWORD).evaluateAttributeExpressions().getValue());

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationService.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationService.java b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationService.java
new file mode 100644
index 0000000..c7fdfd0
--- /dev/null
+++ b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationService.java
@@ -0,0 +1,61 @@
+/*
+ * 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.bootstrap.http;
+
+import okhttp3.mockwebserver.MockWebServer;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.mockito.internal.util.io.IOUtil;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+public class TestHttpNotificationService extends  TestHttpNotificationServiceCommon{
+
+
+    static final String CONFIGURATION_FILE_TEXT = "\n"+
+            "<services>\n"+
+            "         <service>\n"+
+            "            <id>http-notification</id>\n"+
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
+            "            <property name=\"URL\">${test.server}</property>\n"+
+            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
+            "         </service>\n"+
+            "</services>";
+
+
+    @BeforeClass
+    public static void startServer() throws IOException {
+        tempConfigFilePath = "./target/TestHttpNotificationService-config.xml";
+
+        Files.deleteIfExists(Paths.get(tempConfigFilePath));
+
+        mockWebServer = new MockWebServer();
+
+        String configFileOutput = CONFIGURATION_FILE_TEXT.replace("${test.server}", String.valueOf(mockWebServer.url("/")));
+        IOUtil.writeText(configFileOutput, new File(tempConfigFilePath));
+    }
+
+    @AfterClass
+    public static void shutdownServer() throws IOException {
+        Files.deleteIfExists(Paths.get(tempConfigFilePath));
+        mockWebServer.shutdown();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceCommon.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceCommon.java b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceCommon.java
new file mode 100644
index 0000000..c84c704
--- /dev/null
+++ b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceCommon.java
@@ -0,0 +1,128 @@
+/*
+ * 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.bootstrap.http;
+
+import okhttp3.mockwebserver.MockResponse;
+import okhttp3.mockwebserver.MockWebServer;
+import okhttp3.mockwebserver.RecordedRequest;
+import okio.Buffer;
+import org.apache.nifi.bootstrap.NotificationServiceManager;
+import org.apache.nifi.bootstrap.notification.NotificationType;
+import org.junit.Test;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.nifi.bootstrap.notification.http.HttpNotificationService.NOTIFICATION_SUBJECT_KEY;
+import static org.apache.nifi.bootstrap.notification.http.HttpNotificationService.NOTIFICATION_TYPE_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public abstract class TestHttpNotificationServiceCommon {
+
+    public static String tempConfigFilePath;
+    public static MockWebServer mockWebServer;
+
+    @Test
+    public void testStartNotification() throws ParserConfigurationException, SAXException, IOException, InterruptedException {
+        mockWebServer.enqueue(new MockResponse().setResponseCode(200));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+        notificationServiceManager.registerNotificationService(NotificationType.NIFI_STARTED, "http-notification");
+        notificationServiceManager.notify(NotificationType.NIFI_STARTED, "Subject", "Message");
+
+        RecordedRequest recordedRequest = mockWebServer.takeRequest(2, TimeUnit.SECONDS);
+        assertNotNull(recordedRequest);
+        assertEquals(NotificationType.NIFI_STARTED.name(), recordedRequest.getHeader(NOTIFICATION_TYPE_KEY));
+        assertEquals("Subject", recordedRequest.getHeader(NOTIFICATION_SUBJECT_KEY));
+        assertEquals("testing", recordedRequest.getHeader("testProp"));
+
+        Buffer bodyBuffer = recordedRequest.getBody();
+        String bodyString =new String(bodyBuffer.readByteArray(), UTF_8);
+        assertEquals("Message", bodyString);
+    }
+
+    @Test
+    public void testStopNotification() throws ParserConfigurationException, SAXException, IOException, InterruptedException {
+        mockWebServer.enqueue(new MockResponse().setResponseCode(200));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+        notificationServiceManager.registerNotificationService(NotificationType.NIFI_STOPPED, "http-notification");
+        notificationServiceManager.notify(NotificationType.NIFI_STOPPED, "Subject", "Message");
+
+        RecordedRequest recordedRequest = mockWebServer.takeRequest(2, TimeUnit.SECONDS);
+        assertNotNull(recordedRequest);
+        assertEquals(NotificationType.NIFI_STOPPED.name(), recordedRequest.getHeader(NOTIFICATION_TYPE_KEY));
+        assertEquals("Subject", recordedRequest.getHeader(NOTIFICATION_SUBJECT_KEY));
+
+        Buffer bodyBuffer = recordedRequest.getBody();
+        String bodyString =new String(bodyBuffer.readByteArray(), UTF_8);
+        assertEquals("Message", bodyString);
+    }
+
+    @Test
+    public void testDiedNotification() throws ParserConfigurationException, SAXException, IOException, InterruptedException {
+        mockWebServer.enqueue(new MockResponse().setResponseCode(200));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+        notificationServiceManager.registerNotificationService(NotificationType.NIFI_DIED, "http-notification");
+        notificationServiceManager.notify(NotificationType.NIFI_DIED, "Subject", "Message");
+
+        RecordedRequest recordedRequest = mockWebServer.takeRequest(2, TimeUnit.SECONDS);
+        assertNotNull(recordedRequest);
+
+        assertEquals(NotificationType.NIFI_DIED.name(), recordedRequest.getHeader(NOTIFICATION_TYPE_KEY));
+        assertEquals("Subject", recordedRequest.getHeader(NOTIFICATION_SUBJECT_KEY));
+
+        Buffer bodyBuffer = recordedRequest.getBody();
+        String bodyString =new String(bodyBuffer.readByteArray(), UTF_8);
+        assertEquals("Message", bodyString);
+    }
+
+    @Test
+    public void testStartNotificationFailure() throws ParserConfigurationException, SAXException, IOException, InterruptedException {
+        // Web server will still get the request but will return an error. Observe that it is gracefully handled.
+
+        mockWebServer.enqueue(new MockResponse().setResponseCode(500));
+
+        NotificationServiceManager notificationServiceManager = new NotificationServiceManager();
+        notificationServiceManager.setMaxNotificationAttempts(1);
+        notificationServiceManager.loadNotificationServices(new File(tempConfigFilePath));
+        notificationServiceManager.registerNotificationService(NotificationType.NIFI_STARTED, "http-notification");
+        notificationServiceManager.notify(NotificationType.NIFI_STARTED, "Subject", "Message");
+
+        RecordedRequest recordedRequest = mockWebServer.takeRequest(2, TimeUnit.SECONDS);
+        assertNotNull(recordedRequest);
+        assertEquals(NotificationType.NIFI_STARTED.name(), recordedRequest.getHeader(NOTIFICATION_TYPE_KEY));
+        assertEquals("Subject", recordedRequest.getHeader(NOTIFICATION_SUBJECT_KEY));
+
+        Buffer bodyBuffer = recordedRequest.getBody();
+        String bodyString =new String(bodyBuffer.readByteArray(), UTF_8);
+        assertEquals("Message", bodyString);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
new file mode 100644
index 0000000..7d1d452
--- /dev/null
+++ b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
@@ -0,0 +1,87 @@
+/*
+ * 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.bootstrap.http;
+
+import okhttp3.mockwebserver.MockWebServer;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.mockito.internal.util.io.IOUtil;
+
+import javax.net.ssl.SSLContext;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+
+public class TestHttpNotificationServiceSSL extends  TestHttpNotificationServiceCommon{
+
+
+    static final String CONFIGURATION_FILE_TEXT = "\n"+
+            "<services>\n"+
+            "         <service>\n"+
+            "            <id>http-notification</id>\n"+
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
+            "            <property name=\"URL\">${test.server}</property>\n"+
+            "            <property name=\"Truststore Filename\">./src/test/resources/localhost-ts.jks</property>\n"+
+            "            <property name=\"Truststore Type\">JKS</property>\n"+
+            "            <property name=\"Truststore Password\">localtest</property>\n"+
+            "            <property name=\"Keystore Filename\">./src/test/resources/localhost-ks.jks</property>\n"+
+            "            <property name=\"Keystore Type\">JKS</property>\n"+
+            "            <property name=\"Keystore Password\">localtest</property>\n"+
+            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
+            "         </service>\n"+
+            "</services>";
+
+
+    @BeforeClass
+    public static void startServer() throws IOException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException {
+        tempConfigFilePath = "./target/TestHttpNotificationService-config.xml";
+
+        Files.deleteIfExists(Paths.get(tempConfigFilePath));
+
+        mockWebServer = new MockWebServer();
+
+        final SSLContext sslContext = SslContextFactory.createSslContext(
+                "./src/test/resources/localhost-ks.jks",
+                "localtest".toCharArray(),
+                null,
+                "JKS",
+                "./src/test/resources/localhost-ts.jks",
+                "localtest".toCharArray(),
+                "JKS",
+                SslContextFactory.ClientAuth.REQUIRED,
+                "TLS");
+
+        mockWebServer.useHttps(sslContext.getSocketFactory(), false);
+
+        String configFileOutput = CONFIGURATION_FILE_TEXT.replace("${test.server}", String.valueOf(mockWebServer.url("/")));
+        IOUtil.writeText(configFileOutput, new File(tempConfigFilePath));
+    }
+
+    @AfterClass
+    public static void shutdownServer() throws IOException {
+        Files.deleteIfExists(Paths.get(tempConfigFilePath));
+        mockWebServer.shutdown();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/test/resources/localhost-ks.jks
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/test/resources/localhost-ks.jks b/nifi-bootstrap/src/test/resources/localhost-ks.jks
new file mode 100755
index 0000000..df36197
Binary files /dev/null and b/nifi-bootstrap/src/test/resources/localhost-ks.jks differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-bootstrap/src/test/resources/localhost-ts.jks
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/test/resources/localhost-ts.jks b/nifi-bootstrap/src/test/resources/localhost-ts.jks
new file mode 100755
index 0000000..7824378
Binary files /dev/null and b/nifi-bootstrap/src/test/resources/localhost-ts.jks differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-docs/src/main/asciidoc/administration-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 2de8387..dddc5f3 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -1781,8 +1781,8 @@ take effect only after NiFi has been stopped and restarted.
 Notification Services
 ---------------------
 When the NiFi bootstrap starts or stops NiFi, or detects that it has died unexpectedly, it is able to notify configured recipients. Currently,
-the only mechanism supplied is to send an e-mail notification. The notification services configuration file, however,
-is a configurable XML file so that as new notification capabilities are developed, they will be configured similarly.
+the only mechanisms supplied are to send an e-mail or HTTP POST notification. The notification services configuration file
+is an XML file where the notification capabilities are configured.
 
 The default location of the XML file is _conf/bootstrap-notification-services.xml_, but this value can be changed in the _conf/bootstrap.conf_ file.
 
@@ -1805,7 +1805,10 @@ The syntax of the XML file is as follows:
 ....
 
 Once the desired services have been configured, they can then be referenced in the _bootstrap.conf_ file.
-Currently, the only implementation is the `org.apache.nifi.bootstrap.notification.email.EmailNotificationService` implementation.
+
+=== Email Notification Service +
+
+The first Notifier is to send emails and the implementation is `org.apache.nifi.bootstrap.notification.email.EmailNotificationService`.
 It has the following properties available:
 
 |====
@@ -1846,6 +1849,46 @@ A complete example of configuring the Email service would look like the followin
      </service>
 ....
 
+=== HTTP Notification Service +
+
+The second Notifier is to send HTTP POST requests and the implementation is `org.apache.nifi.bootstrap.notification.http.HttpNotificationService`.
+It has the following properties available:
+
+|====
+|*Property*|*Required*|*Description*
+|URL|true|The URL to send the notification to. Expression language is supported.
+|Connection timeout||Max wait time for connection to remote service. Expression language is supported. This defaults to 10s.
+|Write timeout||Max wait time for remote service to read the request sent. Expression language is supported. This defaults to 10s.
+|Truststore Filename||The fully-qualified filename of the Truststore
+|Truststore Type||The Type of the Truststore. Either JKS or PKCS12
+|Truststore Password||The password for the Truststore
+|Keystore Filename||The fully-qualified filename of the Keystore
+|Keystore Type||The password for the Keystore
+|Keystore Password||The password for the key. If this is not specified, but the Keystore Filename, Password, and Type are specified, then the Keystore Password will be assumed to be the same as the Key Password.
+|SSL Protocol||The algorithm to use for this SSL context. This can either be "SSL" or "TLS".
+|====
+
+In addition to the properties above, dynamic properties can be added. They will be added as headers to the HTTP request. Expression language is supported.
+
+The notification message is in the body of the POST request. The type of notification is in the header "notification.type" and the subject uses the header "notification.subject".
+
+A complete example of configuring the HTTP service could look like the following:
+
+....
+     <service>
+        <id>http-notification</id>
+        <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>
+        <property name="URL">https://testServer.com:8080/</property>
+        <property name="Truststore Filename">localhost-ts.jks</property>
+        <property name="Truststore Type">JKS</property>
+        <property name="Truststore Password">localtest<property>
+        <property name="Keystore Filename">localhost-ts.jks</property>
+        <property name="Keystore Type">JKS</property>
+        <property name="Keystore Password">localtest</property>
+        <property name="notification.timestamp">${now()}</property>
+     </service>
+....
+
 [[kerberos_service]]
 Kerberos Service
 ----------------
@@ -2144,14 +2187,14 @@ Providing three total locations, including  _nifi.provenance.repository.director
 	so it is not advisable to write to a single "event file" for a tremendous amount of time, as it will prevent old data from aging off as smoothly.
 |nifi.provenance.repository.rollover.time|The amount of time to wait before rolling over the "event file" that the repository is writing to.
 |nifi.provenance.repository.rollover.size|The amount of data to write to a single "event file." The default value is 100 MB. For production
-	environments where a very large amount of Data Provenance is generated, a value of 1 GB is also very reasonable. 
+	environments where a very large amount of Data Provenance is generated, a value of 1 GB is also very reasonable.
 |nifi.provenance.repository.query.threads|The number of threads to use for Provenance Repository queries. The default value is 2.
 |nifi.provenance.repository.index.threads|The number of threads to use for indexing Provenance events so that they are searchable. The default value is 1.
 	For flows that operate on a very high number of FlowFiles, the indexing of Provenance events could become a bottleneck. If this happens, increasing the
 	value of this property may increase the rate at which the Provenance Repository is able to process these records, resulting in better overall throughput.
 	It is advisable to use at least 1 thread per storage location (i.e., if there are 3 storage locations, at least 3 threads should be used). For high
 	throughput environments, where more CPU and disk I/O is available, it may make sense to increase this value significantly. Typically going beyond
-	2-4 threads per storage location is not valuable. However, this can be tuned depending on the CPU resources available compared to the I/O resources. 
+	2-4 threads per storage location is not valuable. However, this can be tuned depending on the CPU resources available compared to the I/O resources.
 |nifi.provenance.repository.compress.on.rollover|Indicates whether to compress the provenance information when an "event file" is rolled over. The default value is _true_.
 |nifi.provenance.repository.always.sync|If set to _true_, any change to the repository will be synchronized to the disk, meaning that NiFi will ask the operating system
 	not to cache the information. This is very expensive and can significantly reduce NiFi performance. However, if it is _false_, there could be the potential for data
@@ -2181,7 +2224,7 @@ Providing three total locations, including  _nifi.provenance.repository.director
 	NiFi will periodically open each Lucene index and then close it, in order to "warm" the cache. This will result in far faster queries when the Provenance Repository is large. As with
 	all great things, though, it comes with a cost. Warming the cache does take some CPU resources, but more importantly it will evict other data from the Operating System disk cache and
 	will result in reading (potentially a great deal of) data from the disk. This can result in lower NiFi performance. However, if NiFi is running in an environment where CPU and disk
-	are not fully utilized, this feature can result in far faster Provenance queries. 
+	are not fully utilized, this feature can result in far faster Provenance queries.
 |====
 
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-notification-services.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-notification-services.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-notification-services.xml
index 260eaf7..bbfc65a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-notification-services.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-notification-services.xml
@@ -43,4 +43,11 @@
         <property name="To"></property>
      </service>
 -->
+<!--
+     <service>
+        <id>http-notification</id>
+        <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>
+        <property name="URL"></property>
+     </service>
+-->
 </services>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/5419891f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 91aae37..449450d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -205,6 +205,12 @@ language governing permissions and limitations under the License. -->
                 <scope>test</scope>
             </dependency>
             <dependency>
+                <groupId>com.squareup.okhttp3</groupId>
+                <artifactId>mockwebserver</artifactId>
+                <version>3.6.0</version>
+                <scope>test</scope>
+            </dependency>
+            <dependency>
                 <groupId>org.slf4j</groupId>
                 <artifactId>slf4j-simple</artifactId>
                 <version>${org.slf4j.version}</version>
@@ -1442,6 +1448,11 @@ language governing permissions and limitations under the License. -->
                 <artifactId>aws-java-sdk</artifactId>
                 <version>1.11.68</version>
             </dependency>
+            <dependency>
+                <groupId>com.squareup.okhttp3</groupId>
+                <artifactId>okhttp</artifactId>
+                <version>3.6.0</version>
+            </dependency>
             <!-- Ranger dependencies, only included when using -Pinclude-ranger -->
             <dependency>
                 <groupId>org.apache.nifi</groupId>