You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by kd...@apache.org on 2022/06/29 20:53:01 UTC

[nifi] branch main updated: NIFI-10162 Reorganized InvokeHTTP Properties

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 7a47c8cfbd NIFI-10162 Reorganized InvokeHTTP Properties
7a47c8cfbd is described below

commit 7a47c8cfbd458ab037275762c385d50372c130a3
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Mon Jun 27 17:00:59 2022 -0500

    NIFI-10162 Reorganized InvokeHTTP Properties
    
    - Corrected handling of documentation for allowable values
    
    NIFI-10162 Updated Multipart Form-Data Name description
    
    This closes #6163.
    
    Signed-off-by: Kevin Doran <kd...@apache.org>
---
 .../html/HtmlDocumentationWriter.java              |  15 +-
 .../nifi/processors/standard/InvokeHTTP.java       | 920 ++++++++++-----------
 .../nifi/processors/standard/http/HttpHeader.java  |  44 +
 .../nifi/processors/standard/http/HttpMethod.java  |  46 ++
 .../nifi/processors/standard/InvokeHTTPTest.java   | 380 ++++-----
 5 files changed, 713 insertions(+), 692 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
index c086cc2f63..4659fe07a0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
@@ -608,7 +608,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
 
                     for (final PropertyDependency dependency : dependencies) {
                         final Set<String> dependentValues = dependency.getDependentValues();
-                        final String prefix = (capitalizeThe ? "The" : "the") + " <" + dependency.getPropertyDisplayName() + "> Property ";
+                        final String prefix = (capitalizeThe ? "The" : "the") + " [" + dependency.getPropertyDisplayName() + "] Property ";
                         String suffix = "";
                         if (dependentValues == null) {
                             suffix = "has a value specified.";
@@ -638,10 +638,15 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
                                 final StringBuilder sb = new StringBuilder("is set to one of the following values: ");
 
                                 for (final String dependentValue : dependentValues) {
-                                    for (AllowableValue av : dependencyProperty.getAllowableValues()) {
-                                        if (dependentValue.equals(av.getValue())) {
-                                            sb.append("\"").append(av.getDisplayName()).append("\", ");
-                                            break;
+                                    final List<AllowableValue> allowableValues = dependencyProperty.getAllowableValues();
+                                    if (allowableValues == null) {
+                                        sb.append("[").append(dependentValue).append("], ");
+                                    } else {
+                                        for (AllowableValue av : allowableValues) {
+                                            if (dependentValue.equals(av.getValue())) {
+                                                sb.append("[").append(av.getDisplayName()).append("], ");
+                                                break;
+                                            }
                                         }
                                     }
                                 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
index 49fffdc48e..4f25e44aec 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java
@@ -44,6 +44,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
@@ -93,6 +94,7 @@ import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.expression.AttributeExpression;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.flowfile.FlowFile;
@@ -109,6 +111,8 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.standard.http.ContentEncodingStrategy;
 import org.apache.nifi.processors.standard.http.FlowFileNamingStrategy;
 import org.apache.nifi.processors.standard.http.CookieStrategy;
+import org.apache.nifi.processors.standard.http.HttpHeader;
+import org.apache.nifi.processors.standard.http.HttpMethod;
 import org.apache.nifi.processors.standard.util.ProxyAuthenticator;
 import org.apache.nifi.processors.standard.util.SoftLimitBoundedByteArrayOutputStream;
 import org.apache.nifi.proxy.ProxyConfiguration;
@@ -166,367 +170,355 @@ public class InvokeHTTP extends AbstractProcessor {
 
     public static final String DEFAULT_CONTENT_TYPE = "application/octet-stream";
 
-    public static final String FORM_BASE = "post:form";
-
-    // Set of flowfile attributes which we generally always ignore during
-    // processing, including when converting http headers, copying attributes, etc.
-    // This set includes our strings defined above as well as some standard flowfile
-    // attributes.
-    public static final Set<String> IGNORED_ATTRIBUTES = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
-            STATUS_CODE, STATUS_MESSAGE, RESPONSE_BODY, REQUEST_URL, RESPONSE_URL, TRANSACTION_ID, REMOTE_DN,
-            EXCEPTION_CLASS, EXCEPTION_MESSAGE,
-            "uuid", "filename", "path")));
-
-    public static final String HTTP = "http";
-    public static final String HTTPS = "https";
-
-    public static final String GET_METHOD = "GET";
-    public static final String POST_METHOD = "POST";
-    public static final String PUT_METHOD = "PUT";
-    public static final String PATCH_METHOD = "PATCH";
-    public static final String DELETE_METHOD = "DELETE";
-    public static final String HEAD_METHOD = "HEAD";
-    public static final String OPTIONS_METHOD = "OPTIONS";
-
-    private static final Pattern DYNAMIC_FORM_PARAMETER_NAME = Pattern.compile("post:form:(?<formDataName>.*)$");
+    protected static final String FORM_DATA_NAME_BASE = "post:form";
+    private static final Pattern FORM_DATA_NAME_PARAMETER_PATTERN = Pattern.compile("post:form:(?<formDataName>.*)$");
     private static final String FORM_DATA_NAME_GROUP = "formDataName";
-    private static final String CONTENT_ENCODING_HEADER = "Content-Encoding";
 
-    // properties
-    public static final PropertyDescriptor PROP_METHOD = new PropertyDescriptor.Builder()
+    private static final Set<String> IGNORED_REQUEST_ATTRIBUTES = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            STATUS_CODE,
+            STATUS_MESSAGE,
+            RESPONSE_BODY,
+            REQUEST_URL,
+            RESPONSE_URL,
+            TRANSACTION_ID,
+            REMOTE_DN,
+            EXCEPTION_CLASS,
+            EXCEPTION_MESSAGE,
+            CoreAttributes.UUID.key(),
+            CoreAttributes.FILENAME.key(),
+            CoreAttributes.PATH.key()
+    )));
+
+    public static final PropertyDescriptor HTTP_METHOD = new PropertyDescriptor.Builder()
             .name("HTTP Method")
             .description("HTTP request method (GET, POST, PUT, PATCH, DELETE, HEAD, OPTIONS). Arbitrary methods are also supported. "
                     + "Methods other than POST, PUT and PATCH will be sent without a message body.")
             .required(true)
-            .defaultValue(GET_METHOD)
+            .defaultValue(HttpMethod.GET.name())
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
             .build();
 
-    public static final PropertyDescriptor PROP_URL = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor HTTP_URL = new PropertyDescriptor.Builder()
             .name("Remote URL")
-            .description("Remote URL which will be connected to, including scheme, host, port, path.")
+            .displayName("HTTP URL")
+            .description("HTTP remote URL including a scheme of http or https, as well as a hostname or IP address with optional port and path elements.")
             .required(true)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.URL_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor PROP_CONNECT_TIMEOUT = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor HTTP2_DISABLED = new PropertyDescriptor.Builder()
+            .name("disable-http2")
+            .displayName("HTTP/2 Disabled")
+            .description("Disable negotiation of HTTP/2 protocol. HTTP/2 requires TLS. HTTP/1.1 protocol supported is required when HTTP/2 is disabled.")
+            .required(true)
+            .defaultValue("False")
+            .allowableValues("True", "False")
+            .build();
+
+    public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+            .name("SSL Context Service")
+            .description("SSL Context Service provides trusted certificates and client certificates for TLS communication.")
+            .required(false)
+            .identifiesControllerService(SSLContextService.class)
+            .build();
+
+    public static final PropertyDescriptor SOCKET_CONNECT_TIMEOUT = new PropertyDescriptor.Builder()
             .name("Connection Timeout")
-            .description("Max wait time for connection to remote service.")
+            .displayName("Socket Connect Timeout")
+            .description("Maximum time to wait for initial socket connection to the HTTP URL.")
             .required(true)
             .defaultValue("5 secs")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor PROP_READ_TIMEOUT = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor SOCKET_READ_TIMEOUT = new PropertyDescriptor.Builder()
             .name("Read Timeout")
-            .description("Max wait time for response from remote service.")
+            .displayName("Socket Read Timeout")
+            .description("Maximum time to wait for receiving responses from a socket connection to the HTTP URL.")
             .required(true)
             .defaultValue("15 secs")
             .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor PROP_IDLE_TIMEOUT = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor SOCKET_IDLE_TIMEOUT = new PropertyDescriptor.Builder()
             .name("idle-timeout")
-            .displayName("Idle Timeout")
-            .description("Max idle time before closing connection to the remote service.")
+            .displayName("Socket Idle Timeout")
+            .description("Maximum time to wait before closing idle connections to the HTTP URL.")
             .required(true)
             .defaultValue("5 mins")
             .addValidator(StandardValidators.createTimePeriodValidator(1, TimeUnit.MILLISECONDS, Integer.MAX_VALUE, TimeUnit.SECONDS))
             .build();
 
-    public static final PropertyDescriptor PROP_MAX_IDLE_CONNECTIONS = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor SOCKET_IDLE_CONNECTIONS = new PropertyDescriptor.Builder()
             .name("max-idle-connections")
