You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by bn...@apache.org on 2020/09/01 15:33:09 UTC

[trafficserver] branch master updated: URL::parse fixes for empty paths (#7119)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 4e37533  URL::parse fixes for empty paths (#7119)
4e37533 is described below

commit 4e375331c9bc0e9e3955fd15f11b8c352ab2afb0
Author: Brian Neradt <br...@gmail.com>
AuthorDate: Tue Sep 1 10:32:58 2020 -0500

    URL::parse fixes for empty paths (#7119)
    
    URL::parse could not handle URLs with empty paths, such as the
    following:
    
    http://www.example.com?name=something
    
    Note the lack of '/' after the hostname.
    
    This updates the parsing and printing logic to be able to handle this
    while being careful not to break current parse expectations.
---
 .../api/functions/TSUrlHostGet.en.rst              |   2 +-
 include/ts/ts.h                                    |   4 +-
 plugins/header_rewrite/operators.cc                |   4 +-
 proxy/hdrs/URL.cc                                  | 108 ++++--
 proxy/hdrs/URL.h                                   | 102 +++++-
 proxy/hdrs/unit_tests/test_URL.cc                  | 373 +++++++++++++++++++++
 proxy/http/remap/RemapConfig.cc                    |  10 +-
 proxy/http2/unit_tests/test_HTTP2.cc               |  10 +-
 tests/gold_tests/autest-site/microDNS.test.ext     |   2 +-
 .../header_rewrite/gold/set-redirect.gold          |   8 +
 ...e_client.test.py => header_rewrite_url.test.py} |  42 ++-
 .../header_rewrite/rules/set_redirect.conf         |  18 +
 tests/gold_tests/remap/gold/remap-zero-200.gold    |   7 +
 tests/gold_tests/remap/regex_map.test.py           |  62 ++++
 14 files changed, 690 insertions(+), 62 deletions(-)

diff --git a/doc/developer-guide/api/functions/TSUrlHostGet.en.rst b/doc/developer-guide/api/functions/TSUrlHostGet.en.rst
index 3db194c..bb07176 100644
--- a/doc/developer-guide/api/functions/TSUrlHostGet.en.rst
+++ b/doc/developer-guide/api/functions/TSUrlHostGet.en.rst
@@ -51,7 +51,7 @@ and retrieve or modify parts of URLs, such as their host, port or scheme
 information.
 
 :func:`TSUrlSchemeGet`, :func:`TSUrlUserGet`, :func:`TSUrlPasswordGet`,
-:func:`TSUrlHostGet`, :func:`TSUrlHttpParamsGet`, :func:`TSUrlHttpQueryGet`
+:func:`TSUrlHostGet`, :func:`TSUrlPathGet`, :func:`TSUrlHttpParamsGet`, :func:`TSUrlHttpQueryGet`
 and :func:`TSUrlHttpFragmentGet` each retrieve an internal pointer to the
 specified portion of the URL from the marshall buffer :arg:`bufp`. The length
 of the returned string is placed in :arg:`length` and a pointer to the URL
diff --git a/include/ts/ts.h b/include/ts/ts.h
index 86077ad..2df875b 100644
--- a/include/ts/ts.h
+++ b/include/ts/ts.h
@@ -398,8 +398,8 @@ tsapi int TSUrlLengthGet(TSMBuffer bufp, TSMLoc offset);
     string in the parameter length. This is the same length that
     TSUrlLengthGet() returns. The returned string is allocated by a
     call to TSmalloc(). It should be freed by a call to TSfree().
-    The length parameter must present, providing storage for the URL
-    string length value.
+    The length parameter must be present, providing storage for the
+    URL string length value.
     Note: To get the effective URL from a request, use the alternative
           TSHttpTxnEffectiveUrlStringGet or
           TSHttpHdrEffectiveUrlBufGet APIs.
diff --git a/plugins/header_rewrite/operators.cc b/plugins/header_rewrite/operators.cc
index 4c58f2f..f0cd8b4 100644
--- a/plugins/header_rewrite/operators.cc
+++ b/plugins/header_rewrite/operators.cc
@@ -406,7 +406,9 @@ OperatorSetRedirect::exec(const Resources &res) const
     const char *end   = value.size() + start;
     if (remap) {
       // Set new location.
-      TSUrlParse(bufp, url_loc, &start, end);
+      if (TS_PARSE_ERROR == TSUrlParse(bufp, url_loc, &start, end)) {
+        TSDebug(PLUGIN_NAME, "Could not set Location field value to: %s", value.c_str());
+      }
       // Set the new status.
       TSHttpTxnStatusSet(res.txnp, static_cast<TSHttpStatus>(_status.get_int_value()));
       const_cast<Resources &>(res).changed_url = true;
diff --git a/proxy/hdrs/URL.cc b/proxy/hdrs/URL.cc
index c528ee8..c2a903b 100644
--- a/proxy/hdrs/URL.cc
+++ b/proxy/hdrs/URL.cc
@@ -290,10 +290,11 @@ url_copy_onto_as_server_url(URLImpl *s_url, HdrHeap *s_heap, URLImpl *d_url, Hdr
 {
   url_nuke_proxy_stuff(d_url);
 
-  d_url->m_ptr_path     = s_url->m_ptr_path;
-  d_url->m_ptr_params   = s_url->m_ptr_params;
-  d_url->m_ptr_query    = s_url->m_ptr_query;
-  d_url->m_ptr_fragment = s_url->m_ptr_fragment;
+  d_url->m_ptr_path      = s_url->m_ptr_path;
+  d_url->m_path_is_empty = s_url->m_path_is_empty;
+  d_url->m_ptr_params    = s_url->m_ptr_params;
+  d_url->m_ptr_query     = s_url->m_ptr_query;
+  d_url->m_ptr_fragment  = s_url->m_ptr_fragment;
   url_clear_string_ref(d_url);
 
   d_url->m_len_path     = s_url->m_len_path;
@@ -827,9 +828,13 @@ url_length_get(URLImpl *url)
   }
 
   if (url->m_ptr_path) {
-    length += url->m_len_path + 1; // +1 for /
-  } else {
-    length += 1; // +1 for /
+    length += url->m_len_path;
+  }
+
+  if (!url->m_path_is_empty) {
+    // m_ptr_path does not contain the initial "/" and thus m_len_path does not
+    // count it. We account for it here.
+    length += 1; // +1 for "/"
   }
 
   if (url->m_ptr_params && url->m_len_params > 0) {
@@ -1185,26 +1190,30 @@ url_is_strictly_compliant(const char *start, const char *end)
 using namespace UrlImpl;
 
 ParseResult
-url_parse(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings_p, bool strict_uri_parsing)
+url_parse(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings_p, bool strict_uri_parsing,
+          bool verify_host_characters)
 {
   if (strict_uri_parsing && !url_is_strictly_compliant(*start, end)) {
     return PARSE_RESULT_ERROR;
   }
 
   ParseResult zret = url_parse_scheme(heap, url, start, end, copy_strings_p);
-  return PARSE_RESULT_CONT == zret ? url_parse_http(heap, url, start, end, copy_strings_p) : zret;
+  return PARSE_RESULT_CONT == zret ? url_parse_http(heap, url, start, end, copy_strings_p, verify_host_characters) : zret;
 }
 
 ParseResult
-url_parse_no_path_component_breakdown(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings_p)
+url_parse_regex(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings_p)
 {
   ParseResult zret = url_parse_scheme(heap, url, start, end, copy_strings_p);
-  return PARSE_RESULT_CONT == zret ? url_parse_http_no_path_component_breakdown(heap, url, start, end, copy_strings_p) : zret;
+  return PARSE_RESULT_CONT == zret ? url_parse_http_regex(heap, url, start, end, copy_strings_p) : zret;
 }
 
 /**
   Parse internet URL.
 
+  After this function completes, start will point to the first character after the
+  host or @a end if there are not characters after it.
+
   @verbatim
   [://][user[:password]@]host[:port]
 
@@ -1219,7 +1228,8 @@ url_parse_no_path_component_breakdown(HdrHeap *heap, URLImpl *url, const char **
 */
 
 ParseResult
-url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, char const *end, bool copy_strings_p)
+url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, char const *end, bool copy_strings_p,
+                   bool verify_host_characters)
 {
   const char *cur = *start;
   const char *base;              // Base for host/port field.
@@ -1296,8 +1306,14 @@ url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, char const *
       bracket = cur; // location and flag.
       ++cur;
       break;
-    case '/':    // we're done with this phase.
-      end = cur; // cause loop exit
+    // RFC 3986, section 3.2:
+    // The authority component is ...  terminated by the next slash ("/"),
+    // question mark ("?"), or number sign ("#") character, or by the end of
+    // the URI.
+    case '/':
+    case '?':
+    case '#':
+      end = cur; // We're done parsing authority, cause loop exit.
       break;
     default:
       ++cur;
@@ -1324,7 +1340,7 @@ url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, char const *
     }
   }
   if (host._size) {
-    if (validate_host_name(std::string_view(host._ptr, host._size))) {
+    if (!verify_host_characters || validate_host_name(std::string_view(host._ptr, host._size))) {
       url_host_set(heap, url, host._ptr, host._size, copy_strings_p);
     } else {
       return PARSE_RESULT_ERROR;
@@ -1339,9 +1355,6 @@ url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, char const *
     }
     url_port_set(heap, url, port._ptr, port._size, copy_strings_p);
   }
-  if ('/' == *cur) {
-    ++cur; // must do this after filling in host/port.
-  }
   *start = cur;
   return PARSE_RESULT_DONE;
 }
@@ -1352,7 +1365,7 @@ url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, char const *
 // empties params/query/fragment component
 
 ParseResult
-url_parse_http(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings)
+url_parse_http(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings, bool verify_host_characters)
 {
   ParseResult err;
   const char *cur;
@@ -1366,18 +1379,28 @@ url_parse_http(HdrHeap *heap, URLImpl *url, const char **start, const char *end,
   const char *fragment_end   = nullptr;
   char mask;
 
-  err = url_parse_internet(heap, url, start, end, copy_strings);
+  err = url_parse_internet(heap, url, start, end, copy_strings, verify_host_characters);
   if (err < 0) {
     return err;
   }
 
   cur = *start;
+  if (url->m_ptr_host == nullptr && ((end - cur) >= 2) && '/' == *cur && '/' == *(cur + 1)) {
+    // RFC 3986 section-3.3:
+    // If a URI does not contain an authority component, then the path cannot
+    // begin with two slash characters ("//").
+    return PARSE_RESULT_ERROR;
+  }
+  bool nothing_after_host = false;
   if (*start == end) {
+    nothing_after_host = true;
     goto done;
   }
 
-  path_start = cur;
-  mask       = ';' & '?' & '#';
+  if (*cur == '/') {
+    path_start = cur;
+  }
+  mask = ';' & '?' & '#';
 parse_path2:
   if ((*cur & mask) == mask) {
     if (*cur == ';') {
@@ -1431,10 +1454,42 @@ parse_fragment1:
 
 done:
   if (path_start) {
+    // There was an explicit path set with '/'.
     if (!path_end) {
       path_end = cur;
     }
+    if (path_start == path_end) {
+      url->m_path_is_empty = true;
+    } else {
+      url->m_path_is_empty = false;
+      // Per RFC 3986 section 3, the query string does not contain the initial
+      // '?' nor does the fragment contain the initial '#'. The path however
+      // does contain the initial '/' and a path can be empty, containing no
+      // characters at all, not even the initial '/'. Our path_get interface,
+      // however, has long not behaved accordingly, returning only the
+      // characters after the first '/'. This does not allow users to tell
+      // whether the path was absolutely empty. Further, callers have to
+      // account for the missing first '/' character themselves, either in URL
+      // length calculations or when piecing together their own URL. There are
+      // various examples of this in core and in the plugins shipped with Traffic
+      // Server.
+      //
+      // Correcting this behavior by having path_get return the entire path,
+      // (inclusive of any first '/') and an empty string if there were no
+      // characters specified in the path would break existing functionality,
+      // including various plugins that expect this behavior. Rather than
+      // correcting this behavior, therefore, we maintain the current
+      // functionality but add state to determine whether the path was
+      // absolutely empty so we can reconstruct such URLs.
+      ++path_start;
+    }
     url_path_set(heap, url, path_start, path_end - path_start, copy_strings);
+  } else if (!nothing_after_host) {
+    // There was no path set via '/': it is absolutely empty. However, if there
+    // is no path, query, or fragment after the host, we by convention add a
+    // slash after the authority.  Users of URL expect this behavior. Thus the
+    // nothing_after_host check.
+    url->m_path_is_empty = true;
   }
   if (params_start) {
     if (!params_end) {
@@ -1443,12 +1498,14 @@ done:
     url_params_set(heap, url, params_start, params_end - params_start, copy_strings);
   }
   if (query_start) {
+    // There was a query string marked by '?'.
     if (!query_end) {
       query_end = cur;
     }
     url_query_set(heap, url, query_start, query_end - query_start, copy_strings);
   }
   if (fragment_start) {
+    // There was a fragment string marked by '#'.
     if (!fragment_end) {
       fragment_end = cur;
     }
@@ -1460,7 +1517,7 @@ done:
 }
 
 ParseResult
-url_parse_http_no_path_component_breakdown(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings)
+url_parse_http_regex(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings)
 {
   const char *cur = *start;
   const char *host_end;
@@ -1572,8 +1629,9 @@ url_print(URLImpl *url, char *buf_start, int buf_length, int *buf_index_inout, i
     }
   }
 
-  TRY(mime_mem_print("/", 1, buf_start, buf_length, buf_index_inout, buf_chars_to_skip_inout));
-
+  if (!url->m_path_is_empty) {
+    TRY(mime_mem_print("/", 1, buf_start, buf_length, buf_index_inout, buf_chars_to_skip_inout));
+  }
   if (url->m_ptr_path) {
     TRY(mime_mem_print(url->m_ptr_path, url->m_len_path, buf_start, buf_length, buf_index_inout, buf_chars_to_skip_inout));
   }
diff --git a/proxy/hdrs/URL.h b/proxy/hdrs/URL.h
index 36f52aa..ab8b36c 100644
--- a/proxy/hdrs/URL.h
+++ b/proxy/hdrs/URL.h
@@ -74,7 +74,9 @@ struct URLImpl : public HdrHeapObjImpl {
   // 6 bytes
 
   uint32_t m_clean : 1;
-  // 8 bytes + 1 bit, will result in padding
+  /// Whether the URI had an absolutely empty path, not even an initial '/'.
+  uint32_t m_path_is_empty : 1;
+  // 8 bytes + 2 bits, will result in padding
 
   // Marshaling Functions
   int marshal(MarshalXlate *str_xlate, int num_xlate);
@@ -194,14 +196,19 @@ void url_params_set(HdrHeap *heap, URLImpl *url, const char *value, int length,
 void url_query_set(HdrHeap *heap, URLImpl *url, const char *value, int length, bool copy_string);
 void url_fragment_set(HdrHeap *heap, URLImpl *url, const char *value, int length, bool copy_string);
 
+constexpr bool USE_STRICT_URI_PARSING = true;
+
 ParseResult url_parse(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings,
-                      bool strict_uri_parsing = false);
-ParseResult url_parse_no_path_component_breakdown(HdrHeap *heap, URLImpl *url, const char **start, const char *end,
-                                                  bool copy_strings);
-ParseResult url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings);
-ParseResult url_parse_http(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings);
-ParseResult url_parse_http_no_path_component_breakdown(HdrHeap *heap, URLImpl *url, const char **start, const char *end,
-                                                       bool copy_strings);
+                      bool strict_uri_parsing = false, bool verify_host_characters = true);
+
+constexpr bool COPY_STRINGS = true;
+
+ParseResult url_parse_regex(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings);
+ParseResult url_parse_internet(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings,
+                               bool verify_host_characters);
+ParseResult url_parse_http(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings,
+                           bool verify_host_characters);
+ParseResult url_parse_http_regex(HdrHeap *heap, URLImpl *url, const char **start, const char *end, bool copy_strings);
 
 char *url_unescapify(Arena *arena, const char *str, int length);
 
@@ -279,15 +286,48 @@ public:
   const char *fragment_get(int *length);
   void fragment_set(const char *value, int length);
 
+  /**
+   * Parse the given URL string and populate URL state with the parts.
+   *
+   * @param[in] url The URL to parse.
+   *
+   * @return PARSE_RESULT_DONE if parsing was successful, PARSE_RESULT_ERROR
+   * otherwise.
+   */
+  ParseResult parse(std::string_view url);
+
+  /** Same as parse() but do not verify that the host has proper FQDN
+   * characters.
+   *
+   * This is useful for RemapConfig To targets which have "$[0-9]" references
+   * in their host names which will later be substituted for other text.
+   */
+  ParseResult parse_no_host_check(std::string_view url);
+
   ParseResult parse(const char **start, const char *end);
   ParseResult parse(const char *str, int length);
-  ParseResult parse_no_path_component_breakdown(const char *str, int length);
+
+  /** Perform more simplified parsing that is resilient to receiving regular
+   * expressions.
+   *
+   * This simply looks for the first '/' in a URL and considers that the end of
+   * the authority and the beginning of the rest of the URL. This allows for
+   * the '?' character in an authority as a part of a regex without it being
+   * considered a query parameter and, thus, avoids confusing the parser.
+   *
+   * This is only used in RemapConfig and may have no other uses.
+   */
+  ParseResult parse_regex(std::string_view url);
+  ParseResult parse_regex(const char *str, int length);
 
 public:
   static char *unescapify(Arena *arena, const char *str, int length);
   // No gratuitous copies!
   URL(const URL &u) = delete;
   URL &operator=(const URL &u) = delete;
+
+private:
+  static constexpr bool VERIFY_HOST_CHARACTERS = true;
 };
 
 /*-------------------------------------------------------------------------
@@ -690,10 +730,35 @@ URL::fragment_set(const char *value, int length)
 
  */
 inline ParseResult
+URL::parse(std::string_view url)
+{
+  return this->parse(url.data(), static_cast<int>(url.size()));
+}
+
+/**
+  Parser doesn't clear URL first, so if you parse over a non-clear URL,
+  the resulting URL may contain some of the previous data.
+
+ */
+inline ParseResult
+URL::parse_no_host_check(std::string_view url)
+{
+  ink_assert(valid());
+  const char *start = url.data();
+  const char *end   = url.data() + url.length();
+  return url_parse(m_heap, m_url_impl, &start, end, COPY_STRINGS, !USE_STRICT_URI_PARSING, !VERIFY_HOST_CHARACTERS);
+}
+
+/**
+  Parser doesn't clear URL first, so if you parse over a non-clear URL,
+  the resulting URL may contain some of the previous data.
+
+ */
+inline ParseResult
 URL::parse(const char **start, const char *end)
 {
   ink_assert(valid());
-  return url_parse(m_heap, m_url_impl, start, end, true);
+  return url_parse(m_heap, m_url_impl, start, end, COPY_STRINGS);
 }
 
 /**
@@ -716,13 +781,26 @@ URL::parse(const char *str, int length)
 
  */
 inline ParseResult
-URL::parse_no_path_component_breakdown(const char *str, int length)
+URL::parse_regex(std::string_view url)
+{
+  ink_assert(valid());
+  const char *str = url.data();
+  return url_parse_regex(m_heap, m_url_impl, &str, str + url.length(), COPY_STRINGS);
+}
+
+/**
+  Parser doesn't clear URL first, so if you parse over a non-clear URL,
+  the resulting URL may contain some of the previous data.
+
+ */
+inline ParseResult
+URL::parse_regex(const char *str, int length)
 {
   ink_assert(valid());
   if (length < 0)
     length = (int)strlen(str);
   ink_assert(valid());
-  return url_parse_no_path_component_breakdown(m_heap, m_url_impl, &str, str + length, true);
+  return url_parse_regex(m_heap, m_url_impl, &str, str + length, COPY_STRINGS);
 }
 
 /*-------------------------------------------------------------------------
diff --git a/proxy/hdrs/unit_tests/test_URL.cc b/proxy/hdrs/unit_tests/test_URL.cc
index 9ebcd9d..975bd5e 100644
--- a/proxy/hdrs/unit_tests/test_URL.cc
+++ b/proxy/hdrs/unit_tests/test_URL.cc
@@ -93,3 +93,376 @@ TEST_CASE("ParseRulesStrictURI", "[proxy][parseuri]")
     }
   }
 }
+
+struct url_parse_test_case {
+  const std::string input_uri;
+  const std::string expected_printed_url;
+  const bool verify_host_characters;
+  const std::string expected_printed_url_regex;
+  const bool is_valid;
+  const bool is_valid_regex;
+};
+
+constexpr bool IS_VALID               = true;
+constexpr bool VERIFY_HOST_CHARACTERS = true;
+
+// clang-format off
+std::vector<url_parse_test_case> url_parse_test_cases = {
+  {
+    // The following scheme-only URI is technically valid per the spec, but we
+    // have historically returned this as invalid and I'm not comfortable
+    // changing it in case something depends upon this behavior. Besides, a
+    // scheme-only URI is probably not helpful to us nor something likely
+    // Traffic Server will see.
+    "http://",
+    "",
+    VERIFY_HOST_CHARACTERS,
+    "",
+    !IS_VALID,
+    !IS_VALID
+  },
+  {
+    "https:///",
+    "https:///",
+    VERIFY_HOST_CHARACTERS,
+    "https:///",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    // RFC 3986 section-3: When authority is not present, the path cannot begin
+    // with two slash characters ("//"). The parse_regex, though, is more
+    // forgiving.
+    "https:////",
+    "",
+    VERIFY_HOST_CHARACTERS,
+    "https:////",
+    !IS_VALID,
+    IS_VALID
+  },
+  {
+    // By convention, our url_print() function adds a path of '/' at the end of
+    // URLs that have no path, query, or fragment after the authority.
+    "mailto:Test.User@example.com",
+    "mailto:Test.User@example.com/",
+    VERIFY_HOST_CHARACTERS,
+    "mailto:Test.User@example.com/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "mailto:Test.User@example.com:25",
+    "mailto:Test.User@example.com:25/",
+    VERIFY_HOST_CHARACTERS,
+    "mailto:Test.User@example.com:25/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com",
+    "https://www.example.com/",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/",
+    "https://www.example.com/",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com//",
+    "https://www.example.com//",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com//",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://127.0.0.1",
+    "https://127.0.0.1/",
+    VERIFY_HOST_CHARACTERS,
+    "https://127.0.0.1/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://[::1]",
+    "https://[::1]/",
+    VERIFY_HOST_CHARACTERS,
+    "https://[::1]/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://127.0.0.1/",
+    "https://127.0.0.1/",
+    VERIFY_HOST_CHARACTERS,
+    "https://127.0.0.1/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com:8888",
+    "https://www.example.com:8888/",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com:8888/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com:8888/",
+    "https://www.example.com:8888/",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com:8888/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/a/path",
+    "https://www.example.com/a/path",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com//a/path",
+    "https://www.example.com//a/path",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com//a/path",
+    IS_VALID,
+    IS_VALID
+  },
+
+  // Technically a trailing '?' with an empty query string is valid, but we
+  // drop the '?'. The parse_regex, however, makes no distinction between
+  // query, fragment, and path components so it does not cut it out.
+  {
+    "https://www.example.com/a/path?",
+    "https://www.example.com/a/path",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path?",
+    IS_VALID,
+    IS_VALID},
+  {
+    "https://www.example.com/a/path?name=value",
+    "https://www.example.com/a/path?name=value",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path?name=value",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/a/path?name=/a/path/value",
+    "https://www.example.com/a/path?name=/a/path/value",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path?name=/a/path/value",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/a/path?name=/a/path/value;some=other_value",
+    "https://www.example.com/a/path?name=/a/path/value;some=other_value",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path?name=/a/path/value;some=other_value",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/a/path?name=/a/path/value;some=other_value/",
+    "https://www.example.com/a/path?name=/a/path/value;some=other_value/",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path?name=/a/path/value;some=other_value/",
+    IS_VALID,
+    IS_VALID
+  },
+
+  // Again, URL::parse drops a final '?'.
+  {
+    "https://www.example.com?",
+    "https://www.example.com",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com?/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com?name=value",
+    "https://www.example.com?name=value",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com?name=value/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com?name=value/",
+    "https://www.example.com?name=value/",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com?name=value/",
+    IS_VALID,
+    IS_VALID
+  },
+
+  // URL::parse also drops the final '#'.
+  {
+    "https://www.example.com#",
+    "https://www.example.com",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com#/",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com#some=value",
+    "https://www.example.com#some=value",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com#some=value/",
+    IS_VALID,
+    IS_VALID},
+  {
+    "https://www.example.com/a/path#",
+    "https://www.example.com/a/path",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path#",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/a/path#some=value",
+    "https://www.example.com/a/path#some=value",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path#some=value",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    // Note that this final '?' is not for a query parameter but is a part of
+    // the fragment.
+    "https://www.example.com/a/path#some=value?",
+    "https://www.example.com/a/path#some=value?",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path#some=value?",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/a/path#some=value?with_question",
+    "https://www.example.com/a/path#some=value?with_question",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path#some=value?with_question",
+    IS_VALID,
+    IS_VALID
+  },
+  {
+    "https://www.example.com/a/path?name=value?_with_question#some=value?with_question/",
+    "https://www.example.com/a/path?name=value?_with_question#some=value?with_question/",
+    VERIFY_HOST_CHARACTERS,
+    "https://www.example.com/a/path?name=value?_with_question#some=value?with_question/",
+    IS_VALID,
+    IS_VALID
+  },
+
+  // The following are some examples of strings we expect from regex_map in
+  // remap.config.  The "From" portion, which are regular expressions, are
+  // often not parsible by URL::parse but are by URL::parse_regex, which is the
+  // purpose of its existence.
+  {
+    R"(http://(.*)?reactivate\.mail\.yahoo\.com/)",
+    "",
+    VERIFY_HOST_CHARACTERS,
+    R"(http://(.*)?reactivate\.mail\.yahoo\.com/)",
+    !IS_VALID,
+    IS_VALID
+  },
+  {
+    // The following is an example of a "To" URL in a regex_map line. We'll
+    // first verify that the '$' is flagged as invalid for a host in this case.
+    "http://$1reactivate.real.mail.yahoo.com/",
+    "http://$1reactivate.real.mail.yahoo.com/",
+    VERIFY_HOST_CHARACTERS,
+    "http://$1reactivate.real.mail.yahoo.com/",
+    !IS_VALID,
+    IS_VALID
+  },
+  {
+    // Same as above, but this time we pass in !VERIFY_HOST_CHARACTERS. This is
+    // how RemapConfig will call this parse() function.
+    "http://$1reactivate.real.mail.yahoo.com/",
+    "http://$1reactivate.real.mail.yahoo.com/",
+    !VERIFY_HOST_CHARACTERS,
+    "http://$1reactivate.real.mail.yahoo.com/",
+    IS_VALID,
+    IS_VALID
+  }
+};
+// clang-format on
+
+constexpr bool URL_PARSE       = true;
+constexpr bool URL_PARSE_REGEX = false;
+
+/** Test the specified url.parse function.
+ *
+ * URL::parse and URL::parse_regex should behave the same. This function
+ * performs the same behavior for each.
+ *
+ * @param[in] test_case The test case specification to run.
+ *
+ * @param[in] parse_function Whether to run parse() or
+ * parse_regex().
+ */
+void
+test_parse(url_parse_test_case const &test_case, bool parse_function)
+{
+  URL url;
+  HdrHeap *heap = new_HdrHeap();
+  url.create(heap);
+  ParseResult result = PARSE_RESULT_OK;
+  if (parse_function == URL_PARSE) {
+    if (test_case.verify_host_characters) {
+      result = url.parse(test_case.input_uri);
+    } else {
+      result = url.parse_no_host_check(test_case.input_uri);
+    }
+  } else if (parse_function == URL_PARSE_REGEX) {
+    result = url.parse_regex(test_case.input_uri);
+  }
+  bool expected_is_valid = test_case.is_valid;
+  if (parse_function == URL_PARSE_REGEX) {
+    expected_is_valid = test_case.is_valid_regex;
+  }
+  if (expected_is_valid && result != PARSE_RESULT_DONE) {
+    std::printf("Parse URI: \"%s\", expected it to be valid but it was parsed invalid (%d)\n", test_case.input_uri.c_str(), result);
+    CHECK(false);
+  } else if (!expected_is_valid && result != PARSE_RESULT_ERROR) {
+    std::printf("Parse URI: \"%s\", expected it to be invalid but it was parsed valid (%d)\n", test_case.input_uri.c_str(), result);
+    CHECK(false);
+  }
+  if (result == PARSE_RESULT_DONE) {
+    char buf[1024];
+    int index  = 0;
+    int offset = 0;
+    url.print(buf, sizeof(buf), &index, &offset);
+    std::string printed_url{buf, static_cast<size_t>(index)};
+    if (parse_function == URL_PARSE) {
+      CHECK(test_case.expected_printed_url == printed_url);
+      CHECK(test_case.expected_printed_url.size() == printed_url.size());
+    } else if (parse_function == URL_PARSE_REGEX) {
+      CHECK(test_case.expected_printed_url_regex == printed_url);
+      CHECK(test_case.expected_printed_url_regex.size() == printed_url.size());
+    }
+  }
+  heap->destroy();
+}
+
+TEST_CASE("UrlParse", "[proxy][parseurl]")
+{
+  for (auto const &test_case : url_parse_test_cases) {
+    test_parse(test_case, URL_PARSE);
+    test_parse(test_case, URL_PARSE_REGEX);
+  }
+}
diff --git a/proxy/http/remap/RemapConfig.cc b/proxy/http/remap/RemapConfig.cc
index 2e83167..c9d06e9 100644
--- a/proxy/http/remap/RemapConfig.cc
+++ b/proxy/http/remap/RemapConfig.cc
@@ -1087,12 +1087,13 @@ remap_parse_config_bti(const char *path, BUILD_TABLE_INFO *bti)
     }
 
     new_mapping->fromURL.create(nullptr);
-    rparse = new_mapping->fromURL.parse_no_path_component_breakdown(tmp, length);
+    rparse = new_mapping->fromURL.parse_regex(tmp, length);
 
     map_from_start[origLength] = '\0'; // Unwhack
 
     if (rparse != PARSE_RESULT_DONE) {
-      errStr = "malformed From URL";
+      snprintf(errStrBuf, sizeof(errStrBuf), "malformed From URL: %.*s", length, tmp);
+      errStr = errStrBuf;
       goto MAP_ERROR;
     }
 
@@ -1102,11 +1103,12 @@ remap_parse_config_bti(const char *path, BUILD_TABLE_INFO *bti)
     tmp          = map_to;
 
     new_mapping->toURL.create(nullptr);
-    rparse                   = new_mapping->toURL.parse_no_path_component_breakdown(tmp, length);
+    rparse                   = new_mapping->toURL.parse_no_host_check(std::string_view(tmp, length));
     map_to_start[origLength] = '\0'; // Unwhack
 
     if (rparse != PARSE_RESULT_DONE) {
-      errStr = "malformed To URL";
+      snprintf(errStrBuf, sizeof(errStrBuf), "malformed To URL: %.*s", length, tmp);
+      errStr = errStrBuf;
       goto MAP_ERROR;
     }
 
diff --git a/proxy/http2/unit_tests/test_HTTP2.cc b/proxy/http2/unit_tests/test_HTTP2.cc
index e67f50e..21a772a 100644
--- a/proxy/http2/unit_tests/test_HTTP2.cc
+++ b/proxy/http2/unit_tests/test_HTTP2.cc
@@ -64,7 +64,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
       MIMEField *f = hdr_1.field_find(HTTP2_VALUE_METHOD, HTTP2_LEN_METHOD);
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
-      CHECK(v.compare("GET") == 0);
+      CHECK(v == "GET");
     }
 
     // :scheme
@@ -72,7 +72,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
       MIMEField *f = hdr_1.field_find(HTTP2_VALUE_SCHEME, HTTP2_LEN_SCHEME);
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
-      CHECK(v.compare("https") == 0);
+      CHECK(v == "https");
     }
 
     // :authority
@@ -80,7 +80,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
       MIMEField *f = hdr_1.field_find(HTTP2_VALUE_AUTHORITY, HTTP2_LEN_AUTHORITY);
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
-      CHECK(v.compare("trafficserver.apache.org") == 0);
+      CHECK(v == "trafficserver.apache.org");
     }
 
     // :path
@@ -88,7 +88,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
       MIMEField *f = hdr_1.field_find(HTTP2_VALUE_PATH, HTTP2_LEN_PATH);
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
-      CHECK(v.compare("/index.html") == 0);
+      CHECK(v == "/index.html");
     }
 
     // convert to HTTP/1.1
@@ -138,7 +138,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
       MIMEField *f = hdr_1.field_find(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS);
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
-      CHECK(v.compare("200") == 0);
+      CHECK(v == "200");
     }
 
     // no connection header
diff --git a/tests/gold_tests/autest-site/microDNS.test.ext b/tests/gold_tests/autest-site/microDNS.test.ext
index 83fbc7b..e508f5d 100644
--- a/tests/gold_tests/autest-site/microDNS.test.ext
+++ b/tests/gold_tests/autest-site/microDNS.test.ext
@@ -82,7 +82,7 @@ def MakeDNServer(obj, name, filename="dns_file.json", port=False, ip='INADDR_LOO
         jsondata = {'mappings': []}
 
         if default:
-            jsondata['otherwise'] = default
+            jsondata['otherwise'] = [default]
 
         with open(filepath, 'w') as f:
             f.write(json.dumps(jsondata))
diff --git a/tests/gold_tests/pluginTest/header_rewrite/gold/set-redirect.gold b/tests/gold_tests/pluginTest/header_rewrite/gold/set-redirect.gold
new file mode 100644
index 0000000..bd76250
--- /dev/null
+++ b/tests/gold_tests/pluginTest/header_rewrite/gold/set-redirect.gold
@@ -0,0 +1,8 @@
+``
+> HEAD / HTTP/1.1
+> Host: no_path.com
+``
+< HTTP/1.1 301 Redirect
+``
+< Location: http://no_path.com?name=brian/
+``
diff --git a/tests/gold_tests/pluginTest/header_rewrite/header_rewrite_client.test.py b/tests/gold_tests/pluginTest/header_rewrite/header_rewrite_url.test.py
similarity index 58%
rename from tests/gold_tests/pluginTest/header_rewrite/header_rewrite_client.test.py
rename to tests/gold_tests/pluginTest/header_rewrite/header_rewrite_url.test.py
index 0ab0f88..8f84271 100644
--- a/tests/gold_tests/pluginTest/header_rewrite/header_rewrite_client.test.py
+++ b/tests/gold_tests/pluginTest/header_rewrite/header_rewrite_url.test.py
@@ -1,4 +1,5 @@
 '''
+Test header_rewrite with URL conditions and operators.
 '''
 #  Licensed to the Apache Software Foundation (ASF) under one
 #  or more contributor license agreements.  See the NOTICE file
@@ -17,44 +18,63 @@
 #  limitations under the License.
 
 Test.Summary = '''
-Test header_rewrite and CLIENT-URL
+Test header_rewrite with URL conditions and operators.
 '''
 
 Test.ContinueOnFail = True
-# Define default ATS
 ts = Test.MakeATSProcess("ts")
 server = Test.MakeOriginServer("server")
 
+# Configure the server to return 200 responses. The rewrite rules below set a
+# non-200 status, so if curl gets a 200 response something went wrong.
 Test.testName = ""
 request_header = {"headers": "GET / HTTP/1.1\r\nHost: www.example.com\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
-# expected response from the origin server
 response_header = {"headers": "HTTP/1.1 200 OK\r\nConnection: close\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
-
-# add response to the server dictionary
 server.addResponse("sessionfile.log", request_header, response_header)
+request_header = {"headers": "GET / HTTP/1.1\r\nHost: no_path.com\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
+server.addResponse("sessionfile.log", request_header, response_header)
+
 ts.Disk.records_config.update({
     'proxy.config.diags.debug.enabled': 1,
     'proxy.config.diags.debug.tags': 'header.*',
 })
 # The following rule changes the status code returned from origin server to 303
 ts.Setup.CopyAs('rules/rule_client.conf', Test.RunDirectory)
+ts.Setup.CopyAs('rules/set_redirect.conf', Test.RunDirectory)
 
+# This configuration makes use of CLIENT-URL in conditions.
 ts.Disk.remap_config.AddLine(
-    'map http://www.example.com/from_path/ https://127.0.0.1:{0}/to_path/ @plugin=header_rewrite.so @pparam={1}/rule_client.conf'.format(
+    'map http://www.example.com/from_path/ https://127.0.0.1:{0}/to_path/ '
+    '@plugin=header_rewrite.so @pparam={1}/rule_client.conf'.format(
         server.Variables.Port, Test.RunDirectory))
 ts.Disk.remap_config.AddLine(
-    'map http://www.example.com:8080/from_path/ https://127.0.0.1:{0}/to_path/ @plugin=header_rewrite.so @pparam={1}/rule_client.conf'.format(
+    'map http://www.example.com:8080/from_path/ https://127.0.0.1:{0}/to_path/ '
+    '@plugin=header_rewrite.so @pparam={1}/rule_client.conf'.format(
         server.Variables.Port, Test.RunDirectory))
+# This configuration makes use of TO-URL in a set-redirect operator.
+ts.Disk.remap_config.AddLine(
+    'map http://no_path.com http://no_path.com?name=brian/ '
+    '@plugin=header_rewrite.so @pparam={0}/set_redirect.conf'.format(
+        Test.RunDirectory))
 
-# call localhost straight
+# Test CLIENT-URL.
 tr = Test.AddTestRun()
-tr.Processes.Default.Command = 'curl --proxy 127.0.0.1:{0} "http://www.example.com/from_path/hello?=foo=bar" -H "Proxy-Connection: keep-alive" --verbose'.format(
-    ts.Variables.port)
+tr.Processes.Default.Command = (
+    'curl --proxy 127.0.0.1:{0} "http://www.example.com/from_path/hello?=foo=bar" '
+    '-H "Proxy-Connection: keep-alive" --verbose'.format(
+        ts.Variables.port))
 tr.Processes.Default.ReturnCode = 0
-# time delay as proxy.config.http.wait_for_cache could be broken
 tr.Processes.Default.StartBefore(server, ready=When.PortOpen(server.Variables.Port))
 tr.Processes.Default.StartBefore(Test.Processes.ts)
 tr.Processes.Default.Streams.stderr = "gold/header_rewrite-client.gold"
 tr.StillRunningAfter = server
+ts.Streams.All = "gold/header_rewrite-tag.gold"
 
+# Test TO-URL in a set-redirect operator.
+tr = Test.AddTestRun()
+tr.Processes.Default.Command = 'curl --head 127.0.0.1:{0} -H "Host: no_path.com" --verbose'.format(
+    ts.Variables.port)
+tr.Processes.Default.ReturnCode = 0
+tr.Processes.Default.Streams.stderr = "gold/set-redirect.gold"
+tr.StillRunningAfter = server
 ts.Streams.All = "gold/header_rewrite-tag.gold"
diff --git a/tests/gold_tests/pluginTest/header_rewrite/rules/set_redirect.conf b/tests/gold_tests/pluginTest/header_rewrite/rules/set_redirect.conf
new file mode 100644
index 0000000..6a3230a
--- /dev/null
+++ b/tests/gold_tests/pluginTest/header_rewrite/rules/set_redirect.conf
@@ -0,0 +1,18 @@
+#
+# 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.
+
+set-redirect 301 %{TO-URL:URL}
diff --git a/tests/gold_tests/remap/gold/remap-zero-200.gold b/tests/gold_tests/remap/gold/remap-zero-200.gold
new file mode 100644
index 0000000..bcad791
--- /dev/null
+++ b/tests/gold_tests/remap/gold/remap-zero-200.gold
@@ -0,0 +1,7 @@
+``
+> GET / HTTP/1.1
+> Host: zero.one.two.three.com
+``
+< HTTP/1.1 200 OK
+< Date: ``
+``
diff --git a/tests/gold_tests/remap/regex_map.test.py b/tests/gold_tests/remap/regex_map.test.py
new file mode 100644
index 0000000..54b3e7e
--- /dev/null
+++ b/tests/gold_tests/remap/regex_map.test.py
@@ -0,0 +1,62 @@
+'''
+Verify correct behavior of regex_map in remap.config.
+'''
+#  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.
+
+Test.Summary = '''
+Verify correct behavior of regex_map in remap.config.
+'''
+
+Test.ContinueOnFail = True
+ts = Test.MakeATSProcess("ts")
+server = Test.MakeOriginServer("server")
+dns = Test.MakeDNServer("dns", default='127.0.0.1')
+
+Test.testName = ""
+request_header = {"headers": "GET / HTTP/1.1\r\nHost: zero.one.two.three.com\r\n\r\n",
+                  "timestamp": "1469733493.993", "body": ""}
+response_header = {"headers": "HTTP/1.1 200 OK\r\nConnection: close\r\n\r\n",
+                   "timestamp": "1469733493.993", "body": ""}
+server.addResponse("sessionfile.log", request_header, response_header)
+
+ts.Disk.records_config.update({
+    'proxy.config.diags.debug.enabled': 1,
+    'proxy.config.diags.debug.tags': 'http.*|dns|conf_remap',
+    'proxy.config.http.referer_filter': 1,
+    'proxy.config.dns.nameservers': '127.0.0.1:{0}'.format(dns.Variables.Port),
+    'proxy.config.dns.resolv_conf': 'NULL'
+})
+
+ts.Disk.remap_config.AddLine(
+    r'regex_map '
+    r'http://(.*)?one\.two\.three\.com/ '
+    r'http://$1reactivate.four.five.six.com:{}/'.format(server.Variables.Port)
+)
+ts.Disk.remap_config.AddLine(
+    r'regex_map '
+    r'https://\b(?!(.*one|two|three|four|five|six)).+\b\.seven\.eight\.nine\.com/blah12345.html '
+    r'https://www.example.com:{}/one/two/three/blah12345.html'.format(server.Variables.Port)
+)
+
+tr = Test.AddTestRun()
+tr.Processes.Default.Command = 'curl -H"Host: zero.one.two.three.com" http://127.0.0.1:{0}/ --verbose'.format(ts.Variables.port)
+tr.Processes.Default.ReturnCode = 0
+tr.Processes.Default.StartBefore(server)
+tr.Processes.Default.StartBefore(dns)
+tr.Processes.Default.StartBefore(Test.Processes.ts)
+tr.Processes.Default.Streams.stderr = "gold/remap-zero-200.gold"
+tr.StillRunningAfter = server