-            .displayName("Max Idle Connections")
-            .description("Max number of idle connections to keep open.")
+            .displayName("Socket Idle Connections")
+            .description("Maximum number of idle connections to the HTTP URL.")
             .required(true)
             .defaultValue("5")
             .addValidator(StandardValidators.INTEGER_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor PROP_DATE_HEADER = new PropertyDescriptor.Builder()
-            .name("Include Date Header")
-            .description("Include an RFC-2616 Date header in the request.")
-            .required(true)
-            .defaultValue("True")
-            .allowableValues("True", "False")
-            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROP_FOLLOW_REDIRECTS = new PropertyDescriptor.Builder()
-            .name("Follow Redirects")
-            .description("Follow HTTP redirects issued by remote server.")
-            .required(true)
-            .defaultValue("True")
-            .allowableValues("True", "False")
-            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROP_ATTRIBUTES_TO_SEND = new PropertyDescriptor.Builder()
-            .name("Attributes to Send")
-            .description("Regular expression that defines which attributes to send as HTTP headers in the request. "
-                    + "If not defined, no attributes are sent as headers. Also any dynamic properties set will be sent as headers. "
-                    + "The dynamic property key will be the header key and the dynamic property value will be interpreted as expression "
-                    + "language will be the header value.")
-            .required(false)
-            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
-            .build();
-
-    public static final PropertyDescriptor PROP_USERAGENT = new PropertyDescriptor.Builder()
-            .name("Useragent")
-            .displayName("Useragent")
-            .description("The Useragent identifier sent along with each request")
+    @Deprecated
+    public static final PropertyDescriptor PROXY_HOST = new PropertyDescriptor.Builder()
+            .name("Proxy Host")
+            .description("Proxy Host and dependent properties are deprecated in favor of Proxy Configuration Service. Proxy Host can be configured using an IP address or DNS address.")
             .required(false)
-            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
-    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
-            .name("SSL Context Service")
-            .description("The SSL Context Service used to provide client certificate information for TLS/SSL (https) connections."
-                    + " It is also used to connect to HTTPS Proxy.")
+    @Deprecated
+    public static final PropertyDescriptor PROXY_PORT = new PropertyDescriptor.Builder()
+            .name("Proxy Port")
+            .description("Proxy Port and dependent properties are deprecated in favor of Proxy Configuration Service. Port number for the configured Proxy Host address.")
             .required(false)
-            .identifiesControllerService(SSLContextService.class)
+            .addValidator(StandardValidators.PORT_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .dependsOn(PROXY_HOST)
             .build();
 
-    public static final PropertyDescriptor PROP_PROXY_TYPE = new PropertyDescriptor.Builder()
+    @Deprecated
+    public static final PropertyDescriptor PROXY_TYPE = new PropertyDescriptor.Builder()
             .name("Proxy Type")
             .displayName("Proxy Type")
-            .description("The type of the proxy we are connecting to. Must be either " + HTTP + " or " + HTTPS)
-            .defaultValue(HTTP)
+            .description("Proxy Type and dependent properties are deprecated in favor of Proxy Configuration Service. Proxy protocol type is not used")
+            .defaultValue("http")
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .dependsOn(PROXY_HOST)
             .build();
 
-    public static final PropertyDescriptor PROP_PROXY_HOST = new PropertyDescriptor.Builder()
-            .name("Proxy Host")
-            .description("The fully qualified hostname or IP address of the proxy server")
+    @Deprecated
+    public static final PropertyDescriptor PROXY_USERNAME = new PropertyDescriptor.Builder()
+            .name("invokehttp-proxy-user")
+            .displayName("Proxy Username")
+            .description("Proxy Username and dependent properties are deprecated in favor of Proxy Configuration Service. Username to set when authenticating with a Proxy server.")
             .required(false)
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .dependsOn(PROXY_HOST)
             .build();
 
-    public static final PropertyDescriptor PROP_PROXY_PORT = new PropertyDescriptor.Builder()
-            .name("Proxy Port")
-            .description("The port of the proxy server")
+    @Deprecated
+    public static final PropertyDescriptor PROXY_PASSWORD = new PropertyDescriptor.Builder()
+            .name("invokehttp-proxy-password")
+            .displayName("Proxy Password")
+            .description("Proxy Password and dependent properties are deprecated in favor of Proxy Configuration Service. Password to set when authenticating with a Proxy server.")
             .required(false)
-            .addValidator(StandardValidators.PORT_VALIDATOR)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .dependsOn(PROXY_HOST)
             .build();
 
-    public static final PropertyDescriptor PROP_PROXY_USER = new PropertyDescriptor.Builder()
-            .name("invokehttp-proxy-user")
-            .displayName("Proxy Username")
-            .description("Username to set when authenticating against proxy")
+    public static final PropertyDescriptor REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER = new PropertyDescriptor.Builder()
+            .name("oauth2-access-token-provider")
+            .displayName("Request OAuth2 Access Token Provider")
+            .description("Enables managed retrieval of OAuth2 Bearer Token applied to HTTP requests using the Authorization Header.")
+            .identifiesControllerService(OAuth2AccessTokenProvider.class)
             .required(false)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
             .build();
 
-    public static final PropertyDescriptor PROP_PROXY_PASSWORD = new PropertyDescriptor.Builder()
-            .name("invokehttp-proxy-password")
-            .displayName("Proxy Password")
-            .description("Password to set when authenticating against proxy")
+    public static final PropertyDescriptor REQUEST_USERNAME = new PropertyDescriptor.Builder()
+            .name("Basic Authentication Username")
+            .displayName("Request Username")
+            .description("The username provided for authentication of HTTP requests. Encoded using Base64 for HTTP Basic Authentication as described in RFC 7617.")
+            .required(false)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
+            .build();
+
+    public static final PropertyDescriptor REQUEST_PASSWORD = new PropertyDescriptor.Builder()
+            .name("Basic Authentication Password")
+            .displayName("Request Password")
+            .description("The password provided for authentication of HTTP requests. Encoded using Base64 for HTTP Basic Authentication as described in RFC 7617.")
             .required(false)
             .sensitive(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
             .build();
 
-    public static final PropertyDescriptor PROP_CONTENT_ENCODING = new PropertyDescriptor.Builder()
-            .name("Content-Encoding")
-            .displayName("Content-Encoding")
-            .description("HTTP Content-Encoding applied to request body during transmission. The receiving server must support the selected encoding to avoid request failures.")
-            .required(true)
-            .defaultValue(ContentEncodingStrategy.DISABLED.getValue())
-            .allowableValues(ContentEncodingStrategy.class)
+    public static final PropertyDescriptor REQUEST_DIGEST_AUTHENTICATION_ENABLED = new PropertyDescriptor.Builder()
+            .name("Digest Authentication")
+            .displayName("Request Digest Authentication Enabled")
+            .description("Enable Digest Authentication on HTTP requests with Username and Password credentials as described in RFC 7616.")
+            .required(false)
+            .defaultValue("false")
+            .allowableValues("true", "false")
+            .dependsOn(REQUEST_USERNAME)
             .build();
 
-    public static final PropertyDescriptor PROP_CONTENT_TYPE = new PropertyDescriptor.Builder()
-            .name("Content-Type")
-            .description("The Content-Type to specify for when content is being transmitted through a PUT, POST or PATCH. "
-                    + "In the case of an empty value after evaluating an expression language expression, Content-Type defaults to " + DEFAULT_CONTENT_TYPE)
-            .required(true)
-            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
-            .defaultValue("${" + CoreAttributes.MIME_TYPE.key() + "}")
-            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
+    public static final PropertyDescriptor REQUEST_FAILURE_PENALIZATION_ENABLED = new PropertyDescriptor.Builder()
+            .name("Penalize on \"No Retry\"")
+            .displayName("Request Failure Penalization Enabled")
+            .description("Enable penalization of request FlowFiles when receiving HTTP response with a status code between 400 and 499.")
+            .required(false)
+            .defaultValue(Boolean.FALSE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
             .build();
 
-    public static final PropertyDescriptor PROP_SEND_BODY = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor REQUEST_BODY_ENABLED = new PropertyDescriptor.Builder()
             .name("send-message-body")
-            .displayName("Send Message Body")
-            .description("If true, sends the HTTP message body on POST/PUT/PATCH requests (default).  If false, suppresses the message body and content-type header for these requests.")
-            .defaultValue("true")
-            .allowableValues("true", "false")
+            .displayName("Request Body Enabled")
+            .description("Enable sending HTTP request body for PATCH, POST, or PUT methods.")
+            .defaultValue(Boolean.TRUE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
             .required(false)
+            .dependsOn(HTTP_METHOD, HttpMethod.PATCH.name(), HttpMethod.POST.name(), HttpMethod.PUT.name())
             .build();
 
-    public static final PropertyDescriptor PROP_FORM_BODY_FORM_NAME = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor REQUEST_FORM_DATA_NAME = new PropertyDescriptor.Builder()
             .name("form-body-form-name")
-            .displayName("FlowFile Form Data Name")
-            .description("When Send Message Body is true, and FlowFile Form Data Name is set, "
-                    + " the FlowFile will be sent as the message body in multipart/form format with this value "
-                    + "as the form data name.")
+            .displayName("Request Multipart Form-Data Name")
+            .description("Enable sending HTTP request body formatted using multipart/form-data and using the form name configured.")
             .required(false)
             .addValidator(
-                    StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true))
+                    StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING, true)
+            )
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .dependsOn(REQUEST_BODY_ENABLED, Boolean.TRUE.toString())
             .build();
 
-    public static final PropertyDescriptor PROP_SET_FORM_FILE_NAME = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor REQUEST_FORM_DATA_FILENAME_ENABLED = new PropertyDescriptor.Builder()
             .name("set-form-filename")
-            .displayName("Set FlowFile Form Data File Name")
-            .description(
-                    "When Send Message Body is true, FlowFile Form Data Name is set, "
-                            + "and Set FlowFile Form Data File Name is true, the FlowFile's fileName value "
-                            + "will be set as the filename property of the form data.")
+            .displayName("Request Multipart Form-Data Filename Enabled")
+            .description("Enable sending the FlowFile filename attribute as the filename parameter in the Content-Disposition Header for multipart/form-data HTTP requests.")
             .required(false)
-            .defaultValue("true")
-            .allowableValues("true", "false")
+            .defaultValue(Boolean.TRUE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
+            .dependsOn(REQUEST_FORM_DATA_NAME)
             .build();
 
-    // Per RFC 7235, 2617, and 2616.
-    // basic-credentials = base64-user-pass
-    // base64-user-pass = userid ":" password
-    // userid = *<TEXT excluding ":">
-    // password = *TEXT
-    //
-    // OCTET = <any 8-bit sequence of data>
-    // CTL = <any US-ASCII control character (octets 0 - 31) and DEL (127)>
-    // LWS = [CRLF] 1*( SP | HT )
-    // TEXT = <any OCTET except CTLs but including LWS>
-    //
-    // Per RFC 7230, username & password in URL are now disallowed in HTTP and HTTPS URIs.
-    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
-            .name("Basic Authentication 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)
-            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
+    public static final PropertyDescriptor REQUEST_CHUNKED_TRANSFER_ENCODING_ENABLED = new PropertyDescriptor.Builder()
+            .name("Use Chunked Encoding")
+            .displayName("Request Chunked Transfer-Encoding Enabled")
+            .description("Enable sending HTTP requests with the Transfer-Encoding Header set to chunked, and disable sending the Content-Length Header. " +
+                    "Transfer-Encoding applies to the body in HTTP/1.1 requests as described in RFC 7230 Section 3.3.1")
+            .required(true)
+            .defaultValue(Boolean.FALSE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
+            .dependsOn(HTTP_METHOD, HttpMethod.PATCH.name(), HttpMethod.POST.name(), HttpMethod.PUT.name())
             .build();
 
-    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
-            .name("Basic Authentication Password")
-            .displayName("Basic Authentication Password")
-            .description("The password to be used by the client to authenticate against the Remote URL.")
-            .required(false)
-            .sensitive(true)
-            .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
+    public static final PropertyDescriptor REQUEST_CONTENT_ENCODING = new PropertyDescriptor.Builder()
+            .name("Content-Encoding")
+            .displayName("Request Content-Encoding")
+            .description("HTTP Content-Encoding applied to request body during transmission. The receiving server must support the selected encoding to avoid request failures.")
+            .required(true)
+            .defaultValue(ContentEncodingStrategy.DISABLED.getValue())
+            .allowableValues(ContentEncodingStrategy.class)
+            .dependsOn(HTTP_METHOD, HttpMethod.PATCH.name(), HttpMethod.POST.name(), HttpMethod.PUT.name())
             .build();
 
-    public static final PropertyDescriptor PROP_PUT_OUTPUT_IN_ATTRIBUTE = new PropertyDescriptor.Builder()
-            .name("Put Response Body In Attribute")
-            .description("If set, the response body received back will be put into an attribute of the original FlowFile instead of a separate "
-                    + "FlowFile. The attribute key to put to is determined by evaluating value of this property. ")
-            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
+    public static final PropertyDescriptor REQUEST_CONTENT_TYPE = new PropertyDescriptor.Builder()
+            .name("Content-Type")
+            .displayName("Request Content-Type")
+            .description("HTTP Content-Type Header applied to when sending an HTTP request body for PATCH, POST, or PUT methods. " +
+                    String.format("The Content-Type defaults to %s when not configured.", DEFAULT_CONTENT_TYPE))
+            .required(true)
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue("${" + CoreAttributes.MIME_TYPE.key() + "}")
+            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
+            .dependsOn(HTTP_METHOD, HttpMethod.PATCH.name(), HttpMethod.POST.name(), HttpMethod.PUT.name())
             .build();
 
-    public static final PropertyDescriptor PROP_PUT_ATTRIBUTE_MAX_LENGTH = new PropertyDescriptor.Builder()
-            .name("Max Length To Put In Attribute")
-            .description("If routing the response body to an attribute of the original (by setting the \"Put response body in attribute\" "
-                    + "property or by receiving an error status code), the number of characters put to the attribute value will be at "
-                    + "most this amount. This is important because attributes are held in memory and large attributes will quickly "
-                    + "cause out of memory issues. If the output goes longer than this value, it will be truncated to fit. "
-                    + "Consider making this smaller if able.")
-            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
-            .defaultValue("256")
+    public static final PropertyDescriptor REQUEST_DATE_HEADER_ENABLED = new PropertyDescriptor.Builder()
+            .name("Include Date Header")
+            .displayName("Request Date Header Enabled")
+            .description("Enable sending HTTP Date Header on HTTP requests as described in RFC 7231 Section 7.1.1.2.")
+            .required(true)
+            .defaultValue("True")
+            .allowableValues("True", "False")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
-            .name("Digest Authentication")
-            .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.")
+    public static final PropertyDescriptor REQUEST_HEADER_ATTRIBUTES_PATTERN = new PropertyDescriptor.Builder()
+            .name("Attributes to Send")
+            .displayName("Request Header Attributes Pattern")
+            .description("Regular expression that defines which attributes to send as HTTP headers in the request. "
+                    + "If not defined, no attributes are sent as headers. Dynamic properties will be sent as headers. "
+                    + "The dynamic property name will be the header key and the dynamic property value will be interpreted as expression "
+                    + "language will be the header value.")
             .required(false)
-            .defaultValue("false")
-            .allowableValues("true", "false")
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor PROP_OUTPUT_RESPONSE_REGARDLESS = new PropertyDescriptor.Builder()
-            .name("Always Output Response")
-            .description("Will force a response FlowFile to be generated and routed to the 'Response' relationship regardless of what the server status code received is "
-                    + "or if the processor is configured to put the server response body in the request attribute. In the later configuration a request FlowFile with the "
-                    + "response body in the attribute and a typical response FlowFile will be emitted to their respective relationships.")
+    public static final PropertyDescriptor REQUEST_USER_AGENT = new PropertyDescriptor.Builder()
+            .name("Useragent")
+            .displayName("Request User-Agent")
+            .description("HTTP User-Agent Header applied to requests. RFC 7231 Section 5.5.3 describes recommend formatting.")
             .required(false)
-            .defaultValue("false")
-            .allowableValues("true", "false")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
-    public static final PropertyDescriptor PROP_ADD_HEADERS_TO_REQUEST = new PropertyDescriptor.Builder()
-            .name("Add Response Headers to Request")
-            .description("Enabling this property saves all the response headers to the original request. This may be when the response headers are needed "
-                    + "but a response is not generated due to the status code received.")
-            .required(false)
-            .defaultValue("false")
-            .allowableValues("true", "false")
+    public static final PropertyDescriptor RESPONSE_BODY_ATTRIBUTE_NAME = new PropertyDescriptor.Builder()
+            .name("Put Response Body In Attribute")
+            .displayName("Response Body Attribute Name")
+            .description("FlowFile attribute name used to write an HTTP response body for FlowFiles transferred to the Original relationship.")
+            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(AttributeExpression.ResultType.STRING))
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
-    public static final PropertyDescriptor PROP_USE_CHUNKED_ENCODING = new PropertyDescriptor.Builder()
-            .name("Use Chunked Encoding")
-            .description("When POST'ing, PUT'ing or PATCH'ing content set this property to true in order to not pass the 'Content-length' header and instead send 'Transfer-Encoding' with "
-                    + "a value of 'chunked'. This will enable the data transfer mechanism which was introduced in HTTP 1.1 to pass data of unknown lengths in chunks.")
-            .required(true)
-            .defaultValue("false")
-            .allowableValues("true", "false")
+    public static final PropertyDescriptor RESPONSE_BODY_ATTRIBUTE_SIZE = new PropertyDescriptor.Builder()
+            .name("Max Length To Put In Attribute")
+            .displayName("Response Body Attribute Size")
+            .description("Maximum size in bytes applied when writing an HTTP response body to a FlowFile attribute. Attributes exceeding the maximum will be truncated.")
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            .defaultValue("256")
+            .dependsOn(RESPONSE_BODY_ATTRIBUTE_NAME)
             .build();
 
-    public static final PropertyDescriptor PROP_PENALIZE_NO_RETRY = new PropertyDescriptor.Builder()
-            .name("Penalize on \"No Retry\"")
-            .description("Enabling this property will penalize FlowFiles that are routed to the \"No Retry\" relationship.")
-            .required(false)
-            .defaultValue("false")
-            .allowableValues("true", "false")
+    public static final PropertyDescriptor RESPONSE_BODY_IGNORED = new PropertyDescriptor.Builder()
+            .name("ignore-response-content")
+            .displayName("Response Body Ignored")
+            .description("Disable writing HTTP response FlowFiles to Response relationship")
+            .required(true)
+            .defaultValue(Boolean.FALSE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
             .build();
 
-    public static final PropertyDescriptor PROP_USE_ETAG = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor RESPONSE_CACHE_ENABLED = new PropertyDescriptor.Builder()
             .name("use-etag")
-            .description("Enable HTTP entity tag (ETag) support for HTTP requests.")
-            .displayName("Use HTTP ETag")
+            .displayName("Response Cache Enabled")
+            .description("Enable HTTP response caching described in RFC 7234. Caching responses considers ETag and other headers.")
             .required(true)
-            .defaultValue("false")
-            .allowableValues("true", "false")
+            .defaultValue(Boolean.FALSE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
             .build();
 
-    public static final PropertyDescriptor PROP_ETAG_MAX_CACHE_SIZE = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor RESPONSE_CACHE_SIZE = new PropertyDescriptor.Builder()
             .name("etag-max-cache-size")
-            .description("The maximum size that the ETag cache should be allowed to grow to. The default size is 10MB.")
-            .displayName("Maximum ETag Cache Size")
+            .displayName("Response Cache Size")
+            .description("Maximum size of HTTP response cache in bytes. Caching responses considers ETag and other headers.")
             .required(true)
             .defaultValue("10MB")
             .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .dependsOn(RESPONSE_CACHE_ENABLED, Boolean.TRUE.toString())
             .build();
 
-    public static final PropertyDescriptor IGNORE_RESPONSE_CONTENT = new PropertyDescriptor.Builder()
-            .name("ignore-response-content")
-            .description("If true, the processor will not write the response's content into the flow file.")
-            .displayName("Ignore response's content")
-            .required(true)
-            .defaultValue("false")
-            .allowableValues("true", "false")
-            .build();
-
-    public static final PropertyDescriptor DISABLE_HTTP2_PROTOCOL = new PropertyDescriptor.Builder()
-            .name("disable-http2")
-            .description("Determines whether or not to disable use of the HTTP/2 protocol version. If disabled, only HTTP/1.1 is supported.")
-            .displayName("Disable HTTP/2")
+    public static final PropertyDescriptor RESPONSE_COOKIE_STRATEGY = new PropertyDescriptor.Builder()
+            .name("cookie-strategy")
+            .description("Strategy for accepting and persisting HTTP cookies. Accepting cookies enables persistence across multiple requests.")
+            .displayName("Response Cookie Strategy")
             .required(true)
-            .defaultValue("False")
-            .allowableValues("True", "False")
+            .defaultValue(CookieStrategy.DISABLED.name())
+            .allowableValues(CookieStrategy.values())
             .build();
 
-    public static final PropertyDescriptor OAUTH2_ACCESS_TOKEN_PROVIDER = new PropertyDescriptor.Builder()
-            .name("oauth2-access-token-provider")
-            .displayName("OAuth2 Access Token provider")
-            .identifiesControllerService(OAuth2AccessTokenProvider.class)
+    public static final PropertyDescriptor RESPONSE_GENERATION_REQUIRED = new PropertyDescriptor.Builder()
+            .name("Always Output Response")
+            .displayName("Response Generation Required")
+            .description("Enable generation and transfer of a FlowFile to the Response relationship regardless of HTTP response received.")
             .required(false)
+            .defaultValue(Boolean.FALSE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
             .build();
 
-    public static final PropertyDescriptor FLOW_FILE_NAMING_STRATEGY = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor RESPONSE_FLOW_FILE_NAMING_STRATEGY = new PropertyDescriptor.Builder()
             .name("flow-file-naming-strategy")
-            .description("Determines the strategy used for setting the filename attribute of the FlowFile.")
-            .displayName("FlowFile Naming Strategy")
+            .description("Determines the strategy used for setting the filename attribute of FlowFiles transferred to the Response relationship.")
+            .displayName("Response FlowFile Naming Strategy")
             .required(true)
             .defaultValue(FlowFileNamingStrategy.RANDOM.name())
             .allowableValues(
@@ -536,105 +528,113 @@ public class InvokeHTTP extends AbstractProcessor {
             )
             .build();
 
-    public static final PropertyDescriptor PROP_COOKIE_STRATEGY = new PropertyDescriptor.Builder()
-            .name("cookie-strategy")
-            .description("Strategy for accepting and persisting HTTP cookies. Accepting cookies enables persistence across multiple requests.")
-            .displayName("Cookie Strategy")
+    public static final PropertyDescriptor RESPONSE_HEADER_REQUEST_ATTRIBUTES_ENABLED = new PropertyDescriptor.Builder()
+            .name("Add Response Headers to Request")
+            .displayName("Response Header Request Attributes Enabled")
+            .description("Enable adding HTTP response headers as attributes to FlowFiles transferred to the Original relationship.")
+            .required(false)
+            .defaultValue(Boolean.FALSE.toString())
+            .allowableValues(Boolean.TRUE.toString(), Boolean.FALSE.toString())
+            .build();
+
+    public static final PropertyDescriptor RESPONSE_REDIRECTS_ENABLED = new PropertyDescriptor.Builder()
+            .name("Follow Redirects")
+            .displayName("Response Redirects Enabled")
+            .description("Enable following HTTP redirects sent with HTTP 300 series responses as described in RFC 7231 Section 6.4.")
             .required(true)
-            .defaultValue(CookieStrategy.DISABLED.name())
-            .allowableValues(CookieStrategy.values())
+            .defaultValue("True")
+            .allowableValues("True", "False")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
             .build();
 
     private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
-    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
-            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
+
+    private static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
 
     public static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
-            PROP_METHOD,
-            PROP_URL,
-            PROP_SSL_CONTEXT_SERVICE,
-            PROP_CONNECT_TIMEOUT,
-            PROP_READ_TIMEOUT,
-            PROP_IDLE_TIMEOUT,
-            PROP_MAX_IDLE_CONNECTIONS,
-            PROP_DATE_HEADER,
-            PROP_FOLLOW_REDIRECTS,
-            PROP_COOKIE_STRATEGY,
-            DISABLE_HTTP2_PROTOCOL,
-            FLOW_FILE_NAMING_STRATEGY,
-            PROP_ATTRIBUTES_TO_SEND,
-            PROP_USERAGENT,
-            PROP_BASIC_AUTH_USERNAME,
-            PROP_BASIC_AUTH_PASSWORD,
-            OAUTH2_ACCESS_TOKEN_PROVIDER,
+            HTTP_METHOD,
+            HTTP_URL,
+            HTTP2_DISABLED,
+            SSL_CONTEXT_SERVICE,
+            SOCKET_CONNECT_TIMEOUT,
+            SOCKET_READ_TIMEOUT,
+            SOCKET_IDLE_TIMEOUT,
+            SOCKET_IDLE_CONNECTIONS,
             PROXY_CONFIGURATION_SERVICE,
-            PROP_PROXY_HOST,
-            PROP_PROXY_PORT,
-            PROP_PROXY_TYPE,
-            PROP_PROXY_USER,
-            PROP_PROXY_PASSWORD,
-            PROP_PUT_OUTPUT_IN_ATTRIBUTE,
-            PROP_PUT_ATTRIBUTE_MAX_LENGTH,
-            PROP_DIGEST_AUTH,
-            PROP_OUTPUT_RESPONSE_REGARDLESS,
-            PROP_ADD_HEADERS_TO_REQUEST,
-            PROP_CONTENT_ENCODING,
-            PROP_CONTENT_TYPE,
-            PROP_SEND_BODY,
-            PROP_USE_CHUNKED_ENCODING,
-            PROP_PENALIZE_NO_RETRY,
-            PROP_USE_ETAG,
-            PROP_ETAG_MAX_CACHE_SIZE,
-            IGNORE_RESPONSE_CONTENT,
-            PROP_FORM_BODY_FORM_NAME,
-            PROP_SET_FORM_FILE_NAME));
-
-    // relationships
-    public static final Relationship REL_SUCCESS_REQ = new Relationship.Builder()
+            PROXY_HOST,
+            PROXY_PORT,
+            PROXY_TYPE,
+            PROXY_USERNAME,
+            PROXY_PASSWORD,
+            REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER,
+            REQUEST_USERNAME,
+            REQUEST_PASSWORD,
+            REQUEST_DIGEST_AUTHENTICATION_ENABLED,
+            REQUEST_FAILURE_PENALIZATION_ENABLED,
+            REQUEST_BODY_ENABLED,
+            REQUEST_FORM_DATA_NAME,
+            REQUEST_FORM_DATA_FILENAME_ENABLED,
+            REQUEST_CHUNKED_TRANSFER_ENCODING_ENABLED,
+            REQUEST_CONTENT_ENCODING,
+            REQUEST_CONTENT_TYPE,
+            REQUEST_DATE_HEADER_ENABLED,
+            REQUEST_HEADER_ATTRIBUTES_PATTERN,
+            REQUEST_USER_AGENT,
+            RESPONSE_BODY_ATTRIBUTE_NAME,
+            RESPONSE_BODY_ATTRIBUTE_SIZE,
+            RESPONSE_BODY_IGNORED,
+            RESPONSE_CACHE_ENABLED,
+            RESPONSE_CACHE_SIZE,
+            RESPONSE_COOKIE_STRATEGY,
+            RESPONSE_GENERATION_REQUIRED,
+            RESPONSE_FLOW_FILE_NAMING_STRATEGY,
+            RESPONSE_HEADER_REQUEST_ATTRIBUTES_ENABLED,
+            RESPONSE_REDIRECTS_ENABLED
+    ));
+
+    public static final Relationship ORIGINAL = new Relationship.Builder()
             .name("Original")
-            .description("The original FlowFile will be routed upon success (2xx status codes). It will have new attributes detailing the "
-                    + "success of the request.")
+            .description("Request FlowFiles transferred when receiving HTTP responses with a status code between 200 and 299.")
             .build();
 
-    public static final Relationship REL_RESPONSE = new Relationship.Builder()
+    public static final Relationship RESPONSE = new Relationship.Builder()
             .name("Response")
-            .description("A Response FlowFile will be routed upon success (2xx status codes). If the 'Output Response Regardless' property "
-                    + "is true then the response will be sent to this relationship regardless of the status code received.")
+            .description("Response FlowFiles transferred when receiving HTTP responses with a status code between 200 and 299.")
             .build();
 
-    public static final Relationship REL_RETRY = new Relationship.Builder()
+    public static final Relationship RETRY = new Relationship.Builder()
             .name("Retry")
-            .description("The original FlowFile will be routed on any status code that can be retried (5xx status codes). It will have new "
-                    + "attributes detailing the request.")
+            .description("Request FlowFiles transferred when receiving HTTP responses with a status code between 500 and 599.")
             .build();
 
-    public static final Relationship REL_NO_RETRY = new Relationship.Builder()
+    public static final Relationship NO_RETRY = new Relationship.Builder()
             .name("No Retry")
-            .description("The original FlowFile will be routed on any status code that should NOT be retried (1xx, 3xx, 4xx status codes).  "
-                    + "It will have new attributes detailing the request.")
+            .description("Request FlowFiles transferred when receiving HTTP responses with a status code between 400 an 499.")
             .build();
 
-    public static final Relationship REL_FAILURE = new Relationship.Builder()
+    public static final Relationship FAILURE = new Relationship.Builder()
             .name("Failure")
-            .description("The original FlowFile will be routed on any type of connection failure, timeout or general exception. "
-                    + "It will have new attributes detailing the request.")
+            .description("Request FlowFiles transferred when receiving socket communication errors.")
             .build();
 
     public static final Set<Relationship> RELATIONSHIPS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
-            REL_SUCCESS_REQ, REL_RESPONSE, REL_RETRY, REL_NO_RETRY, REL_FAILURE)));
-
-    // RFC 2616 Date Time Formatter with hard-coded GMT Zone
-    // https://tools.ietf.org/html/rfc2616#section-3.3 - date format header should not be localized
+            ORIGINAL,
+            RESPONSE,
+            RETRY,
+            NO_RETRY,
+            FAILURE
+    )));
+
+    // RFC 2616 Date Time Formatter with hard-coded GMT Zone and US Locale. RFC 2616 Section 3.3 indicates the header should not be localized
     private static final DateTimeFormatter RFC_2616_DATE_TIME = DateTimeFormatter.ofPattern("EEE, dd MMM yyyy HH:mm:ss 'GMT'", Locale.US);
 
-    // Multiple Header Delimiter
     private static final String MULTIPLE_HEADER_DELIMITER = ", ";
 
     private volatile Set<String> dynamicPropertyNames = new HashSet<>();
 
-    private volatile Pattern regexAttributesToSend = null;
+    private volatile Pattern requestHeaderAttributesPattern = null;
 
-    private volatile boolean useChunked = false;
+    private volatile boolean chunkedTransferEncoding = false;
 
     private volatile Optional<OAuth2AccessTokenProvider> oauth2AccessTokenProviderOptional;
 
@@ -647,9 +647,9 @@ public class InvokeHTTP extends AbstractProcessor {
 
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
-        if (propertyDescriptorName.startsWith(FORM_BASE)) {
+        if (propertyDescriptorName.startsWith(FORM_DATA_NAME_BASE)) {
 
-            Matcher matcher = DYNAMIC_FORM_PARAMETER_NAME.matcher(propertyDescriptorName);
+            Matcher matcher = FORM_DATA_NAME_PARAMETER_PATTERN.matcher(propertyDescriptorName);
             if (matcher.matches()) {
                 return new PropertyDescriptor.Builder()
                         .required(false)
@@ -689,12 +689,12 @@ public class InvokeHTTP extends AbstractProcessor {
             this.dynamicPropertyNames = Collections.unmodifiableSet(newDynamicPropertyNames);
         } else {
             // compile the attributes-to-send filter pattern
-            if (PROP_ATTRIBUTES_TO_SEND.getName().equalsIgnoreCase(descriptor.getName())) {
+            if (REQUEST_HEADER_ATTRIBUTES_PATTERN.getName().equalsIgnoreCase(descriptor.getName())) {
                 if (newValue == null || newValue.isEmpty()) {
-                    regexAttributesToSend = null;
+                    requestHeaderAttributesPattern = null;
                 } else {
                     final String trimmedValue = StringUtils.trimToEmpty(newValue);
-                    regexAttributesToSend = Pattern.compile(trimmedValue);
+                    requestHeaderAttributesPattern = Pattern.compile(trimmedValue);
                 }
             }
         }
@@ -703,15 +703,15 @@ public class InvokeHTTP extends AbstractProcessor {
     @Override
     protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
         final List<ValidationResult> results = new ArrayList<>(3);
-        final boolean proxyHostSet = validationContext.getProperty(PROP_PROXY_HOST).isSet();
-        final boolean proxyPortSet = validationContext.getProperty(PROP_PROXY_PORT).isSet();
+        final boolean proxyHostSet = validationContext.getProperty(PROXY_HOST).isSet();
+        final boolean proxyPortSet = validationContext.getProperty(PROXY_PORT).isSet();
 
         if ((proxyHostSet && !proxyPortSet) || (!proxyHostSet && proxyPortSet)) {
             results.add(new ValidationResult.Builder().subject("Proxy Host and Port").valid(false).explanation("If Proxy Host or Proxy Port is set, both must be set").build());
         }
 
-        final boolean proxyUserSet = validationContext.getProperty(PROP_PROXY_USER).isSet();
-        final boolean proxyPwdSet = validationContext.getProperty(PROP_PROXY_PASSWORD).isSet();
+        final boolean proxyUserSet = validationContext.getProperty(PROXY_USERNAME).isSet();
+        final boolean proxyPwdSet = validationContext.getProperty(PROXY_PASSWORD).isSet();
 
         if ((proxyUserSet && !proxyPwdSet) || (!proxyUserSet && proxyPwdSet)) {
             results.add(new ValidationResult.Builder().subject("Proxy User and Password").valid(false).explanation("If Proxy Username or Proxy Password is set, both must be set").build());
@@ -720,25 +720,13 @@ public class InvokeHTTP extends AbstractProcessor {
             results.add(new ValidationResult.Builder().subject("Proxy").valid(false).explanation("If Proxy username is set, proxy host must be set").build());
         }
 
-        final String proxyType = validationContext.getProperty(PROP_PROXY_TYPE).evaluateAttributeExpressions().getValue();
-
-        if (!HTTP.equals(proxyType) && !HTTPS.equals(proxyType)) {
-            results.add(new ValidationResult.Builder().subject(PROP_PROXY_TYPE.getDisplayName()).valid(false)
-                    .explanation(PROP_PROXY_TYPE.getDisplayName() + " must be either " + HTTP + " or " + HTTPS).build());
-        }
-
-        if (HTTPS.equals(proxyType)
-                && !validationContext.getProperty(PROP_SSL_CONTEXT_SERVICE).isSet()) {
-            results.add(new ValidationResult.Builder().subject("SSL Context Service").valid(false).explanation("If Proxy Type is HTTPS, SSL Context Service must be set").build());
-        }
-
         ProxyConfiguration.validateProxySpec(validationContext, results, PROXY_SPECS);
 
         // Check for dynamic properties for form components.
         // Even if the flowfile is not sent, we may still send form parameters.
         boolean hasFormData = false;
         for (final PropertyDescriptor descriptor : validationContext.getProperties().keySet()) {
-            final Matcher matcher = DYNAMIC_FORM_PARAMETER_NAME.matcher(descriptor.getName());
+            final Matcher matcher = FORM_DATA_NAME_PARAMETER_PATTERN.matcher(descriptor.getName());
             if (matcher.matches()) {
                 hasFormData = true;
                 break;
@@ -746,35 +734,35 @@ public class InvokeHTTP extends AbstractProcessor {
         }
 
         // if form data exists, and send body is true, Flowfile Form Data Name must be set.
-        final boolean sendBody = validationContext.getProperty(PROP_SEND_BODY).asBoolean();
-        final boolean contentNameSet = validationContext.getProperty(PROP_FORM_BODY_FORM_NAME).isSet();
+        final boolean requestBodyEnabled = validationContext.getProperty(REQUEST_BODY_ENABLED).asBoolean();
+        final boolean contentNameSet = validationContext.getProperty(REQUEST_FORM_DATA_NAME).isSet();
         if (hasFormData) {
-            if (sendBody && !contentNameSet) {
+            if (requestBodyEnabled && !contentNameSet) {
                 final String explanation = String.format("[%s] is required when Form Data properties are configured and [%s] is enabled",
-                        PROP_FORM_BODY_FORM_NAME.getDisplayName(),
-                        PROP_SEND_BODY.getDisplayName());
+                        REQUEST_FORM_DATA_NAME.getDisplayName(),
+                        REQUEST_BODY_ENABLED.getDisplayName());
                 results.add(new ValidationResult.Builder()
-                        .subject(PROP_FORM_BODY_FORM_NAME.getDisplayName())
+                        .subject(REQUEST_FORM_DATA_NAME.getDisplayName())
                         .valid(false)
                         .explanation(explanation)
                         .build());
             }
         }
-        if (!sendBody && contentNameSet) {
+        if (!requestBodyEnabled && contentNameSet) {
             final String explanation = String.format("[%s] must be [true] when Form Data properties are configured and [%s] is configured",
-                    PROP_SEND_BODY.getDisplayName(),
-                    PROP_FORM_BODY_FORM_NAME.getDisplayName());
+                    REQUEST_BODY_ENABLED.getDisplayName(),
+                    REQUEST_FORM_DATA_NAME.getDisplayName());
             results.add(new ValidationResult.Builder()
-                    .subject(PROP_FORM_BODY_FORM_NAME.getDisplayName())
+                    .subject(REQUEST_FORM_DATA_NAME.getDisplayName())
                     .valid(false)
                     .explanation(explanation)
                     .build());
         }
 
-        boolean usingUserNamePasswordAuthorization = validationContext.getProperty(PROP_BASIC_AUTH_USERNAME).isSet()
-            || validationContext.getProperty(PROP_BASIC_AUTH_PASSWORD).isSet();
+        boolean usingUserNamePasswordAuthorization = validationContext.getProperty(REQUEST_USERNAME).isSet()
+            || validationContext.getProperty(REQUEST_PASSWORD).isSet();
 
-        boolean usingOAuth2Authorization = validationContext.getProperty(OAUTH2_ACCESS_TOKEN_PROVIDER).isSet();
+        boolean usingOAuth2Authorization = validationContext.getProperty(REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER).isSet();
 
         if (usingUserNamePasswordAuthorization && usingOAuth2Authorization) {
             results.add(new ValidationResult.Builder()
@@ -795,14 +783,14 @@ public class InvokeHTTP extends AbstractProcessor {
 
         final ProxyConfiguration proxyConfig = ProxyConfiguration.getConfiguration(context, () -> {
             final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
-            final String proxyHost = context.getProperty(PROP_PROXY_HOST).evaluateAttributeExpressions().getValue();
-            final Integer proxyPort = context.getProperty(PROP_PROXY_PORT).evaluateAttributeExpressions().asInteger();
+            final String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
+            final Integer proxyPort = context.getProperty(PROXY_PORT).evaluateAttributeExpressions().asInteger();
             if (proxyHost != null && proxyPort != null) {
                 componentProxyConfig.setProxyType(Type.HTTP);
                 componentProxyConfig.setProxyServerHost(proxyHost);
                 componentProxyConfig.setProxyServerPort(proxyPort);
-                final String proxyUsername = trimToEmpty(context.getProperty(PROP_PROXY_USER).evaluateAttributeExpressions().getValue());
-                final String proxyPassword = context.getProperty(PROP_PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
+                final String proxyUsername = trimToEmpty(context.getProperty(PROXY_USERNAME).evaluateAttributeExpressions().getValue());
+                final String proxyPassword = context.getProperty(PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
                 componentProxyConfig.setProxyUserName(proxyUsername);
                 componentProxyConfig.setProxyUserPassword(proxyPassword);
             }
@@ -818,45 +806,38 @@ public class InvokeHTTP extends AbstractProcessor {
             }
         }
 
-        // configure ETag cache if enabled
-        final boolean etagEnabled = context.getProperty(PROP_USE_ETAG).asBoolean();
-        if (etagEnabled) {
-            final int maxCacheSizeBytes = context.getProperty(PROP_ETAG_MAX_CACHE_SIZE).asDataSize(DataUnit.B).intValue();
-            okHttpClientBuilder.cache(new Cache(getETagCacheDir(), maxCacheSizeBytes));
+        // Configure caching
+        final boolean cachingEnabled = context.getProperty(RESPONSE_CACHE_ENABLED).asBoolean();
+        if (cachingEnabled) {
+            final int maxCacheSizeBytes = context.getProperty(RESPONSE_CACHE_SIZE).asDataSize(DataUnit.B).intValue();
+            okHttpClientBuilder.cache(new Cache(getResponseCacheDirectory(), maxCacheSizeBytes));
         }
 
-        // Configure whether HTTP/2 protocol should be disabled
-        if (context.getProperty(DISABLE_HTTP2_PROTOCOL).asBoolean()) {
+        if (context.getProperty(HTTP2_DISABLED).asBoolean()) {
             okHttpClientBuilder.protocols(Collections.singletonList(Protocol.HTTP_1_1));
         }
 
-        // Set timeouts
-        okHttpClientBuilder.connectTimeout((context.getProperty(PROP_CONNECT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue()), TimeUnit.MILLISECONDS);
-        okHttpClientBuilder.readTimeout(context.getProperty(PROP_READ_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue(), TimeUnit.MILLISECONDS);
-
-        // Set connectionpool limits
+        okHttpClientBuilder.followRedirects(context.getProperty(RESPONSE_REDIRECTS_ENABLED).asBoolean());
+        okHttpClientBuilder.connectTimeout((context.getProperty(SOCKET_CONNECT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue()), TimeUnit.MILLISECONDS);
+        okHttpClientBuilder.readTimeout(context.getProperty(SOCKET_READ_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue(), TimeUnit.MILLISECONDS);
         okHttpClientBuilder.connectionPool(
                 new ConnectionPool(
-                        context.getProperty(PROP_MAX_IDLE_CONNECTIONS).asInteger(),
-                        context.getProperty(PROP_IDLE_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue(), TimeUnit.MILLISECONDS
+                        context.getProperty(SOCKET_IDLE_CONNECTIONS).asInteger(),
+                        context.getProperty(SOCKET_IDLE_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue(), TimeUnit.MILLISECONDS
                 )
         );
 
-        // Set whether to follow redirects
-        okHttpClientBuilder.followRedirects(context.getProperty(PROP_FOLLOW_REDIRECTS).asBoolean());
-
-        // Apply the TLS configuration if present
-        final SSLContextService sslService = context.getProperty(PROP_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
         if (sslService != null) {
             final SSLContext sslContext = sslService.createContext();
             final SSLSocketFactory socketFactory = sslContext.getSocketFactory();
             final TlsConfiguration tlsConfiguration = sslService.createTlsConfiguration();
-            final X509TrustManager trustManager = SslContextFactory.getX509TrustManager(tlsConfiguration);
+            final X509TrustManager trustManager = Objects.requireNonNull(SslContextFactory.getX509TrustManager(tlsConfiguration), "Trust Manager not found");
             okHttpClientBuilder.sslSocketFactory(socketFactory, trustManager);
         }
 
-        // Configure cookie strategy
-        switch(CookieStrategy.valueOf(context.getProperty(PROP_COOKIE_STRATEGY).getValue())) {
+        final CookieStrategy cookieStrategy = CookieStrategy.valueOf(context.getProperty(RESPONSE_COOKIE_STRATEGY).getValue());
+        switch (cookieStrategy) {
             case DISABLED:
                 break;
             case ACCEPT_ALL:
@@ -868,15 +849,15 @@ public class InvokeHTTP extends AbstractProcessor {
 
         setAuthenticator(okHttpClientBuilder, context);
 
-        useChunked = context.getProperty(PROP_USE_CHUNKED_ENCODING).asBoolean();
+        chunkedTransferEncoding = context.getProperty(REQUEST_CHUNKED_TRANSFER_ENCODING_ENABLED).asBoolean();
 
         okHttpClientAtomicReference.set(okHttpClientBuilder.build());
     }
 
     @OnScheduled
     public void initOauth2AccessTokenProvider(final ProcessContext context) {
-        if (context.getProperty(OAUTH2_ACCESS_TOKEN_PROVIDER).isSet()) {
-            OAuth2AccessTokenProvider oauth2AccessTokenProvider = context.getProperty(OAUTH2_ACCESS_TOKEN_PROVIDER).asControllerService(OAuth2AccessTokenProvider.class);
+        if (context.getProperty(REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER).isSet()) {
+            OAuth2AccessTokenProvider oauth2AccessTokenProvider = context.getProperty(REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER).asControllerService(OAuth2AccessTokenProvider.class);
 
             oauth2AccessTokenProvider.getAccessDetails();
 
@@ -887,17 +868,12 @@ public class InvokeHTTP extends AbstractProcessor {
     }
 
     private void setAuthenticator(OkHttpClient.Builder okHttpClientBuilder, ProcessContext context) {
-        final String authUser = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_USERNAME).getValue());
+        final String authUser = trimToEmpty(context.getProperty(REQUEST_USERNAME).getValue());
 
         // If the username/password properties are set then check if digest auth is being used
-        if (!authUser.isEmpty() && "true".equalsIgnoreCase(context.getProperty(PROP_DIGEST_AUTH).getValue())) {
-            final String authPass = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_PASSWORD).getValue());
-
-            /*
-             * 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
-             */
+        if (!authUser.isEmpty() && "true".equalsIgnoreCase(context.getProperty(REQUEST_DIGEST_AUTHENTICATION_ENABLED).getValue())) {
+            final String authPass = trimToEmpty(context.getProperty(REQUEST_PASSWORD).getValue());
+
             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);
@@ -914,39 +890,34 @@ public class InvokeHTTP extends AbstractProcessor {
         FlowFile requestFlowFile = session.get();
 
         // Checking to see if the property to put the body of the response in an attribute was set
-        boolean putToAttribute = context.getProperty(PROP_PUT_OUTPUT_IN_ATTRIBUTE).isSet();
+        boolean putToAttribute = context.getProperty(RESPONSE_BODY_ATTRIBUTE_NAME).isSet();
         if (requestFlowFile == null) {
             if (context.hasNonLoopConnection()) {
                 return;
             }
 
-            String request = context.getProperty(PROP_METHOD).evaluateAttributeExpressions().getValue().toUpperCase();
-            if (POST_METHOD.equals(request) || PUT_METHOD.equals(request) || PATCH_METHOD.equals(request)) {
+            final String method = getRequestMethod(context, null);
+            final Optional<HttpMethod> requestMethodFound = findRequestMethod(method);
+            final HttpMethod requestMethod = requestMethodFound.orElse(HttpMethod.GET);
+            if (requestMethod.isRequestBodySupported()) {
                 return;
             } else if (putToAttribute) {
                 requestFlowFile = session.create();
             }
         }
 
-        // Setting some initial variables
-        final int maxAttributeSize = context.getProperty(PROP_PUT_ATTRIBUTE_MAX_LENGTH).asInteger();
+        final int maxAttributeSize = context.getProperty(RESPONSE_BODY_ATTRIBUTE_SIZE).asInteger();
         final ComponentLog logger = getLogger();
-
-        // Every request/response cycle has a unique transaction id which will be stored as a flowfile attribute.
         final UUID txId = UUID.randomUUID();
 
         FlowFile responseFlowFile = null;
         try {
-            // read the url property from the context
-            final String urlProperty = trimToEmpty(context.getProperty(PROP_URL).evaluateAttributeExpressions(requestFlowFile).getValue());
+            final String urlProperty = trimToEmpty(context.getProperty(HTTP_URL).evaluateAttributeExpressions(requestFlowFile).getValue());
             final URL url = new URL(urlProperty);
 
             Request httpRequest = configureRequest(context, session, requestFlowFile, url);
-
-            // log request
             logRequest(logger, httpRequest);
 
-            // emit send provenance event if successfully sent to the server
             if (httpRequest.body() != null) {
                 session.getProvenanceReporter().send(requestFlowFile, url.toExternalForm(), true);
             }
@@ -954,7 +925,6 @@ public class InvokeHTTP extends AbstractProcessor {
             final long startNanos = System.nanoTime();
 
             try (Response responseHttp = okHttpClient.newCall(httpRequest).execute()) {
-                // output the raw response headers (DEBUG level only)
                 logResponse(logger, url, responseHttp);
 
                 // store the status code and message
@@ -975,16 +945,16 @@ public class InvokeHTTP extends AbstractProcessor {
                 }
 
                 // If the property to add the response headers to the request flowfile is true then add them
-                if (context.getProperty(PROP_ADD_HEADERS_TO_REQUEST).asBoolean() && requestFlowFile != null) {
+                if (context.getProperty(RESPONSE_HEADER_REQUEST_ATTRIBUTES_ENABLED).asBoolean() && requestFlowFile != null) {
                     // write the response headers as attributes
                     // this will overwrite any existing flowfile attributes
                     requestFlowFile = session.putAllAttributes(requestFlowFile, convertAttributesFromHeaders(responseHttp));
                 }
 
                 boolean outputBodyToRequestAttribute = (!isSuccess(statusCode) || putToAttribute) && requestFlowFile != null;
-                boolean outputBodyToResponseContent = (isSuccess(statusCode) && !putToAttribute) || context.getProperty(PROP_OUTPUT_RESPONSE_REGARDLESS).asBoolean();
+                boolean outputBodyToResponseContent = (isSuccess(statusCode) && !putToAttribute) || context.getProperty(RESPONSE_GENERATION_REQUIRED).asBoolean();
                 ResponseBody responseBody = responseHttp.body();
-                boolean bodyExists = responseBody != null && !context.getProperty(IGNORE_RESPONSE_CONTENT).asBoolean();
+                boolean bodyExists = responseBody != null && !context.getProperty(RESPONSE_BODY_IGNORED).asBoolean();
 
                 InputStream responseBodyStream = null;
                 SoftLimitBoundedByteArrayOutputStream outputStreamToRequestAttribute = null;
@@ -1017,7 +987,7 @@ public class InvokeHTTP extends AbstractProcessor {
                         responseFlowFile = session.putAllAttributes(responseFlowFile, convertAttributesFromHeaders(responseHttp));
 
                         // update FlowFile's filename attribute with an extracted value from the remote URL
-                        if (FlowFileNamingStrategy.URL_PATH.equals(getFlowFileNamingStrategy(context)) && GET_METHOD.equals(httpRequest.method())) {
+                        if (FlowFileNamingStrategy.URL_PATH.equals(getFlowFileNamingStrategy(context)) && HttpMethod.GET.name().equals(httpRequest.method())) {
                             String fileName = getFileNameFromUrl(url);
                             if (fileName != null) {
                                 responseFlowFile = session.putAttribute(responseFlowFile, CoreAttributes.FILENAME.key(), fileName);
@@ -1050,7 +1020,7 @@ public class InvokeHTTP extends AbstractProcessor {
 
                     // if not successful and request flowfile is not null, store the response body into a flowfile attribute
                     if (outputBodyToRequestAttribute && bodyExists) {
-                        String attributeKey = context.getProperty(PROP_PUT_OUTPUT_IN_ATTRIBUTE).evaluateAttributeExpressions(requestFlowFile).getValue();
+                        String attributeKey = context.getProperty(RESPONSE_BODY_ATTRIBUTE_NAME).evaluateAttributeExpressions(requestFlowFile).getValue();
                         if (attributeKey == null) {
                             attributeKey = RESPONSE_BODY;
                         }
@@ -1067,9 +1037,9 @@ public class InvokeHTTP extends AbstractProcessor {
                         String bodyString = new String(outputBuffer, 0, size, getCharsetFromMediaType(responseBody.contentType()));
                         requestFlowFile = session.putAttribute(requestFlowFile, attributeKey, bodyString);
 
-                        final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
-                        session.getProvenanceReporter().modifyAttributes(requestFlowFile, "The " + attributeKey + " has been added. The value of which is the body of a http call to "
-                                + url.toExternalForm() + ". It took " + millis + "millis,");
+                        final long processingDuration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
+                        final String eventDetails = String.format("Response Body Attribute Added [%s] Processing Duration [%d ms]", attributeKey, processingDuration);
+                        session.getProvenanceReporter().modifyAttributes(requestFlowFile, eventDetails);
                     }
                 } finally {
                     if (outputStreamToRequestAttribute != null) {
@@ -1086,21 +1056,17 @@ public class InvokeHTTP extends AbstractProcessor {
 
             }
         } catch (final Exception e) {
-            // penalize or yield
-            if (requestFlowFile != null) {
-                logger.error("Routing to {} due to exception: {}", new Object[]{REL_FAILURE.getName(), e}, e);
+            if (requestFlowFile == null) {
+                logger.error("Request Processing failed", e);
+                context.yield();
+            } else {
+                logger.error("Request Processing failed: {}", requestFlowFile, e);
                 requestFlowFile = session.penalize(requestFlowFile);
                 requestFlowFile = session.putAttribute(requestFlowFile, EXCEPTION_CLASS, e.getClass().getName());
                 requestFlowFile = session.putAttribute(requestFlowFile, EXCEPTION_MESSAGE, e.getMessage());
-                // transfer original to failure
-                session.transfer(requestFlowFile, REL_FAILURE);
-            } else {
-                logger.error("Yielding processor due to exception encountered as a source processor: {}", e);
-                context.yield();
+                session.transfer(requestFlowFile, FAILURE);
             }
 
-
-            // cleanup response flowfile, if applicable
             if (responseFlowFile != null) {
                 session.remove(responseFlowFile);
             }
@@ -1111,61 +1077,45 @@ public class InvokeHTTP extends AbstractProcessor {
         final Request.Builder requestBuilder = new Request.Builder();
 
         requestBuilder.url(url);
-        final String authUser = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_USERNAME).getValue());
+        final String authUser = trimToEmpty(context.getProperty(REQUEST_USERNAME).getValue());
 
         // If the username/password properties are set then check if digest auth is being used
-        if ("false".equalsIgnoreCase(context.getProperty(PROP_DIGEST_AUTH).getValue())) {
+        if ("false".equalsIgnoreCase(context.getProperty(REQUEST_DIGEST_AUTHENTICATION_ENABLED).getValue())) {
             if (!authUser.isEmpty()) {
-                final String authPass = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_PASSWORD).getValue());
+                final String authPass = trimToEmpty(context.getProperty(REQUEST_PASSWORD).getValue());
 
                 String credential = Credentials.basic(authUser, authPass);
-                requestBuilder.header("Authorization", credential);
+                requestBuilder.header(HttpHeader.AUTHORIZATION.getHeader(), credential);
             } else {
                 oauth2AccessTokenProviderOptional.ifPresent(oauth2AccessTokenProvider ->
-                    requestBuilder.addHeader("Authorization", "Bearer " + oauth2AccessTokenProvider.getAccessDetails().getAccessToken())
+                    requestBuilder.addHeader(HttpHeader.AUTHORIZATION.getHeader(), "Bearer " + oauth2AccessTokenProvider.getAccessDetails().getAccessToken())
                 );
             }
         }
 
-        final String contentEncoding = context.getProperty(PROP_CONTENT_ENCODING).getValue();
+        final String contentEncoding = context.getProperty(REQUEST_CONTENT_ENCODING).getValue();
         final ContentEncodingStrategy contentEncodingStrategy = ContentEncodingStrategy.valueOf(contentEncoding);
         if (ContentEncodingStrategy.GZIP == contentEncodingStrategy) {
-            requestBuilder.addHeader(CONTENT_ENCODING_HEADER, ContentEncodingStrategy.GZIP.getValue().toLowerCase());
+            requestBuilder.addHeader(HttpHeader.CONTENT_ENCODING.getHeader(), ContentEncodingStrategy.GZIP.getValue().toLowerCase());
         }
 
-        // set the request method
-        String method = trimToEmpty(context.getProperty(PROP_METHOD).evaluateAttributeExpressions(requestFlowFile).getValue()).toUpperCase();
-        switch (method) {
-            case GET_METHOD:
-                requestBuilder.get();
-                break;
-            case POST_METHOD:
-                RequestBody requestBody = getRequestBodyToSend(session, context, requestFlowFile, contentEncodingStrategy);
-                requestBuilder.post(requestBody);
-                break;
-            case PUT_METHOD:
-                requestBody = getRequestBodyToSend(session, context, requestFlowFile, contentEncodingStrategy);
-                requestBuilder.put(requestBody);
-                break;
-            case PATCH_METHOD:
+        final String method = getRequestMethod(context, requestFlowFile);
+        final Optional<HttpMethod> httpMethodFound = findRequestMethod(method);
+
+        final RequestBody requestBody;
+        if (httpMethodFound.isPresent()) {
+            final HttpMethod httpMethod = httpMethodFound.get();
+            if (httpMethod.isRequestBodySupported()) {
                 requestBody = getRequestBodyToSend(session, context, requestFlowFile, contentEncodingStrategy);
-                requestBuilder.patch(requestBody);
-                break;
-            case HEAD_METHOD:
-                requestBuilder.head();
-                break;
-            case DELETE_METHOD:
-                requestBuilder.delete();
-                break;
-            default:
-                requestBuilder.method(method, null);
+            } else {
+                requestBody = null;
+            }
+        } else {
+            requestBody = null;
         }
-
-        String userAgent = trimToEmpty(context.getProperty(PROP_USERAGENT).evaluateAttributeExpressions(requestFlowFile).getValue());
-        requestBuilder.addHeader("User-Agent", userAgent);
+        requestBuilder.method(method, requestBody);
 
         setHeaderProperties(context, requestBuilder, requestFlowFile);
-
         return requestBuilder.build();
     }
 
@@ -1173,24 +1123,24 @@ public class InvokeHTTP extends AbstractProcessor {
                                              final FlowFile requestFlowFile,
                                              final ContentEncodingStrategy contentEncodingStrategy
     ) {
-        boolean sendBody = context.getProperty(PROP_SEND_BODY).asBoolean();
+        boolean requestBodyEnabled = context.getProperty(REQUEST_BODY_ENABLED).asBoolean();
 
-        String evalContentType = context.getProperty(PROP_CONTENT_TYPE)
+        String evalContentType = context.getProperty(REQUEST_CONTENT_TYPE)
                 .evaluateAttributeExpressions(requestFlowFile).getValue();
         final String contentType = StringUtils.isBlank(evalContentType) ? DEFAULT_CONTENT_TYPE : evalContentType;
-        String contentKey = context.getProperty(PROP_FORM_BODY_FORM_NAME).evaluateAttributeExpressions(requestFlowFile).getValue();
+        String formDataName = context.getProperty(REQUEST_FORM_DATA_NAME).evaluateAttributeExpressions(requestFlowFile).getValue();
 
         // Check for dynamic properties for form components.
         // Even if the flowfile is not sent, we may still send form parameters.
         Map<String, PropertyDescriptor> propertyDescriptors = new HashMap<>();
         for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
-            Matcher matcher = DYNAMIC_FORM_PARAMETER_NAME.matcher(entry.getKey().getName());
+            Matcher matcher = FORM_DATA_NAME_PARAMETER_PATTERN.matcher(entry.getKey().getName());
             if (matcher.matches()) {
                 propertyDescriptors.put(matcher.group(FORM_DATA_NAME_GROUP), entry.getKey());
             }
         }
 
-        final boolean contentLengthUnknown = useChunked || ContentEncodingStrategy.GZIP == contentEncodingStrategy;
+        final boolean contentLengthUnknown = chunkedTransferEncoding || ContentEncodingStrategy.GZIP == contentEncodingStrategy;
         RequestBody requestBody = new RequestBody() {
             @Nullable
             @Override
@@ -1221,10 +1171,10 @@ public class InvokeHTTP extends AbstractProcessor {
             }
         };
 
-        if (propertyDescriptors.size() > 0 || StringUtils.isNotEmpty(contentKey)) {
+        if (propertyDescriptors.size() > 0 || StringUtils.isNotEmpty(formDataName)) {
             // we have form data
             MultipartBody.Builder builder = new Builder().setType(MultipartBody.FORM);
-            boolean useFileName = context.getProperty(PROP_SET_FORM_FILE_NAME).asBoolean();
+            boolean useFileName = context.getProperty(REQUEST_FORM_DATA_FILENAME_ENABLED).asBoolean();
             String contentFileName = null;
             if (useFileName) {
                 contentFileName = requestFlowFile.getAttribute(CoreAttributes.FILENAME.key());
@@ -1235,28 +1185,30 @@ public class InvokeHTTP extends AbstractProcessor {
                         .evaluateAttributeExpressions(requestFlowFile).getValue();
                 builder.addFormDataPart(entry.getKey(), propValue);
             }
-            if (sendBody) {
-                builder.addFormDataPart(contentKey, contentFileName, requestBody);
+            if (requestBodyEnabled) {
+                builder.addFormDataPart(formDataName, contentFileName, requestBody);
             }
             return builder.build();
-        } else if (sendBody) {
+        } else if (requestBodyEnabled) {
             return requestBody;
         }
         return RequestBody.create(new byte[0], null);
     }
 
     private void setHeaderProperties(final ProcessContext context, final Request.Builder requestBuilder, final FlowFile requestFlowFile) {
-        // check if we should send the a Date header with the request
-        if (context.getProperty(PROP_DATE_HEADER).asBoolean()) {
+        final String userAgent = trimToEmpty(context.getProperty(REQUEST_USER_AGENT).evaluateAttributeExpressions(requestFlowFile).getValue());
+        requestBuilder.addHeader(HttpHeader.USER_AGENT.getHeader(), userAgent);
+
+        if (context.getProperty(REQUEST_DATE_HEADER_ENABLED).asBoolean()) {
             final ZonedDateTime universalCoordinatedTimeNow = ZonedDateTime.now(ZoneOffset.UTC);
-            requestBuilder.addHeader("Date", RFC_2616_DATE_TIME.format(universalCoordinatedTimeNow));
+            requestBuilder.addHeader(HttpHeader.DATE.getHeader(), RFC_2616_DATE_TIME.format(universalCoordinatedTimeNow));
         }
 
         for (String headerKey : dynamicPropertyNames) {
             String headerValue = context.getProperty(headerKey).evaluateAttributeExpressions(requestFlowFile).getValue();
 
-            // don't include dynamic form data properties
-            if (DYNAMIC_FORM_PARAMETER_NAME.matcher(headerKey).matches()) {
+            // ignore form data name dynamic properties
+            if (FORM_DATA_NAME_PARAMETER_PATTERN.matcher(headerKey).matches()) {
                 continue;
             }
 
@@ -1266,14 +1218,12 @@ public class InvokeHTTP extends AbstractProcessor {
         // iterate through the flowfile attributes, adding any attribute that
         // matches the attributes-to-send pattern. if the pattern is not set
         // (it's an optional property), ignore that attribute entirely
-        if (regexAttributesToSend != null && requestFlowFile != null) {
+        if (requestHeaderAttributesPattern != null && requestFlowFile != null) {
             Map<String, String> attributes = requestFlowFile.getAttributes();
-            Matcher m = regexAttributesToSend.matcher("");
+            Matcher m = requestHeaderAttributesPattern.matcher("");
             for (Map.Entry<String, String> entry : attributes.entrySet()) {
                 String headerKey = trimToEmpty(entry.getKey());
-
-                // don't include any of the ignored attributes
-                if (IGNORED_ATTRIBUTES.contains(headerKey)) {
+                if (IGNORED_REQUEST_ATTRIBUTES.contains(headerKey)) {
                     continue;
                 }
 
@@ -1296,8 +1246,8 @@ public class InvokeHTTP extends AbstractProcessor {
 
         // If the property to output the response flowfile regardless of status code is set then transfer it
         boolean responseSent = false;
-        if (context.getProperty(PROP_OUTPUT_RESPONSE_REGARDLESS).asBoolean()) {
-            session.transfer(response, REL_RESPONSE);
+        if (context.getProperty(RESPONSE_GENERATION_REQUIRED).asBoolean()) {
+            session.transfer(response, RESPONSE);
             responseSent = true;
         }
 
@@ -1306,26 +1256,26 @@ public class InvokeHTTP extends AbstractProcessor {
         if (isSuccess(statusCode)) {
             // we have two flowfiles to transfer
             if (request != null) {
-                session.transfer(request, REL_SUCCESS_REQ);
+                session.transfer(request, ORIGINAL);
             }
             if (response != null && !responseSent) {
-                session.transfer(response, REL_RESPONSE);
+                session.transfer(response, RESPONSE);
             }
 
             // 5xx -> RETRY
         } else if (statusCode / 100 == 5) {
             if (request != null) {
                 request = session.penalize(request);
-                session.transfer(request, REL_RETRY);
+                session.transfer(request, RETRY);
             }
 
             // 1xx, 3xx, 4xx -> NO RETRY
         } else {
             if (request != null) {
-                if (context.getProperty(PROP_PENALIZE_NO_RETRY).asBoolean()) {
+                if (context.getProperty(REQUEST_FAILURE_PENALIZATION_ENABLED).asBoolean()) {
                     request = session.penalize(request);
                 }
-                session.transfer(request, REL_NO_RETRY);
+                session.transfer(request, NO_RETRY);
             }
         }
 
@@ -1356,7 +1306,7 @@ public class InvokeHTTP extends AbstractProcessor {
                 if (list.size() == 1) {
                     sb.append(list.get(0));
                 } else {
-                    sb.append(list.toString());
+                    sb.append(list);
                 }
                 sb.append("\n");
             }
@@ -1396,21 +1346,12 @@ public class InvokeHTTP extends AbstractProcessor {
         return contentType != null ? contentType.charset(StandardCharsets.UTF_8) : StandardCharsets.UTF_8;
     }
 
-    /**
-     * Retrieve the directory in which OkHttp should cache responses. This method opts
-     * to use a temp directory to write the cache, which means that the cache will be written
-     * to a new location each time this processor is scheduled.
-     * <p>
-     * Ref: https://github.com/square/okhttp/wiki/Recipes#response-caching
-     *
-     * @return the directory in which the ETag cache should be written
-     */
-    private static File getETagCacheDir() throws IOException {
+    private static File getResponseCacheDirectory() throws IOException {
         return Files.createTempDirectory(InvokeHTTP.class.getSimpleName()).toFile();
     }
 
     private FlowFileNamingStrategy getFlowFileNamingStrategy(final ProcessContext context) {
-        final String strategy = context.getProperty(FLOW_FILE_NAMING_STRATEGY).getValue();
+        final String strategy = context.getProperty(RESPONSE_FLOW_FILE_NAMING_STRATEGY).getValue();
         return FlowFileNamingStrategy.valueOf(strategy);
     }
 
@@ -1424,4 +1365,15 @@ public class InvokeHTTP extends AbstractProcessor {
 
         return fileName;
     }
+
+    private Optional<HttpMethod> findRequestMethod(String method) {
+        return Arrays.stream(HttpMethod.values())
+                .filter(httpMethod -> httpMethod.name().equals(method))
+                .findFirst();
+    }
+
+    private String getRequestMethod(final PropertyContext context, final FlowFile flowFile) {
+        final String method = context.getProperty(HTTP_METHOD).evaluateAttributeExpressions(flowFile).getValue().toUpperCase();
+        return trimToEmpty(method);
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/http/HttpHeader.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/http/HttpHeader.java
new file mode 100644
index 0000000000..34bb25b792
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/http/HttpHeader.java
@@ -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.processors.standard.http;
+
+/**
+ * HTTP Header reusable enumerated values
+ */
+public enum HttpHeader {
+    /** Authorization defined in RFC 7235 Section 4.2 */
+    AUTHORIZATION("Authorization"),
+
+    /** Content-Encoding defined in RFC 7231 Section 3.1.2.2 */
+    CONTENT_ENCODING("Content-Encoding"),
+
+    /** Date defined in RFC 7231 Section 7.1.1.2 */
+    DATE("Date"),
+
+    /** User-Agent defined in RFC 7231 Section 5.5.3 */
+    USER_AGENT("User-Agent");
+
+    private final String header;
+
+    HttpHeader(final String header) {
+        this.header = header;
+    }
+
+    public String getHeader() {
+        return header;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/http/HttpMethod.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/http/HttpMethod.java
new file mode 100644
index 0000000000..c0fdbcade9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/http/HttpMethod.java
@@ -0,0 +1,46 @@
+/*
+ * 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.http;
+
+/**
+ * HTTP Method reusable enumerated values
+ */
+public enum HttpMethod {
+    DELETE(false),
+
+    GET(false),
+
+    HEAD(false),
+
+    OPTIONS(false),
+
+    PATCH(true),
+
+    POST(true),
+
+    PUT(true);
+
+    private final boolean requestBodySupported;
+
+    HttpMethod(final boolean requestBodySupported) {
+        this.requestBodySupported = requestBodySupported;
+    }
+
+    public boolean isRequestBodySupported() {
+        return requestBodySupported;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/InvokeHTTPTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/InvokeHTTPTest.java
index ce0c2603db..224706b7f6 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/InvokeHTTPTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/InvokeHTTPTest.java
@@ -28,6 +28,8 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processors.standard.http.ContentEncodingStrategy;
 import org.apache.nifi.processors.standard.http.FlowFileNamingStrategy;
 import org.apache.nifi.processors.standard.http.CookieStrategy;
+import org.apache.nifi.processors.standard.http.HttpHeader;
+import org.apache.nifi.processors.standard.http.HttpMethod;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.security.util.StandardTlsConfiguration;
 import org.apache.nifi.security.util.TemporaryKeyStoreBuilder;
@@ -74,13 +76,6 @@ import static java.net.HttpURLConnection.HTTP_MOVED_TEMP;
 import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
 import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED;
 import static java.net.HttpURLConnection.HTTP_INTERNAL_ERROR;
-import static org.apache.nifi.processors.standard.InvokeHTTP.GET_METHOD;
-import static org.apache.nifi.processors.standard.InvokeHTTP.POST_METHOD;
-import static org.apache.nifi.processors.standard.InvokeHTTP.PUT_METHOD;
-import static org.apache.nifi.processors.standard.InvokeHTTP.PATCH_METHOD;
-import static org.apache.nifi.processors.standard.InvokeHTTP.DELETE_METHOD;
-import static org.apache.nifi.processors.standard.InvokeHTTP.HEAD_METHOD;
-import static org.apache.nifi.processors.standard.InvokeHTTP.OPTIONS_METHOD;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -164,7 +159,7 @@ public class InvokeHTTPTest {
         mockWebServer = new MockWebServer();
         runner = TestRunners.newTestRunner(new InvokeHTTP());
         // Disable Connection Pooling
-        runner.setProperty(InvokeHTTP.PROP_MAX_IDLE_CONNECTIONS, Integer.toString(0));
+        runner.setProperty(InvokeHTTP.SOCKET_IDLE_CONNECTIONS, Integer.toString(0));
     }
 
     @AfterEach
@@ -177,61 +172,50 @@ public class InvokeHTTPTest {
         runner.assertNotValid();
     }
 
-    @Test
-    public void testNotValidWithProxyTypeInvalid() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
-        runner.setProperty(InvokeHTTP.PROP_PROXY_TYPE, String.class.getSimpleName());
-        runner.assertNotValid();
-    }
-
+    @SuppressWarnings("deprecation")
     @Test
     public void testNotValidWithProxyHostWithoutProxyPort() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
-        runner.setProperty(InvokeHTTP.PROP_PROXY_HOST, String.class.getSimpleName());
+        runner.setProperty(InvokeHTTP.HTTP_URL, HTTP_LOCALHOST_URL);
+        runner.setProperty(InvokeHTTP.PROXY_HOST, String.class.getSimpleName());
         runner.assertNotValid();
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testNotValidWithProxyUserWithoutProxyPassword() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
-        runner.setProperty(InvokeHTTP.PROP_PROXY_USER, String.class.getSimpleName());
+        runner.setProperty(InvokeHTTP.HTTP_URL, HTTP_LOCALHOST_URL);
+        runner.setProperty(InvokeHTTP.PROXY_USERNAME, String.class.getSimpleName());
         runner.assertNotValid();
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testNotValidWithProxyUserAndPasswordWithoutProxyHost() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
-        runner.setProperty(InvokeHTTP.PROP_PROXY_USER, String.class.getSimpleName());
-        runner.setProperty(InvokeHTTP.PROP_PROXY_PASSWORD, String.class.getSimpleName());
-        runner.assertNotValid();
-    }
-
-    @Test
-    public void testNotValidWithHttpsProxyTypeWithoutSslContextService() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
-        runner.setProperty(InvokeHTTP.PROP_PROXY_TYPE, InvokeHTTP.HTTPS);
+        runner.setProperty(InvokeHTTP.HTTP_URL, HTTP_LOCALHOST_URL);
+        runner.setProperty(InvokeHTTP.PROXY_USERNAME, String.class.getSimpleName());
+        runner.setProperty(InvokeHTTP.PROXY_PASSWORD, String.class.getSimpleName());
         runner.assertNotValid();
     }
 
     @Test
     public void testNotValidWithPostFormPropertyWithoutFormBodyFormName() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
+        runner.setProperty(InvokeHTTP.HTTP_URL, HTTP_LOCALHOST_URL);
         runner.setProperty(POST_FORM_PARAMETER_KEY, String.class.getSimpleName());
         runner.assertNotValid();
     }
 
     @Test
-    public void testNotValidWithPostFormPropertyAndFormBodyFormNameWithoutSendBodyEnabled() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
+    public void testNotValidWithPostFormPropertyAndFormBodyFormNameWithoutRequestBodyEnabled() {
+        runner.setProperty(InvokeHTTP.HTTP_URL, HTTP_LOCALHOST_URL);
         runner.setProperty(POST_FORM_PARAMETER_KEY, String.class.getSimpleName());
-        runner.setProperty(InvokeHTTP.PROP_FORM_BODY_FORM_NAME, String.class.getSimpleName());
-        runner.setProperty(InvokeHTTP.PROP_SEND_BODY, Boolean.FALSE.toString());
+        runner.setProperty(InvokeHTTP.REQUEST_FORM_DATA_NAME, String.class.getSimpleName());
+        runner.setProperty(InvokeHTTP.REQUEST_BODY_ENABLED, Boolean.FALSE.toString());
         runner.assertNotValid();
     }
 
     @Test
     public void testValidWithMinimumProperties() {
-        runner.setProperty(InvokeHTTP.PROP_URL, HTTP_LOCALHOST_URL);
+        runner.setProperty(InvokeHTTP.HTTP_URL, HTTP_LOCALHOST_URL);
         runner.assertValid();
     }
 
@@ -250,7 +234,7 @@ public class InvokeHTTPTest {
         runner.setIncomingConnection(false);
         runner.setNonLoopConnection(false);
         setUrlProperty();
-        runner.setProperty(InvokeHTTP.PROP_METHOD, POST_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.POST.name());
 
         runner.run();
         runner.assertQueueEmpty();
@@ -261,7 +245,7 @@ public class InvokeHTTPTest {
         runner.setIncomingConnection(false);
         runner.setNonLoopConnection(false);
 
-        runner.setProperty(InvokeHTTP.PROP_URL, "${file.name}");
+        runner.setProperty(InvokeHTTP.HTTP_URL, "${file.name}");
 
         runner.run();
 
@@ -272,14 +256,14 @@ public class InvokeHTTPTest {
     @Test
     public void testRunGetMalformedUrlExceptionFailure() {
         final String urlAttributeKey = "request.url";
-        runner.setProperty(InvokeHTTP.PROP_URL, String.format("${%s}", urlAttributeKey));
+        runner.setProperty(InvokeHTTP.HTTP_URL, String.format("${%s}", urlAttributeKey));
 
         final Map<String, String> attributes = new HashMap<>();
         attributes.put(urlAttributeKey, String.class.getSimpleName());
         runner.enqueue(FLOW_FILE_CONTENT, attributes);
         runner.run();
 
-        runner.assertAllFlowFilesTransferred(InvokeHTTP.REL_FAILURE);
+        runner.assertAllFlowFilesTransferred(InvokeHTTP.FAILURE);
         runner.assertPenalizeCount(1);
 
         final MockFlowFile flowFile = getFailureFlowFile();
@@ -291,14 +275,14 @@ public class InvokeHTTPTest {
     public void testRunGetMethodIllegalArgumentExceptionFailure() {
         setUrlProperty();
         final String methodAttributeKey = "request.method";
-        runner.setProperty(InvokeHTTP.PROP_METHOD, String.format("${%s}", methodAttributeKey));
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, String.format("${%s}", methodAttributeKey));
 
         final Map<String, String> attributes = new HashMap<>();
         attributes.put(methodAttributeKey, null);
         runner.enqueue(FLOW_FILE_CONTENT, attributes);
         runner.run();
 
-        runner.assertAllFlowFilesTransferred(InvokeHTTP.REL_FAILURE);
+        runner.assertAllFlowFilesTransferred(InvokeHTTP.FAILURE);
         runner.assertPenalizeCount(1);
 
         final MockFlowFile flowFile = getFailureFlowFile();
@@ -308,22 +292,22 @@ public class InvokeHTTPTest {
 
     @Test
     public void testRunGetHttp200Success() throws InterruptedException {
-        assertRequestMethodSuccess(GET_METHOD);
+        assertRequestMethodSuccess(HttpMethod.GET);
     }
 
     @Test
-    public void testRunGetHttp200SuccessIgnoreResponseContentEnabled() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.IGNORE_RESPONSE_CONTENT, Boolean.TRUE.toString());
-        assertRequestMethodSuccess(GET_METHOD);
+    public void testRunGetHttp200SuccessResponseBodyIgnoredEnabled() throws InterruptedException {
+        runner.setProperty(InvokeHTTP.RESPONSE_BODY_IGNORED, Boolean.TRUE.toString());
+        assertRequestMethodSuccess(HttpMethod.GET);
 
         final MockFlowFile responseFlowFile = getResponseFlowFile();
         assertEquals(StringUtils.EMPTY, responseFlowFile.getContent());
     }
 
     @Test
-    public void testRunGetHttp200SuccessOutputBodyAttribute() {
+    public void testRunGetHttp200SuccessResponseBodyAttributeName() {
         final String outputAttributeKey = String.class.getSimpleName();
-        runner.setProperty(InvokeHTTP.PROP_PUT_OUTPUT_IN_ATTRIBUTE, outputAttributeKey);
+        runner.setProperty(InvokeHTTP.RESPONSE_BODY_ATTRIBUTE_NAME, outputAttributeKey);
         setUrlProperty();
 
         final String body = String.class.getName();
@@ -331,16 +315,16 @@ public class InvokeHTTPTest {
         runner.enqueue(FLOW_FILE_CONTENT);
         runner.run();
 
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_SUCCESS_REQ, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.ORIGINAL, HTTP_OK);
 
         final MockFlowFile flowFile = getRequestFlowFile();
         flowFile.assertAttributeEquals(outputAttributeKey, body);
     }
 
     @Test
-    public void testRunGetHttp200SuccessOutputBodyAttributeNoIncomingConnections() {
+    public void testRunGetHttp200SuccessResponseBodyAttributeNameNoIncomingConnections() {
         final String outputAttributeKey = String.class.getSimpleName();
-        runner.setProperty(InvokeHTTP.PROP_PUT_OUTPUT_IN_ATTRIBUTE, outputAttributeKey);
+        runner.setProperty(InvokeHTTP.RESPONSE_BODY_ATTRIBUTE_NAME, outputAttributeKey);
         setUrlProperty();
         runner.setIncomingConnection(false);
         runner.setNonLoopConnection(false);
@@ -349,9 +333,9 @@ public class InvokeHTTPTest {
         mockWebServer.enqueue(new MockResponse().setResponseCode(HTTP_OK).setBody(body));
         runner.run();
 
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_SUCCESS_REQ, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.ORIGINAL, HTTP_OK);
 
-        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(InvokeHTTP.REL_SUCCESS_REQ).iterator().next();
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(InvokeHTTP.ORIGINAL).iterator().next();
         flowFile.assertAttributeEquals(outputAttributeKey, body);
     }
 
@@ -364,50 +348,52 @@ public class InvokeHTTPTest {
         mockWebServer.enqueue(new MockResponse().setResponseCode(HTTP_OK));
         runner.run();
 
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testRunGetHttp200SuccessProxyHostPortConfigured() throws InterruptedException {
         final String mockWebServerUrl = getMockWebServerUrl();
         final URI uri = URI.create(mockWebServerUrl);
 
-        runner.setProperty(InvokeHTTP.PROP_URL, mockWebServerUrl);
-        runner.setProperty(InvokeHTTP.PROP_PROXY_HOST, uri.getHost());
-        runner.setProperty(InvokeHTTP.PROP_PROXY_PORT, Integer.toString(uri.getPort()));
+        runner.setProperty(InvokeHTTP.HTTP_URL, mockWebServerUrl);
+        runner.setProperty(InvokeHTTP.PROXY_HOST, uri.getHost());
+        runner.setProperty(InvokeHTTP.PROXY_PORT, Integer.toString(uri.getPort()));
 
         mockWebServer.enqueue(new MockResponse().setResponseCode(HTTP_OK));
         runner.enqueue(FLOW_FILE_CONTENT);
         runner.run();
 
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
         final RecordedRequest request = takeRequestCompleted();
         final String requestLine = request.getRequestLine();
 
-        final String proxyRequestLine = String.format("%s %s HTTP/1.1", GET_METHOD, mockWebServerUrl);
+        final String proxyRequestLine = String.format("%s %s HTTP/1.1", HttpMethod.GET.name(), mockWebServerUrl);
         assertEquals(proxyRequestLine, requestLine);
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testRunGetHttp200SuccessProxyHostPortUserPasswordConfigured() throws InterruptedException {
         final String mockWebServerUrl = getMockWebServerUrl();
         final URI uri = URI.create(mockWebServerUrl);
 
-        runner.setProperty(InvokeHTTP.PROP_URL, mockWebServerUrl);
-        runner.setProperty(InvokeHTTP.PROP_PROXY_HOST, uri.getHost());
-        runner.setProperty(InvokeHTTP.PROP_PROXY_PORT, Integer.toString(uri.getPort()));
-        runner.setProperty(InvokeHTTP.PROP_PROXY_USER, String.class.getSimpleName());
-        runner.setProperty(InvokeHTTP.PROP_PROXY_PASSWORD, String.class.getName());
+        runner.setProperty(InvokeHTTP.HTTP_URL, mockWebServerUrl);
+        runner.setProperty(InvokeHTTP.PROXY_HOST, uri.getHost());
+        runner.setProperty(InvokeHTTP.PROXY_PORT, Integer.toString(uri.getPort()));
+        runner.setProperty(InvokeHTTP.PROXY_USERNAME, String.class.getSimpleName());
+        runner.setProperty(InvokeHTTP.PROXY_PASSWORD, String.class.getName());
 
         mockWebServer.enqueue(new MockResponse().setResponseCode(HTTP_OK));
         runner.enqueue(FLOW_FILE_CONTENT);
         runner.run();
 
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
         final RecordedRequest request = takeRequestCompleted();
         final String requestLine = request.getRequestLine();
 
-        final String proxyRequestLine = String.format("%s %s HTTP/1.1", GET_METHOD, mockWebServerUrl);
+        final String proxyRequestLine = String.format("%s %s HTTP/1.1", HttpMethod.GET.name(), mockWebServerUrl);
         assertEquals(proxyRequestLine, requestLine);
     }
 
@@ -421,7 +407,7 @@ public class InvokeHTTPTest {
         runner.run();
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final MockFlowFile responseFlowFile = getResponseFlowFile();
         responseFlowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), TEXT_PLAIN);
@@ -429,12 +415,12 @@ public class InvokeHTTPTest {
 
     @Test
     public void testRunGetHttp200SuccessRequestDateHeader() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_DATE_HEADER, StringUtils.capitalize(Boolean.TRUE.toString()));
+        runner.setProperty(InvokeHTTP.REQUEST_DATE_HEADER_ENABLED, StringUtils.capitalize(Boolean.TRUE.toString()));
 
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         final String dateHeader = request.getHeader(DATE_HEADER);
@@ -448,8 +434,8 @@ public class InvokeHTTPTest {
     }
 
     @Test
-    public void testRunGetHttp200SuccessSendAttributesAndDynamicProperties() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_ATTRIBUTES_TO_SEND, String.format("^%s$", ACCEPT_HEADER));
+    public void testRunGetHttp200SuccessRequestHeaderAttributesAndDynamicProperties() throws InterruptedException {
+        runner.setProperty(InvokeHTTP.REQUEST_HEADER_ATTRIBUTES_PATTERN, String.format("^%s$", ACCEPT_HEADER));
         final String defaultContentTypeHeader = "Default-Content-Type";
         runner.setProperty(defaultContentTypeHeader, InvokeHTTP.DEFAULT_CONTENT_TYPE);
         setUrlProperty();
@@ -461,7 +447,7 @@ public class InvokeHTTPTest {
         runner.run();
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         final String acceptHeader = request.getHeader(ACCEPT_HEADER);
@@ -470,7 +456,7 @@ public class InvokeHTTPTest {
         final String contentType = request.getHeader(defaultContentTypeHeader);
         assertEquals(InvokeHTTP.DEFAULT_CONTENT_TYPE, contentType);
 
-        runner.removeProperty(InvokeHTTP.PROP_ATTRIBUTES_TO_SEND);
+        runner.removeProperty(InvokeHTTP.REQUEST_HEADER_ATTRIBUTES_PATTERN);
         runner.removeProperty(defaultContentTypeHeader);
         mockWebServer.enqueue(new MockResponse().setResponseCode(HTTP_OK));
         runner.enqueue(FLOW_FILE_CONTENT, attributes);
@@ -482,9 +468,9 @@ public class InvokeHTTPTest {
     }
 
     @Test
-    public void testRunGetHttp200SuccessResponseHeaderRequestFlowFileAttributes() {
+    public void testRunGetHttp200SuccessResponseHeaderRequestAttributes() {
         setUrlProperty();
-        runner.setProperty(InvokeHTTP.PROP_ADD_HEADERS_TO_REQUEST, Boolean.TRUE.toString());
+        runner.setProperty(InvokeHTTP.RESPONSE_HEADER_REQUEST_ATTRIBUTES_ENABLED, Boolean.TRUE.toString());
 
         final String firstHeader = String.class.getSimpleName();
         final String secondHeader = Integer.class.getSimpleName();
@@ -498,7 +484,7 @@ public class InvokeHTTPTest {
         runner.run();
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final MockFlowFile requestFlowFile = getRequestFlowFile();
         requestFlowFile.assertAttributeEquals(CONTENT_LENGTH_HEADER, Integer.toString(0));
@@ -508,21 +494,21 @@ public class InvokeHTTPTest {
     }
 
     @Test
-    public void testRunGetHttp200SuccessCacheTagEnabled() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_USE_ETAG, Boolean.TRUE.toString());
+    public void testRunGetHttp200SuccessResponseCacheEnabled() throws InterruptedException {
+        runner.setProperty(InvokeHTTP.RESPONSE_CACHE_ENABLED, Boolean.TRUE.toString());
 
-        assertRequestMethodSuccess(GET_METHOD);
+        assertRequestMethodSuccess(HttpMethod.GET);
     }
 
     @Test
     public void testRunGetHttp200SuccessBasicAuthentication() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_BASIC_AUTH_USERNAME, String.class.getSimpleName());
-        runner.setProperty(InvokeHTTP.PROP_BASIC_AUTH_PASSWORD, String.class.getName());
+        runner.setProperty(InvokeHTTP.REQUEST_USERNAME, String.class.getSimpleName());
+        runner.setProperty(InvokeHTTP.REQUEST_PASSWORD, String.class.getName());
 
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         final String authorization = request.getHeader(AUTHORIZATION_HEADER);
@@ -534,9 +520,9 @@ public class InvokeHTTPTest {
 
     @Test
     public void testRunGetHttp200SuccessDigestAuthentication() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_BASIC_AUTH_USERNAME, String.class.getSimpleName());
-        runner.setProperty(InvokeHTTP.PROP_BASIC_AUTH_PASSWORD, String.class.getName());
-        runner.setProperty(InvokeHTTP.PROP_DIGEST_AUTH, Boolean.TRUE.toString());
+        runner.setProperty(InvokeHTTP.REQUEST_USERNAME, String.class.getSimpleName());
+        runner.setProperty(InvokeHTTP.REQUEST_PASSWORD, String.class.getName());
+        runner.setProperty(InvokeHTTP.REQUEST_DIGEST_AUTHENTICATION_ENABLED, Boolean.TRUE.toString());
 
         final String realm = UUID.randomUUID().toString();
         final String nonce = UUID.randomUUID().toString();
@@ -546,7 +532,7 @@ public class InvokeHTTPTest {
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         assertNull(request.getHeader(AUTHORIZATION_HEADER), "Authorization Header found");
@@ -570,7 +556,7 @@ public class InvokeHTTPTest {
 
     @Test
     public void testRunGetSslContextServiceMutualTrustedClientCertificateMissing() throws InitializationException, GeneralSecurityException {
-        runner.setProperty(InvokeHTTP.DISABLE_HTTP2_PROTOCOL, StringUtils.capitalize(Boolean.TRUE.toString()));
+        runner.setProperty(InvokeHTTP.HTTP2_DISABLED, StringUtils.capitalize(Boolean.TRUE.toString()));
         setSslContextConfiguration(generatedTlsConfiguration, truststoreTlsConfiguration);
         mockWebServer.requireClientAuth();
 
@@ -578,7 +564,7 @@ public class InvokeHTTPTest {
         runner.enqueue(FLOW_FILE_CONTENT);
         runner.run();
 
-        runner.assertAllFlowFilesTransferred(InvokeHTTP.REL_FAILURE);
+        runner.assertAllFlowFilesTransferred(InvokeHTTP.FAILURE);
         final MockFlowFile flowFile = getFailureFlowFile();
         flowFile.assertAttributeExists(InvokeHTTP.EXCEPTION_CLASS);
         flowFile.assertAttributeExists(InvokeHTTP.EXCEPTION_MESSAGE);
@@ -587,12 +573,12 @@ public class InvokeHTTPTest {
     @Test
     public void testRunGetHttp200SuccessUserAgentConfigured() throws InterruptedException {
         final String userAgent = UUID.randomUUID().toString();
-        runner.setProperty(InvokeHTTP.PROP_USERAGENT, userAgent);
+        runner.setProperty(InvokeHTTP.REQUEST_USER_AGENT, userAgent);
 
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         final String userAgentHeader = request.getHeader(USER_AGENT_HEADER);
@@ -600,28 +586,28 @@ public class InvokeHTTPTest {
     }
 
     @Test
-    public void testRunGetHttp302NoRetryFollowRedirectsDefaultEnabled() {
+    public void testRunGetHttp302NoRetryResponseRedirectsDefaultEnabled() {
         mockWebServer.enqueue(new MockResponse().setResponseCode(HTTP_MOVED_TEMP).setHeader(LOCATION_HEADER, getMockWebServerUrl()));
         enqueueResponseCodeAndRun(HTTP_OK);
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_NO_RETRY, 0);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.NO_RETRY, 0);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
     }
 
     @Test
-    public void testRunGetHttp302NoRetryFollowRedirectsDisabled() {
-        runner.setProperty(InvokeHTTP.PROP_FOLLOW_REDIRECTS, StringUtils.capitalize(Boolean.FALSE.toString()));
+    public void testRunGetHttp302NoRetryResponseRedirectsDisabled() {
+        runner.setProperty(InvokeHTTP.RESPONSE_REDIRECTS_ENABLED, StringUtils.capitalize(Boolean.FALSE.toString()));
         enqueueResponseCodeAndRun(HTTP_MOVED_TEMP);
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_RESPONSE, 0);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_NO_RETRY, HTTP_MOVED_TEMP);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.RESPONSE, 0);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.NO_RETRY, HTTP_MOVED_TEMP);
     }
 
     @Test
     public void testRunGetHttp302CookieStrategyAcceptAll() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_COOKIE_STRATEGY, CookieStrategy.ACCEPT_ALL.name());
+        runner.setProperty(InvokeHTTP.RESPONSE_COOKIE_STRATEGY, CookieStrategy.ACCEPT_ALL.name());
         mockWebServer.enqueue(new MockResponse().setResponseCode(HTTP_MOVED_TEMP)
             .addHeader(SET_COOKIE_HEADER, COOKIE_1)
             .addHeader(SET_COOKIE_HEADER, COOKIE_2)
@@ -635,9 +621,9 @@ public class InvokeHTTPTest {
         final String expectedHeader = String.format("%s; %s", COOKIE_1, COOKIE_2);
         assertEquals(expectedHeader, request2.getHeader(COOKIE_HEADER));
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_NO_RETRY, 0);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.NO_RETRY, 0);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
     }
 
     @Test
@@ -654,92 +640,92 @@ public class InvokeHTTPTest {
         RecordedRequest request2 = mockWebServer.takeRequest();
         assertNull(request2.getHeader(COOKIE_HEADER));
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_NO_RETRY, 0);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.NO_RETRY, 0);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
     }
 
     @Test
     public void testRunGetHttp400NoRetryMinimumProperties() {
         enqueueResponseCodeAndRun(HTTP_BAD_REQUEST);
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_RESPONSE, 0);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_NO_RETRY, HTTP_BAD_REQUEST);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.RESPONSE, 0);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.NO_RETRY, HTTP_BAD_REQUEST);
     }
 
     @Test
     public void testRunGetHttp400NoRetryPenalizeNoRetry() {
-        runner.setProperty(InvokeHTTP.PROP_PENALIZE_NO_RETRY, Boolean.TRUE.toString());
+        runner.setProperty(InvokeHTTP.REQUEST_FAILURE_PENALIZATION_ENABLED, Boolean.TRUE.toString());
 
         enqueueResponseCodeAndRun(HTTP_BAD_REQUEST);
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_RESPONSE, 0);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.RESPONSE, 0);
         runner.assertPenalizeCount(1);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_NO_RETRY, HTTP_BAD_REQUEST);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.NO_RETRY, HTTP_BAD_REQUEST);
     }
 
     @Test
     public void testRunGetHttp500RetryMinimumProperties() {
         enqueueResponseCodeAndRun(HTTP_INTERNAL_ERROR);
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_RESPONSE, 0);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RETRY, HTTP_INTERNAL_ERROR);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.RESPONSE, 0);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RETRY, HTTP_INTERNAL_ERROR);
     }
 
     @Test
-    public void testRunGetHttp500RetryOutputResponseRegardless() {
-        runner.setProperty(InvokeHTTP.PROP_OUTPUT_RESPONSE_REGARDLESS, Boolean.TRUE.toString());
+    public void testRunGetHttp500RetryResponseGeneratedRequired() {
+        runner.setProperty(InvokeHTTP.RESPONSE_GENERATION_REQUIRED, Boolean.TRUE.toString());
 
         enqueueResponseCodeAndRun(HTTP_INTERNAL_ERROR);
 
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RETRY, HTTP_INTERNAL_ERROR);
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_INTERNAL_ERROR);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RETRY, HTTP_INTERNAL_ERROR);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_INTERNAL_ERROR);
     }
 
     @Test
     public void testRunDeleteHttp200Success() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, DELETE_METHOD);
-        assertRequestMethodSuccess(DELETE_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.DELETE.name());
+        assertRequestMethodSuccess(HttpMethod.DELETE);
     }
 
     @Test
     public void testRunHeadHttp200Success() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, HEAD_METHOD);
-        assertRequestMethodSuccess(HEAD_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.HEAD.name());
+        assertRequestMethodSuccess(HttpMethod.HEAD);
     }
 
     @Test
     public void testRunOptionsHttp200Success() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, OPTIONS_METHOD);
-        assertRequestMethodSuccess(OPTIONS_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.OPTIONS.name());
+        assertRequestMethodSuccess(HttpMethod.OPTIONS);
     }
 
     @Test
     public void testRunPatchHttp200Success() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, PATCH_METHOD);
-        assertRequestMethodSuccess(PATCH_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.PATCH.name());
+        assertRequestMethodSuccess(HttpMethod.PATCH);
     }
 
     @Test
     public void testRunPostHttp200Success() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, POST_METHOD);
-        assertRequestMethodSuccess(POST_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.POST.name());
+        assertRequestMethodSuccess(HttpMethod.POST);
     }
 
     @Test
     public void testRunPostHttp200SuccessContentEncodingGzip() throws InterruptedException, IOException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, POST_METHOD);
-        runner.setProperty(InvokeHTTP.PROP_CONTENT_ENCODING, ContentEncodingStrategy.GZIP.getValue());
-        runner.setProperty(InvokeHTTP.PROP_SEND_BODY, Boolean.TRUE.toString());
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.POST.name());
+        runner.setProperty(InvokeHTTP.REQUEST_CONTENT_ENCODING, ContentEncodingStrategy.GZIP.getValue());
+        runner.setProperty(InvokeHTTP.REQUEST_BODY_ENABLED, Boolean.TRUE.toString());
 
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         final String contentLength = request.getHeader(CONTENT_LENGTH_HEADER);
@@ -757,13 +743,13 @@ public class InvokeHTTPTest {
 
     @Test
     public void testRunPostHttp200SuccessChunkedEncoding() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, POST_METHOD);
-        runner.setProperty(InvokeHTTP.PROP_USE_CHUNKED_ENCODING, Boolean.TRUE.toString());
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.POST.name());
+        runner.setProperty(InvokeHTTP.REQUEST_CHUNKED_TRANSFER_ENCODING_ENABLED, Boolean.TRUE.toString());
 
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         final String contentLength = request.getHeader(CONTENT_LENGTH_HEADER);
@@ -775,14 +761,14 @@ public class InvokeHTTPTest {
 
     @Test
     public void testRunPostHttp200SuccessFormData() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, POST_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.POST.name());
 
         final String formName = "multipart-form";
-        runner.setProperty(InvokeHTTP.PROP_FORM_BODY_FORM_NAME, formName);
+        runner.setProperty(InvokeHTTP.REQUEST_FORM_DATA_NAME, formName);
 
         final String formDataParameter = String.class.getName();
         final String formDataParameterName = "label";
-        final String formDataPropertyName = String.format("%s:%s", InvokeHTTP.FORM_BASE, formDataParameterName);
+        final String formDataPropertyName = String.format("%s:%s", InvokeHTTP.FORM_DATA_NAME_BASE, formDataParameterName);
         runner.setProperty(formDataPropertyName, formDataParameter);
 
         setUrlProperty();
@@ -791,7 +777,7 @@ public class InvokeHTTPTest {
         runner.run();
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
         final String contentType = request.getHeader(CONTENT_TYPE_HEADER);
@@ -806,8 +792,8 @@ public class InvokeHTTPTest {
 
     @Test
     public void testRunPutHttp200Success() throws InterruptedException {
-        runner.setProperty(InvokeHTTP.PROP_METHOD, PUT_METHOD);
-        assertRequestMethodSuccess(PUT_METHOD);
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, HttpMethod.PUT.name());
+        assertRequestMethodSuccess(HttpMethod.PUT);
     }
 
     @ParameterizedTest(name = "{index} => When {0} http://baseUrl/{1}, filename of the response FlowFile should be {2}")
@@ -816,9 +802,9 @@ public class InvokeHTTPTest {
         URL baseUrl = new URL(getMockWebServerUrl());
         URL targetUrl = new URL(baseUrl, inputUrl);
 
-        runner.setProperty(InvokeHTTP.PROP_METHOD, httpMethod);
-        runner.setProperty(InvokeHTTP.PROP_URL, targetUrl.toString());
-        runner.setProperty(InvokeHTTP.FLOW_FILE_NAMING_STRATEGY, FlowFileNamingStrategy.URL_PATH.name());
+        runner.setProperty(InvokeHTTP.HTTP_METHOD, httpMethod);
+        runner.setProperty(InvokeHTTP.HTTP_URL, targetUrl.toString());
+        runner.setProperty(InvokeHTTP.RESPONSE_FLOW_FILE_NAMING_STRATEGY, FlowFileNamingStrategy.URL_PATH.name());
 
         Map<String, String> ffAttributes = new HashMap<>();
         ffAttributes.put(CoreAttributes.FILENAME.key(), FLOW_FILE_INITIAL_FILENAME);
@@ -828,38 +814,37 @@ public class InvokeHTTPTest {
 
         runner.run();
 
-        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(InvokeHTTP.REL_RESPONSE).iterator().next();
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(InvokeHTTP.RESPONSE).iterator().next();
         flowFile.assertAttributeEquals(CoreAttributes.FILENAME.key(), expectedFileName);
     }
 
     private static Stream<Arguments> testResponseFlowFileFilenameExtractedFromRemoteUrl() {
         return Stream.of(
-            Arguments.of(GET_METHOD, "file", "file"),
-            Arguments.of(GET_METHOD, "file/", "file"),
-            Arguments.of(GET_METHOD, "file.txt", "file.txt"),
-            Arguments.of(GET_METHOD, "file.txt/", "file.txt"),
-            Arguments.of(GET_METHOD, "file.txt/?qp=v", "file.txt"),
-            Arguments.of(GET_METHOD, "f%69%6Cle.txt", "f%69%6Cle.txt"),
-            Arguments.of(GET_METHOD, "path/to/file.txt", "file.txt"),
-            Arguments.of(GET_METHOD, "", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(POST_METHOD, "has/path", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(POST_METHOD, "", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(PUT_METHOD, "has/path", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(PUT_METHOD, "", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(PATCH_METHOD, "", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(PATCH_METHOD, "has/path", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(DELETE_METHOD, "", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(DELETE_METHOD, "has/path", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(HEAD_METHOD, "", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(HEAD_METHOD, "has/path", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(OPTIONS_METHOD, "", FLOW_FILE_INITIAL_FILENAME),
-            Arguments.of(OPTIONS_METHOD, "has/path", FLOW_FILE_INITIAL_FILENAME)
+            Arguments.of(HttpMethod.GET.name(), "file", "file"),
+            Arguments.of(HttpMethod.GET.name(), "file/", "file"),
+            Arguments.of(HttpMethod.GET.name(), "file.txt", "file.txt"),
+            Arguments.of(HttpMethod.GET.name(), "file.txt/", "file.txt"),
+            Arguments.of(HttpMethod.GET.name(), "file.txt/?qp=v", "file.txt"),
+            Arguments.of(HttpMethod.GET.name(), "f%69%6Cle.txt", "f%69%6Cle.txt"),
+            Arguments.of(HttpMethod.GET.name(), "path/to/file.txt", "file.txt"),
+            Arguments.of(HttpMethod.GET.name(), "", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.POST.name(), "has/path", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.POST.name(), "", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.PUT.name(), "has/path", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.PUT.name(), "", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.PATCH.name(), "", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.PATCH.name(), "has/path", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.DELETE.name(), "", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.DELETE.name(), "has/path", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.HEAD.name(), "", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.HEAD.name(), "has/path", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.OPTIONS.name(), "", FLOW_FILE_INITIAL_FILENAME),
+            Arguments.of(HttpMethod.OPTIONS.name(), "has/path", FLOW_FILE_INITIAL_FILENAME)
         );
     }
 
     @Test
     public void testValidWhenOAuth2Set() throws Exception {
-        // GIVEN
         String oauth2AccessTokenProviderId = "oauth2AccessTokenProviderId";
 
         OAuth2AccessTokenProvider oauth2AccessTokenProvider = mock(OAuth2AccessTokenProvider.class, Answers.RETURNS_DEEP_STUBS);
@@ -870,16 +855,13 @@ public class InvokeHTTPTest {
 
         setUrlProperty();
 
-        // WHEN
-        runner.setProperty(InvokeHTTP.OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
+        runner.setProperty(InvokeHTTP.REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
 
-        // THEN
         runner.assertValid();
     }
 
     @Test
     public void testInvalidWhenOAuth2AndUserNameSet() throws Exception {
-        // GIVEN
         String oauth2AccessTokenProviderId = "oauth2AccessTokenProviderId";
 
         OAuth2AccessTokenProvider oauth2AccessTokenProvider = mock(OAuth2AccessTokenProvider.class, Answers.RETURNS_DEEP_STUBS);
@@ -890,17 +872,14 @@ public class InvokeHTTPTest {
 
         setUrlProperty();
 
-        // WHEN
-        runner.setProperty(InvokeHTTP.OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
-        runner.setProperty(InvokeHTTP.PROP_BASIC_AUTH_USERNAME, "userName");
+        runner.setProperty(InvokeHTTP.REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
+        runner.setProperty(InvokeHTTP.REQUEST_USERNAME, "userName");
 
-        // THEN
         runner.assertNotValid();
     }
 
     @Test
     public void testInvalidWhenOAuth2AndPasswordSet() throws Exception {
-        // GIVEN
         String oauth2AccessTokenProviderId = "oauth2AccessTokenProviderId";
 
         OAuth2AccessTokenProvider oauth2AccessTokenProvider = mock(OAuth2AccessTokenProvider.class, Answers.RETURNS_DEEP_STUBS);
@@ -911,17 +890,14 @@ public class InvokeHTTPTest {
 
         setUrlProperty();
 
-        // WHEN
-        runner.setProperty(InvokeHTTP.OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
-        runner.setProperty(InvokeHTTP.PROP_BASIC_AUTH_PASSWORD, "password");
+        runner.setProperty(InvokeHTTP.REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
+        runner.setProperty(InvokeHTTP.REQUEST_PASSWORD, "password");
 
-        // THEN
         runner.assertNotValid();
     }
 
     @Test
     public void testOAuth2AuthorizationHeader() throws Exception {
-        // GIVEN
         String accessToken = "access_token";
 
         String oauth2AccessTokenProviderId = "oauth2AccessTokenProviderId";
@@ -937,21 +913,19 @@ public class InvokeHTTPTest {
 
         mockWebServer.enqueue(new MockResponse());
 
-        // WHEN
-        runner.setProperty(InvokeHTTP.OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
+        runner.setProperty(InvokeHTTP.REQUEST_OAUTH2_ACCESS_TOKEN_PROVIDER, oauth2AccessTokenProviderId);
         runner.enqueue("unimportant");
         runner.run();
 
-        // THEN
         RecordedRequest recordedRequest = mockWebServer.takeRequest();
 
-        String actualAuthorizationHeader = recordedRequest.getHeader("Authorization");
+        String actualAuthorizationHeader = recordedRequest.getHeader(HttpHeader.AUTHORIZATION.getHeader());
         assertEquals("Bearer " + accessToken, actualAuthorizationHeader);
 
     }
 
     private void setUrlProperty() {
-        runner.setProperty(InvokeHTTP.PROP_URL, getMockWebServerUrl());
+        runner.setProperty(InvokeHTTP.HTTP_URL, getMockWebServerUrl());
     }
 
     private String getMockWebServerUrl() {
@@ -972,25 +946,25 @@ public class InvokeHTTPTest {
     }
 
     private MockFlowFile getFailureFlowFile() {
-        return runner.getFlowFilesForRelationship(InvokeHTTP.REL_FAILURE).iterator().next();
+        return runner.getFlowFilesForRelationship(InvokeHTTP.FAILURE).iterator().next();
     }
 
     private MockFlowFile getRequestFlowFile() {
-        return runner.getFlowFilesForRelationship(InvokeHTTP.REL_SUCCESS_REQ).iterator().next();
+        return runner.getFlowFilesForRelationship(InvokeHTTP.ORIGINAL).iterator().next();
     }
 
     private MockFlowFile getResponseFlowFile() {
-        return runner.getFlowFilesForRelationship(InvokeHTTP.REL_RESPONSE).iterator().next();
+        return runner.getFlowFilesForRelationship(InvokeHTTP.RESPONSE).iterator().next();
     }
 
-    private void assertRequestMethodSuccess(final String method) throws InterruptedException {
+    private void assertRequestMethodSuccess(final HttpMethod httpMethod) throws InterruptedException {
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
         final RecordedRequest request = takeRequestCompleted();
-        assertEquals(method, request.getMethod());
+        assertEquals(httpMethod.name(), request.getMethod());
     }
 
     private void assertRelationshipStatusCodeEquals(final Relationship relationship, final int statusCode) {
@@ -1018,11 +992,11 @@ public class InvokeHTTPTest {
             assertFalse(errorMessages.isEmpty(), message);
         }
 
-        runner.assertTransferCount(InvokeHTTP.REL_RESPONSE, 1);
-        runner.assertTransferCount(InvokeHTTP.REL_SUCCESS_REQ, 1);
-        runner.assertTransferCount(InvokeHTTP.REL_RETRY, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_NO_RETRY, 0);
-        runner.assertTransferCount(InvokeHTTP.REL_FAILURE, 0);
+        runner.assertTransferCount(InvokeHTTP.RESPONSE, 1);
+        runner.assertTransferCount(InvokeHTTP.ORIGINAL, 1);
+        runner.assertTransferCount(InvokeHTTP.RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.NO_RETRY, 0);
+        runner.assertTransferCount(InvokeHTTP.FAILURE, 0);
     }
 
     private void assertResponseSuccessSslContextConfigured(final TlsConfiguration serverTlsConfiguration, final TlsConfiguration clientTlsConfiguration) throws InitializationException, TlsException {
@@ -1030,9 +1004,9 @@ public class InvokeHTTPTest {
         enqueueResponseCodeAndRun(HTTP_OK);
 
         assertResponseSuccessRelationships();
-        assertRelationshipStatusCodeEquals(InvokeHTTP.REL_RESPONSE, HTTP_OK);
+        assertRelationshipStatusCodeEquals(InvokeHTTP.RESPONSE, HTTP_OK);
 
-        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(InvokeHTTP.REL_RESPONSE).iterator().next();
+        final MockFlowFile flowFile = runner.getFlowFilesForRelationship(InvokeHTTP.RESPONSE).iterator().next();
         flowFile.assertAttributeExists(InvokeHTTP.REMOTE_DN);
     }
 
@@ -1053,9 +1027,9 @@ public class InvokeHTTPTest {
 
         runner.addControllerService(serviceIdentifier, sslContextService);
         runner.enableControllerService(sslContextService);
-        runner.setProperty(InvokeHTTP.PROP_SSL_CONTEXT_SERVICE, serviceIdentifier);
-        runner.setProperty(InvokeHTTP.PROP_READ_TIMEOUT, TLS_CONNECTION_TIMEOUT);
-        runner.setProperty(InvokeHTTP.PROP_CONNECT_TIMEOUT, TLS_CONNECTION_TIMEOUT);
+        runner.setProperty(InvokeHTTP.SSL_CONTEXT_SERVICE, serviceIdentifier);
+        runner.setProperty(InvokeHTTP.SOCKET_READ_TIMEOUT, TLS_CONNECTION_TIMEOUT);
+        runner.setProperty(InvokeHTTP.SOCKET_CONNECT_TIMEOUT, TLS_CONNECTION_TIMEOUT);
         return sslContextService;
     }