You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by ma...@apache.org on 2022/08/15 00:19:01 UTC

[trafficserver] branch master updated: Add HTTP header version converter (#8990)

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

maskit 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 85c756f39 Add HTTP header version converter (#8990)
85c756f39 is described below

commit 85c756f39bf3de42b71f7ca5bdc2a2b1c46fa0d5
Author: Masakazu Kitajo <ma...@apache.org>
AuthorDate: Mon Aug 15 09:18:54 2022 +0900

    Add HTTP header version converter (#8990)
    
    * Add HTTP header version converter
    
    * Fix the position of pragma once
    
    * Include HTTP.h in the source file but not the header file
    
    * Check HTTP_TYPE_UNKNOWN explicitly
    
    * Add doxygen comments
    
    * clang-format
---
 proxy/hdrs/HTTP.cc                                |  33 ++-
 proxy/hdrs/HTTP.h                                 |  16 +-
 proxy/hdrs/Makefile.am                            |   2 +
 proxy/hdrs/VersionConverter.cc                    | 275 +++++++++++++++++++++
 proxy/hdrs/VersionConverter.h                     | 115 +++++++++
 proxy/hdrs/unit_tests/test_Hdrs.cc                |   2 +-
 proxy/http/RegressionHttpTransact.cc              |   2 +-
 proxy/http/remap/unit-tests/nexthop_test_stubs.cc |   2 +-
 proxy/http2/HTTP2.cc                              | 279 ++--------------------
 proxy/http2/HTTP2.h                               |  13 -
 proxy/http2/Http2ConnectionState.cc               |   3 +-
 proxy/http2/Http2Stream.cc                        |   7 +-
 proxy/http2/unit_tests/test_HTTP2.cc              |  16 +-
 proxy/http3/Http3HeaderFramer.cc                  |  15 +-
 proxy/http3/Http3HeaderFramer.h                   |   3 +-
 proxy/http3/Http3HeaderVIOAdaptor.cc              |  38 +--
 proxy/http3/Http3HeaderVIOAdaptor.h               |   4 +-
 proxy/http3/Makefile.am                           |   2 -
 18 files changed, 471 insertions(+), 356 deletions(-)

diff --git a/proxy/hdrs/HTTP.cc b/proxy/hdrs/HTTP.cc
index 4e17d542f..3dae7b348 100644
--- a/proxy/hdrs/HTTP.cc
+++ b/proxy/hdrs/HTTP.cc
@@ -270,12 +270,12 @@ http_init()
   -------------------------------------------------------------------------*/
 
 HTTPHdrImpl *
-http_hdr_create(HdrHeap *heap, HTTPType polarity)
+http_hdr_create(HdrHeap *heap, HTTPType polarity, HTTPVersion version)
 {
   HTTPHdrImpl *hh;
 
   hh = (HTTPHdrImpl *)heap->allocate_obj(sizeof(HTTPHdrImpl), HDR_HEAP_OBJ_HTTP_HEADER);
-  http_hdr_init(heap, hh, polarity);
+  http_hdr_init(heap, hh, polarity, version);
   return (hh);
 }
 
@@ -283,7 +283,7 @@ http_hdr_create(HdrHeap *heap, HTTPType polarity)
   -------------------------------------------------------------------------*/
 
 void
-http_hdr_init(HdrHeap *heap, HTTPHdrImpl *hh, HTTPType polarity)
+http_hdr_init(HdrHeap *heap, HTTPHdrImpl *hh, HTTPType polarity, HTTPVersion version)
 {
   memset(&(hh->u), 0, sizeof(hh->u));
   hh->m_polarity    = polarity;
@@ -293,6 +293,31 @@ http_hdr_init(HdrHeap *heap, HTTPHdrImpl *hh, HTTPType polarity)
     hh->u.req.m_url_impl       = url_create(heap);
     hh->u.req.m_method_wks_idx = -1;
   }
+
+  if (version == HTTP_2_0 || version == HTTP_3_0) {
+    MIMEField *field;
+    switch (polarity) {
+    case HTTP_TYPE_REQUEST:
+      field = mime_field_create_named(heap, hh->m_fields_impl, PSEUDO_HEADER_METHOD.data(), PSEUDO_HEADER_METHOD.size());
+      mime_hdr_field_attach(hh->m_fields_impl, field, false, nullptr);
+
+      field = mime_field_create_named(heap, hh->m_fields_impl, PSEUDO_HEADER_SCHEME.data(), PSEUDO_HEADER_SCHEME.size());
+      mime_hdr_field_attach(hh->m_fields_impl, field, false, nullptr);
+
+      field = mime_field_create_named(heap, hh->m_fields_impl, PSEUDO_HEADER_AUTHORITY.data(), PSEUDO_HEADER_AUTHORITY.size());
+      mime_hdr_field_attach(hh->m_fields_impl, field, false, nullptr);
+
+      field = mime_field_create_named(heap, hh->m_fields_impl, PSEUDO_HEADER_PATH.data(), PSEUDO_HEADER_PATH.size());
+      mime_hdr_field_attach(hh->m_fields_impl, field, false, nullptr);
+      break;
+    case HTTP_TYPE_RESPONSE:
+      field = mime_field_create_named(heap, hh->m_fields_impl, PSEUDO_HEADER_STATUS.data(), PSEUDO_HEADER_STATUS.size());
+      mime_hdr_field_attach(hh->m_fields_impl, field, false, nullptr);
+      break;
+    default:
+      ink_abort("HTTP_TYPE_UNKNOWN");
+    }
+  }
 }
 
 /*-------------------------------------------------------------------------
@@ -348,7 +373,7 @@ http_hdr_clone(HTTPHdrImpl *s_hh, HdrHeap *s_heap, HdrHeap *d_heap)
   //      one single memcpy.  For this first optimization, we just
   //      copy each object separately.
 
-  d_hh = http_hdr_create(d_heap, s_hh->m_polarity);
+  d_hh = http_hdr_create(d_heap, s_hh->m_polarity, s_hh->m_version);
   http_hdr_copy_onto(s_hh, s_heap, d_hh, d_heap, ((s_heap != d_heap) ? true : false));
   return (d_hh);
 }
diff --git a/proxy/hdrs/HTTP.h b/proxy/hdrs/HTTP.h
index 6d01fb2e1..7e493f724 100644
--- a/proxy/hdrs/HTTP.h
+++ b/proxy/hdrs/HTTP.h
@@ -238,6 +238,12 @@ enum SquidHitMissCode {
   SQUID_HIT_RWW     = SQUID_HIT_LEVEL_6
 };
 
+constexpr std::string_view PSEUDO_HEADER_SCHEME    = ":scheme";
+constexpr std::string_view PSEUDO_HEADER_AUTHORITY = ":authority";
+constexpr std::string_view PSEUDO_HEADER_PATH      = ":path";
+constexpr std::string_view PSEUDO_HEADER_METHOD    = ":method";
+constexpr std::string_view PSEUDO_HEADER_STATUS    = ":status";
+
 enum HTTPType {
   HTTP_TYPE_UNKNOWN,
   HTTP_TYPE_REQUEST,
@@ -417,8 +423,8 @@ void http_hdr_adjust(HTTPHdrImpl *hdrp, int32_t offset, int32_t length, int32_t
 /* Public */
 void http_init();
 
-HTTPHdrImpl *http_hdr_create(HdrHeap *heap, HTTPType polarity);
-void http_hdr_init(HdrHeap *heap, HTTPHdrImpl *hh, HTTPType polarity);
+HTTPHdrImpl *http_hdr_create(HdrHeap *heap, HTTPType polarity, HTTPVersion version);
+void http_hdr_init(HdrHeap *heap, HTTPHdrImpl *hh, HTTPType polarity, HTTPVersion version);
 HTTPHdrImpl *http_hdr_clone(HTTPHdrImpl *s_hh, HdrHeap *s_heap, HdrHeap *d_heap);
 void http_hdr_copy_onto(HTTPHdrImpl *s_hh, HdrHeap *s_heap, HTTPHdrImpl *d_hh, HdrHeap *d_heap, bool inherit_strs);
 
@@ -492,7 +498,7 @@ public:
 
   int valid() const;
 
-  void create(HTTPType polarity, HdrHeap *heap = nullptr);
+  void create(HTTPType polarity, HTTPVersion version = HTTP_INVALID, HdrHeap *heap = nullptr);
   void clear();
   void reset();
   void copy(const HTTPHdr *hdr);
@@ -686,7 +692,7 @@ HTTPHdr::valid() const
   -------------------------------------------------------------------------*/
 
 inline void
-HTTPHdr::create(HTTPType polarity, HdrHeap *heap)
+HTTPHdr::create(HTTPType polarity, HTTPVersion version, HdrHeap *heap)
 {
   if (heap) {
     m_heap = heap;
@@ -694,7 +700,7 @@ HTTPHdr::create(HTTPType polarity, HdrHeap *heap)
     m_heap = new_HdrHeap();
   }
 
-  m_http = http_hdr_create(m_heap, polarity);
+  m_http = http_hdr_create(m_heap, polarity, version);
   m_mime = m_http->m_fields_impl;
 }
 
diff --git a/proxy/hdrs/Makefile.am b/proxy/hdrs/Makefile.am
index 7fcd2484a..66e19c0e2 100644
--- a/proxy/hdrs/Makefile.am
+++ b/proxy/hdrs/Makefile.am
@@ -44,6 +44,8 @@ libhdrs_a_SOURCES = \
 	MIME.h \
 	URL.cc \
 	URL.h \
+	VersionConverter.cc \
+	VersionConverter.h \
 	HuffmanCodec.cc \
 	HuffmanCodec.h \
 	XPACK.cc \
diff --git a/proxy/hdrs/VersionConverter.cc b/proxy/hdrs/VersionConverter.cc
new file mode 100644
index 000000000..599fbe3ab
--- /dev/null
+++ b/proxy/hdrs/VersionConverter.cc
@@ -0,0 +1,275 @@
+/** @file
+
+  HTTP header version converter
+
+  @section license License
+
+  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.
+ */
+
+#include "VersionConverter.h"
+#include "HTTP.h"
+#include "tscpp/util/LocalBuffer.h"
+
+int
+VersionConverter::convert(HTTPHdr &header, int from, int to) const
+{
+  int type;
+
+  switch (http_hdr_type_get(header.m_http)) {
+  case HTTP_TYPE_REQUEST:
+    type = 0;
+    break;
+  case HTTP_TYPE_RESPONSE:
+    type = 1;
+    break;
+  case HTTP_TYPE_UNKNOWN:
+    ink_abort("HTTP_TYPE_UNKNOWN");
+    break;
+  }
+
+  ink_assert(MIN_VERSION <= from && from <= MAX_VERSION);
+  ink_assert(MIN_VERSION <= to && to <= MAX_VERSION);
+
+  int ret = (this->*_convert_functions[type][from - 1][to - 1])(header);
+  if (ret < 0) {
+    return -1;
+  }
+
+  // Check validity of all names and values
+  for (auto &&mf : header) {
+    if (!mf.name_is_valid(is_control_BIT | is_ws_BIT) || !mf.value_is_valid()) {
+      return -1;
+    }
+  }
+
+  return 0;
+}
+
+int
+VersionConverter::_convert_nop(HTTPHdr &header) const
+{
+  return 0;
+}
+
+int
+VersionConverter::_convert_req_from_1_to_2(HTTPHdr &header) const
+{
+  // :method
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_METHOD.data(), PSEUDO_HEADER_METHOD.size()); field != nullptr) {
+    int value_len;
+    const char *value = header.method_get(&value_len);
+
+    field->value_set(header.m_heap, header.m_mime, value, value_len);
+  } else {
+    ink_abort("initialize HTTP/2 pseudo-headers");
+    return PARSE_RESULT_ERROR;
+  }
+
+  // :scheme
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_SCHEME.data(), PSEUDO_HEADER_SCHEME.size()); field != nullptr) {
+    int value_len;
+    const char *value = header.scheme_get(&value_len);
+
+    if (value != nullptr) {
+      field->value_set(header.m_heap, header.m_mime, value, value_len);
+    } else {
+      field->value_set(header.m_heap, header.m_mime, URL_SCHEME_HTTPS, URL_LEN_HTTPS);
+    }
+  } else {
+    ink_abort("initialize HTTP/2 pseudo-headers");
+    return PARSE_RESULT_ERROR;
+  }
+
+  // :authority
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_AUTHORITY.data(), PSEUDO_HEADER_AUTHORITY.size()); field != nullptr) {
+    int value_len;
+    const char *value = header.host_get(&value_len);
+
+    if (header.is_port_in_header()) {
+      int port = header.port_get();
+      ts::LocalBuffer<char> buf(value_len + 8);
+      char *host_and_port = buf.data();
+      value_len           = snprintf(host_and_port, value_len + 8, "%.*s:%d", value_len, value, port);
+
+      field->value_set(header.m_heap, header.m_mime, host_and_port, value_len);
+    } else {
+      field->value_set(header.m_heap, header.m_mime, value, value_len);
+    }
+  } else {
+    ink_abort("initialize HTTP/2 pseudo-headers");
+    return PARSE_RESULT_ERROR;
+  }
+
+  // :path
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_PATH.data(), PSEUDO_HEADER_PATH.size()); field != nullptr) {
+    int value_len     = 0;
+    const char *value = header.path_get(&value_len);
+
+    ts::LocalBuffer<char> buf(value_len + 1);
+    char *path = buf.data();
+    path[0]    = '/';
+    memcpy(path + 1, value, value_len);
+
+    field->value_set(header.m_heap, header.m_mime, path, value_len + 1);
+  } else {
+    ink_abort("initialize HTTP/2 pseudo-headers");
+    return PARSE_RESULT_ERROR;
+  }
+
+  // TODO: remove host/Host header
+  // [RFC 7540] 8.1.2.3. Clients that generate HTTP/2 requests directly SHOULD use the ":authority" pseudo-header field instead
+  // of the Host header field.
+
+  this->_remove_connection_specific_header_fields(header);
+
+  return 0;
+}
+
+int
+VersionConverter::_convert_req_from_2_to_1(HTTPHdr &header) const
+{
+  // HTTP Version
+  header.version_set(HTTPVersion(1, 1));
+
+  // :scheme
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_SCHEME.data(), PSEUDO_HEADER_SCHEME.size());
+      field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
+    int scheme_len;
+    const char *scheme = field->value_get(&scheme_len);
+    const char *scheme_wks;
+
+    int scheme_wks_idx = hdrtoken_tokenize(scheme, scheme_len, &scheme_wks);
+
+    if (!(scheme_wks_idx > 0 && hdrtoken_wks_to_token_type(scheme_wks) == HDRTOKEN_TYPE_SCHEME)) {
+      // unkown scheme, validate the scheme
+      if (!validate_scheme({scheme, static_cast<size_t>(scheme_len)})) {
+        return PARSE_RESULT_ERROR;
+      }
+    }
+
+    header.m_http->u.req.m_url_impl->set_scheme(header.m_heap, scheme, scheme_wks_idx, scheme_len, true);
+
+    header.field_delete(field);
+  } else {
+    return PARSE_RESULT_ERROR;
+  }
+
+  // :authority
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_AUTHORITY.data(), PSEUDO_HEADER_AUTHORITY.size());
+      field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
+    int authority_len;
+    const char *authority = field->value_get(&authority_len);
+
+    header.m_http->u.req.m_url_impl->set_host(header.m_heap, authority, authority_len, true);
+
+    header.field_delete(field);
+  } else {
+    return PARSE_RESULT_ERROR;
+  }
+
+  // :path
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_PATH.data(), PSEUDO_HEADER_PATH.size());
+      field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
+    int path_len;
+    const char *path = field->value_get(&path_len);
+
+    // cut first '/' if there, because `url_print()` add '/' before printing path
+    if (path_len >= 1 && path[0] == '/') {
+      ++path;
+      --path_len;
+    }
+
+    header.m_http->u.req.m_url_impl->set_path(header.m_heap, path, path_len, true);
+
+    header.field_delete(field);
+  } else {
+    return PARSE_RESULT_ERROR;
+  }
+
+  // :method
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_METHOD.data(), PSEUDO_HEADER_METHOD.size());
+      field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
+    int method_len;
+    const char *method = field->value_get(&method_len);
+
+    header.method_set(method, method_len);
+    header.field_delete(field);
+  } else {
+    return PARSE_RESULT_ERROR;
+  }
+
+  // Combine Cookie header.([RFC 7540] 8.1.2.5.)
+  if (MIMEField *field = header.field_find(MIME_FIELD_COOKIE, MIME_LEN_COOKIE); field != nullptr) {
+    header.field_combine_dups(field, true, ';');
+  }
+
+  return 0;
+}
+
+int
+VersionConverter::_convert_res_from_1_to_2(HTTPHdr &header) const
+{
+  constexpr int STATUS_VALUE_LEN = 3;
+
+  // :status
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_STATUS.data(), PSEUDO_HEADER_STATUS.size()); field != nullptr) {
+    // ink_small_itoa() requires 5+ buffer length
+    char status_str[STATUS_VALUE_LEN + 3];
+    mime_format_int(status_str, header.status_get(), sizeof(status_str));
+
+    field->value_set(header.m_heap, header.m_mime, status_str, STATUS_VALUE_LEN);
+  } else {
+    ink_abort("initialize HTTP/2 pseudo-headers");
+    return PARSE_RESULT_ERROR;
+  }
+
+  this->_remove_connection_specific_header_fields(header);
+
+  return 0;
+}
+
+int
+VersionConverter::_convert_res_from_2_to_1(HTTPHdr &header) const
+{
+  // HTTP Version
+  header.version_set(HTTPVersion(1, 1));
+
+  // Set status from :status
+  if (MIMEField *field = header.field_find(PSEUDO_HEADER_STATUS.data(), PSEUDO_HEADER_STATUS.size()); field != nullptr) {
+    int status_len;
+    const char *status = field->value_get(&status_len);
+
+    header.status_set(http_parse_status(status, status + status_len));
+    header.field_delete(field);
+  } else {
+    return -1;
+  }
+
+  return 0;
+}
+
+void
+VersionConverter::_remove_connection_specific_header_fields(HTTPHdr &header) const
+{
+  // Intermediaries SHOULD remove connection-specific header fields.
+  for (auto &&h : connection_specific_header_fields) {
+    if (MIMEField *field = header.field_find(h.data(), h.size()); field != nullptr) {
+      header.field_delete(field);
+    }
+  }
+}
diff --git a/proxy/hdrs/VersionConverter.h b/proxy/hdrs/VersionConverter.h
new file mode 100644
index 000000000..bb0e0e25d
--- /dev/null
+++ b/proxy/hdrs/VersionConverter.h
@@ -0,0 +1,115 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  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.
+ */
+
+#pragma once
+
+#include <string_view>
+
+class HTTPHdr;
+
+/**
+ * HTTP Header Version Converter
+ *
+ * This converts HTTP header from a version to another. Currently it supports conversion among HTTP/1.1, HTTP/2, and HTTP/3.
+ */
+class VersionConverter
+{
+public:
+  /**
+   * Converts HTTP header version
+   *
+   * HTTP/2 and HTTP/3 use pseudo headers to store information that are in a
+   * request line or a status line on HTTP/1.1. This function relocates the
+   * information accordingly base on versions specified. @p from and @p to
+   * receive integers 1-3 that correspond to HTTP/1.1, HTTP/2 and HTTP/3.
+   *
+   * @param header A reference to HTTPHdr to be converted
+   * @param from Original HTTP version
+   * @param to Target HTTP version
+   * @return Returns @c 0 if conversion succeeds
+   */
+  int convert(HTTPHdr &header, int from, int to) const;
+
+private:
+  int _convert_nop(HTTPHdr &header) const;
+  int _convert_req_from_1_to_2(HTTPHdr &header) const;
+  int _convert_req_from_2_to_1(HTTPHdr &header) const;
+  int _convert_res_from_1_to_2(HTTPHdr &header) const;
+  int _convert_res_from_2_to_1(HTTPHdr &header) const;
+
+  void _remove_connection_specific_header_fields(HTTPHdr &header) const;
+
+  using convert_function = int (VersionConverter::*)(HTTPHdr &) const;
+
+  static constexpr int MIN_VERSION = 1;
+  static constexpr int MAX_VERSION = 3;
+  static constexpr int N_VERSIONS  = MAX_VERSION - MIN_VERSION + 1;
+
+  static constexpr convert_function _convert_functions[2][N_VERSIONS][N_VERSIONS] = {
+    {
+      // Request
+      {
+        // From 1
+        &VersionConverter::_convert_nop,
+        &VersionConverter::_convert_req_from_1_to_2,
+        &VersionConverter::_convert_req_from_1_to_2,
+      },
+      {
+        // From 2
+        &VersionConverter::_convert_req_from_2_to_1,
+        &VersionConverter::_convert_nop,
+        &VersionConverter::_convert_nop,
+      },
+      {
+        // From 3
+        &VersionConverter::_convert_req_from_2_to_1,
+        &VersionConverter::_convert_nop,
+        &VersionConverter::_convert_nop,
+      },
+    },
+    {
+      // Response
+      {
+        // From 1
+        &VersionConverter::_convert_nop,
+        &VersionConverter::_convert_res_from_1_to_2,
+        &VersionConverter::_convert_res_from_1_to_2,
+      },
+      {
+        // From 2
+        &VersionConverter::_convert_res_from_2_to_1,
+        &VersionConverter::_convert_nop,
+        &VersionConverter::_convert_nop,
+      },
+      {
+        // From 3
+        &VersionConverter::_convert_res_from_2_to_1,
+        &VersionConverter::_convert_nop,
+        &VersionConverter::_convert_nop,
+      },
+    }};
+
+  static constexpr std::string_view connection_specific_header_fields[] = {
+    "Connection", "Keep-Alive", "Proxy-Connection", "Transfer-Encoding", "Upgrade",
+  };
+};
diff --git a/proxy/hdrs/unit_tests/test_Hdrs.cc b/proxy/hdrs/unit_tests/test_Hdrs.cc
index 7d9c39eef..8a02a89b7 100644
--- a/proxy/hdrs/unit_tests/test_Hdrs.cc
+++ b/proxy/hdrs/unit_tests/test_Hdrs.cc
@@ -78,7 +78,7 @@ TEST_CASE("HdrTestHttpParse", "[proxy][hdrtest]")
     HTTPHdr req_hdr;
     HdrHeap *heap = new_HdrHeap(HdrHeap::DEFAULT_SIZE + 64); // extra to prevent proxy allocation.
 
-    req_hdr.create(HTTP_TYPE_REQUEST, heap);
+    req_hdr.create(HTTP_TYPE_REQUEST, HTTP_1_1, heap);
 
     http_parser_clear(&parser);
 
diff --git a/proxy/http/RegressionHttpTransact.cc b/proxy/http/RegressionHttpTransact.cc
index 501a01e59..7008edea9 100644
--- a/proxy/http/RegressionHttpTransact.cc
+++ b/proxy/http/RegressionHttpTransact.cc
@@ -34,7 +34,7 @@ static void
 init_sm(HttpSM *sm)
 {
   sm->init();
-  sm->t_state.hdr_info.client_request.create(HTTP_TYPE_REQUEST, nullptr);
+  sm->t_state.hdr_info.client_request.create(HTTP_TYPE_REQUEST, HTTP_1_1, nullptr);
 }
 
 static void
diff --git a/proxy/http/remap/unit-tests/nexthop_test_stubs.cc b/proxy/http/remap/unit-tests/nexthop_test_stubs.cc
index 905651261..af54522ea 100644
--- a/proxy/http/remap/unit-tests/nexthop_test_stubs.cc
+++ b/proxy/http/remap/unit-tests/nexthop_test_stubs.cc
@@ -100,7 +100,7 @@ build_request(int64_t sm_id, HttpSM *sm, sockaddr_in *ip, const char *os_hostnam
     delete sm->t_state.request_data.hdr;
   }
   sm->t_state.request_data.hdr = new HTTPHdr();
-  sm->t_state.request_data.hdr->create(HTTP_TYPE_REQUEST, myHeap);
+  sm->t_state.request_data.hdr->create(HTTP_TYPE_REQUEST, HTTP_1_1, myHeap);
   sm->t_state.request_data.hostname_str = sm->t_state.arena.str_store(os_hostname, strlen(os_hostname));
   sm->t_state.request_data.xact_start   = time(nullptr);
   ink_zero(sm->t_state.request_data.src_ip);
diff --git a/proxy/http2/HTTP2.cc b/proxy/http2/HTTP2.cc
index 5b263bbe7..ce9d8c92c 100644
--- a/proxy/http2/HTTP2.cc
+++ b/proxy/http2/HTTP2.cc
@@ -21,6 +21,7 @@
  *  limitations under the License.
  */
 
+#include "hdrs/VersionConverter.h"
 #include "HTTP2.h"
 #include "HPACK.h"
 
@@ -32,20 +33,6 @@
 
 const char *const HTTP2_CONNECTION_PREFACE = "PRI * HTTP/2.0\r\n\r\nSM\r\n\r\n";
 
-// Constant strings for pseudo headers
-const char *HTTP2_VALUE_SCHEME    = ":scheme";
-const char *HTTP2_VALUE_METHOD    = ":method";
-const char *HTTP2_VALUE_AUTHORITY = ":authority";
-const char *HTTP2_VALUE_PATH      = ":path";
-const char *HTTP2_VALUE_STATUS    = ":status";
-
-const unsigned HTTP2_LEN_SCHEME    = countof(":scheme") - 1;
-const unsigned HTTP2_LEN_METHOD    = countof(":method") - 1;
-const unsigned HTTP2_LEN_AUTHORITY = countof(":authority") - 1;
-const unsigned HTTP2_LEN_PATH      = countof(":path") - 1;
-const unsigned HTTP2_LEN_STATUS    = countof(":status") - 1;
-
-static size_t HTTP2_LEN_STATUS_VALUE_STR         = 3;
 static const uint32_t HTTP2_MAX_TABLE_SIZE_LIMIT = 64 * 1024;
 
 namespace
@@ -55,7 +42,8 @@ struct Http2HeaderName {
   int name_len     = 0;
 };
 
-Http2HeaderName http2_connection_specific_headers[5] = {};
+static VersionConverter hvc;
+
 } // namespace
 
 // Statistics
@@ -423,244 +411,25 @@ http2_parse_window_update(IOVec iov, uint32_t &size)
 ParseResult
 http2_convert_header_from_2_to_1_1(HTTPHdr *headers)
 {
-  ink_assert(http_hdr_type_get(headers->m_http) != HTTP_TYPE_UNKNOWN);
-
-  // HTTP Version
-  headers->version_set(HTTPVersion(1, 1));
-
-  if (http_hdr_type_get(headers->m_http) == HTTP_TYPE_REQUEST) {
-    // :scheme
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_SCHEME, HTTP2_LEN_SCHEME);
-        field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
-      int scheme_len;
-      const char *scheme = field->value_get(&scheme_len);
-      const char *scheme_wks;
-
-      int scheme_wks_idx = hdrtoken_tokenize(scheme, scheme_len, &scheme_wks);
-
-      if (!(scheme_wks_idx > 0 && hdrtoken_wks_to_token_type(scheme_wks) == HDRTOKEN_TYPE_SCHEME)) {
-        // unkown scheme, validate the scheme
-        if (!validate_scheme({scheme, static_cast<size_t>(scheme_len)})) {
-          return PARSE_RESULT_ERROR;
-        }
-      }
-
-      headers->m_http->u.req.m_url_impl->set_scheme(headers->m_heap, scheme, scheme_wks_idx, scheme_len, true);
-
-      headers->field_delete(field);
-    } else {
-      return PARSE_RESULT_ERROR;
-    }
-
-    // :authority
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_AUTHORITY, HTTP2_LEN_AUTHORITY);
-        field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
-      int authority_len;
-      const char *authority = field->value_get(&authority_len);
-
-      headers->m_http->u.req.m_url_impl->set_host(headers->m_heap, authority, authority_len, true);
-
-      headers->field_delete(field);
-    } else {
-      return PARSE_RESULT_ERROR;
-    }
-
-    // :path
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_PATH, HTTP2_LEN_PATH);
-        field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
-      int path_len;
-      const char *path = field->value_get(&path_len);
-
-      // cut first '/' if there, because `url_print()` add '/' before printing path
-      if (path_len >= 1 && path[0] == '/') {
-        ++path;
-        --path_len;
-      }
-
-      headers->m_http->u.req.m_url_impl->set_path(headers->m_heap, path, path_len, true);
-
-      headers->field_delete(field);
-    } else {
-      return PARSE_RESULT_ERROR;
-    }
-
-    // :method
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_METHOD, HTTP2_LEN_METHOD);
-        field != nullptr && field->value_is_valid(is_control_BIT | is_ws_BIT)) {
-      int method_len;
-      const char *method = field->value_get(&method_len);
-
-      headers->method_set(method, method_len);
-      headers->field_delete(field);
-    } else {
-      return PARSE_RESULT_ERROR;
-    }
-
-    // Combine Cookie headers ([RFC 7540] 8.1.2.5.)
-    if (MIMEField *field = headers->field_find(MIME_FIELD_COOKIE, MIME_LEN_COOKIE); field != nullptr) {
-      headers->field_combine_dups(field, true, ';');
-    }
+  if (hvc.convert(*headers, 2, 1) == 0) {
+    return PARSE_RESULT_DONE;
   } else {
-    // Set status from :status
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS); field != nullptr) {
-      int status_len;
-      const char *status = field->value_get(&status_len);
-
-      headers->status_set(http_parse_status(status, status + status_len));
-      headers->field_delete(field);
-    } else {
-      return PARSE_RESULT_ERROR;
-    }
-  }
-
-  // Check validity of all names and values
-  for (auto &mf : *headers) {
-    if (!mf.name_is_valid(is_control_BIT | is_ws_BIT) || !mf.value_is_valid()) {
-      return PARSE_RESULT_ERROR;
-    }
-  }
-
-  return PARSE_RESULT_DONE;
-}
-
-/**
-  Initialize HTTPHdr for HTTP/2
-
-  Reserve HTTP/2 Pseudo-Header Fields in front of HTTPHdr. Value of these header fields will be set by
-  `http2_convert_header_from_1_1_to_2()`. When a HTTPHdr for HTTP/2 headers is created, this should be called immediately.
-  Because all pseudo-header fields MUST appear in the header block before regular header fields.
- */
-void
-http2_init_pseudo_headers(HTTPHdr &hdr)
-{
-  switch (http_hdr_type_get(hdr.m_http)) {
-  case HTTP_TYPE_REQUEST: {
-    MIMEField *method = hdr.field_create(HTTP2_VALUE_METHOD, HTTP2_LEN_METHOD);
-    hdr.field_attach(method);
-
-    MIMEField *scheme = hdr.field_create(HTTP2_VALUE_SCHEME, HTTP2_LEN_SCHEME);
-    hdr.field_attach(scheme);
-
-    MIMEField *authority = hdr.field_create(HTTP2_VALUE_AUTHORITY, HTTP2_LEN_AUTHORITY);
-    hdr.field_attach(authority);
-
-    MIMEField *path = hdr.field_create(HTTP2_VALUE_PATH, HTTP2_LEN_PATH);
-    hdr.field_attach(path);
-
-    break;
-  }
-  case HTTP_TYPE_RESPONSE: {
-    MIMEField *status = hdr.field_create(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS);
-    hdr.field_attach(status);
-
-    break;
-  }
-  default:
-    ink_abort("HTTP_TYPE_UNKNOWN");
+    return PARSE_RESULT_ERROR;
   }
 }
 
 /**
   Convert HTTP/1.1 HTTPHdr to HTTP/2
 
-  Assuming HTTP/2 Pseudo-Header Fields are reserved by `http2_init_pseudo_headers()`.
+  Assuming HTTP/2 Pseudo-Header Fields are reserved by passing a version to `HTTPHdr::create()`.
  */
 ParseResult
 http2_convert_header_from_1_1_to_2(HTTPHdr *headers)
 {
-  switch (http_hdr_type_get(headers->m_http)) {
-  case HTTP_TYPE_REQUEST: {
-    // :method
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_METHOD, HTTP2_LEN_METHOD); field != nullptr) {
-      int value_len;
-      const char *value = headers->method_get(&value_len);
-
-      field->value_set(headers->m_heap, headers->m_mime, value, value_len);
-    } else {
-      ink_abort("initialize HTTP/2 pseudo-headers");
-      return PARSE_RESULT_ERROR;
-    }
-
-    // :scheme
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_SCHEME, HTTP2_LEN_SCHEME); field != nullptr) {
-      int value_len;
-      const char *value = headers->scheme_get(&value_len);
-
-      if (value != nullptr) {
-        field->value_set(headers->m_heap, headers->m_mime, value, value_len);
-      } else {
-        field->value_set(headers->m_heap, headers->m_mime, URL_SCHEME_HTTPS, URL_LEN_HTTPS);
-      }
-    } else {
-      ink_abort("initialize HTTP/2 pseudo-headers");
-      return PARSE_RESULT_ERROR;
-    }
-
-    // :authority
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_AUTHORITY, HTTP2_LEN_AUTHORITY); field != nullptr) {
-      int value_len;
-      const char *value = headers->host_get(&value_len);
-
-      if (headers->is_port_in_header()) {
-        int port = headers->port_get();
-        ts::LocalBuffer<char> buf(value_len + 8);
-        char *host_and_port = buf.data();
-        value_len           = snprintf(host_and_port, value_len + 8, "%.*s:%d", value_len, value, port);
-
-        field->value_set(headers->m_heap, headers->m_mime, host_and_port, value_len);
-      } else {
-        field->value_set(headers->m_heap, headers->m_mime, value, value_len);
-      }
-    } else {
-      ink_abort("initialize HTTP/2 pseudo-headers");
-      return PARSE_RESULT_ERROR;
-    }
-
-    // :path
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_PATH, HTTP2_LEN_PATH); field != nullptr) {
-      int value_len     = 0;
-      const char *value = headers->path_get(&value_len);
-
-      ts::LocalBuffer<char> buf(value_len + 1);
-      char *path = buf.data();
-      path[0]    = '/';
-      memcpy(path + 1, value, value_len);
-
-      field->value_set(headers->m_heap, headers->m_mime, path, value_len + 1);
-    } else {
-      ink_abort("initialize HTTP/2 pseudo-headers");
-      return PARSE_RESULT_ERROR;
-    }
-
-    // TODO: remove host/Host header
-    // [RFC 7540] 8.1.2.3. Clients that generate HTTP/2 requests directly SHOULD use the ":authority" pseudo-header field instead
-    // of the Host header field.
-
-    break;
-  }
-  case HTTP_TYPE_RESPONSE: {
-    // :status
-    if (MIMEField *field = headers->field_find(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS); field != nullptr) {
-      // ink_small_itoa() requires 5+ buffer length
-      char status_str[HTTP2_LEN_STATUS_VALUE_STR + 3];
-      mime_format_int(status_str, headers->status_get(), sizeof(status_str));
-
-      field->value_set(headers->m_heap, headers->m_mime, status_str, HTTP2_LEN_STATUS_VALUE_STR);
-    } else {
-      ink_abort("initialize HTTP/2 pseudo-headers");
-      return PARSE_RESULT_ERROR;
-    }
-    break;
-  }
-  default:
-    ink_abort("HTTP_TYPE_UNKNOWN");
-  }
-
-  // Intermediaries SHOULD remove connection-specific header fields.
-  for (auto &h : http2_connection_specific_headers) {
-    if (MIMEField *field = headers->field_find(h.name, h.name_len); field != nullptr) {
-      headers->field_delete(field);
-    }
+  if (hvc.convert(*headers, 1, 2) == 0) {
+    return PARSE_RESULT_DONE;
+  } else {
+    return PARSE_RESULT_ERROR;
   }
 
   return PARSE_RESULT_DONE;
@@ -750,7 +519,7 @@ http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint32_
   }
 
   // :path pseudo header MUST NOT empty for http or https URIs
-  field = hdr->field_find(HTTP2_VALUE_PATH, HTTP2_LEN_PATH);
+  field = hdr->field_find(PSEUDO_HEADER_PATH.data(), PSEUDO_HEADER_PATH.size());
   if (field) {
     field->value_get(&len);
     if (len == 0) {
@@ -778,11 +547,11 @@ http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint32_
   if (!is_trailing_header) {
     // Check pseudo headers
     if (hdr->fields_count() >= 4) {
-      if (hdr->field_find(HTTP2_VALUE_SCHEME, HTTP2_LEN_SCHEME) == nullptr ||
-          hdr->field_find(HTTP2_VALUE_METHOD, HTTP2_LEN_METHOD) == nullptr ||
-          hdr->field_find(HTTP2_VALUE_PATH, HTTP2_LEN_PATH) == nullptr ||
-          hdr->field_find(HTTP2_VALUE_AUTHORITY, HTTP2_LEN_AUTHORITY) == nullptr ||
-          hdr->field_find(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS) != nullptr) {
+      if (hdr->field_find(PSEUDO_HEADER_SCHEME.data(), PSEUDO_HEADER_SCHEME.size()) == nullptr ||
+          hdr->field_find(PSEUDO_HEADER_METHOD.data(), PSEUDO_HEADER_METHOD.size()) == nullptr ||
+          hdr->field_find(PSEUDO_HEADER_PATH.data(), PSEUDO_HEADER_PATH.size()) == nullptr ||
+          hdr->field_find(PSEUDO_HEADER_AUTHORITY.data(), PSEUDO_HEADER_AUTHORITY.size()) == nullptr ||
+          hdr->field_find(PSEUDO_HEADER_STATUS.data(), PSEUDO_HEADER_STATUS.size()) != nullptr) {
         // Decoded header field is invalid
         return Http2ErrorCode::HTTP2_ERROR_PROTOCOL_ERROR;
       }
@@ -927,21 +696,7 @@ Http2::init()
   http2_init();
 }
 
-/**
-  mime_init() needs to be called
- */
 void
 http2_init()
 {
-  ink_assert(MIME_FIELD_CONNECTION != nullptr);
-  ink_assert(MIME_FIELD_KEEP_ALIVE != nullptr);
-  ink_assert(MIME_FIELD_PROXY_CONNECTION != nullptr);
-  ink_assert(MIME_FIELD_TRANSFER_ENCODING != nullptr);
-  ink_assert(MIME_FIELD_UPGRADE != nullptr);
-
-  http2_connection_specific_headers[0] = {MIME_FIELD_CONNECTION, MIME_LEN_CONNECTION};
-  http2_connection_specific_headers[1] = {MIME_FIELD_KEEP_ALIVE, MIME_LEN_KEEP_ALIVE};
-  http2_connection_specific_headers[2] = {MIME_FIELD_PROXY_CONNECTION, MIME_LEN_PROXY_CONNECTION};
-  http2_connection_specific_headers[3] = {MIME_FIELD_TRANSFER_ENCODING, MIME_LEN_TRANSFER_ENCODING};
-  http2_connection_specific_headers[4] = {MIME_FIELD_UPGRADE, MIME_LEN_UPGRADE};
 }
diff --git a/proxy/http2/HTTP2.h b/proxy/http2/HTTP2.h
index b5065668e..4418d6856 100644
--- a/proxy/http2/HTTP2.h
+++ b/proxy/http2/HTTP2.h
@@ -43,18 +43,6 @@ typedef int32_t Http2WindowSize;
 extern const char *const HTTP2_CONNECTION_PREFACE;
 const size_t HTTP2_CONNECTION_PREFACE_LEN = 24;
 
-extern const char *HTTP2_VALUE_SCHEME;
-extern const char *HTTP2_VALUE_METHOD;
-extern const char *HTTP2_VALUE_AUTHORITY;
-extern const char *HTTP2_VALUE_PATH;
-extern const char *HTTP2_VALUE_STATUS;
-
-extern const unsigned HTTP2_LEN_SCHEME;
-extern const unsigned HTTP2_LEN_METHOD;
-extern const unsigned HTTP2_LEN_AUTHORITY;
-extern const unsigned HTTP2_LEN_PATH;
-extern const unsigned HTTP2_LEN_STATUS;
-
 const size_t HTTP2_FRAME_HEADER_LEN       = 9;
 const size_t HTTP2_DATA_PADLEN_LEN        = 1;
 const size_t HTTP2_HEADERS_PADLEN_LEN     = 1;
@@ -370,7 +358,6 @@ Http2ErrorCode http2_encode_header_blocks(HTTPHdr *, uint8_t *, uint32_t, uint32
 
 ParseResult http2_convert_header_from_2_to_1_1(HTTPHdr *);
 ParseResult http2_convert_header_from_1_1_to_2(HTTPHdr *);
-void http2_init_pseudo_headers(HTTPHdr &);
 void http2_init();
 
 // Not sure where else to put this, but figure this is as good of a start as
diff --git a/proxy/http2/Http2ConnectionState.cc b/proxy/http2/Http2ConnectionState.cc
index b0e3a1cd9..07f325daf 100644
--- a/proxy/http2/Http2ConnectionState.cc
+++ b/proxy/http2/Http2ConnectionState.cc
@@ -1829,8 +1829,7 @@ Http2ConnectionState::send_push_promise_frame(Http2Stream *stream, URL &url, con
 
   HTTPHdr hdr;
   ts::PostScript hdr_defer([&]() -> void { hdr.destroy(); });
-  hdr.create(HTTP_TYPE_REQUEST);
-  http2_init_pseudo_headers(hdr);
+  hdr.create(HTTP_TYPE_REQUEST, HTTP_2_0);
   hdr.url_set(&url);
   hdr.method_set(HTTP_METHOD_GET, HTTP_LEN_GET);
 
diff --git a/proxy/http2/Http2Stream.cc b/proxy/http2/Http2Stream.cc
index d019d44ef..7d3a8b7ac 100644
--- a/proxy/http2/Http2Stream.cc
+++ b/proxy/http2/Http2Stream.cc
@@ -57,9 +57,7 @@ Http2Stream::Http2Stream(ProxySession *session, Http2StreamId sid, ssize_t initi
   this->_reader = this->_request_buffer.alloc_reader();
 
   _req_header.create(HTTP_TYPE_REQUEST);
-  response_header.create(HTTP_TYPE_RESPONSE);
-  // TODO: init _req_header instead of response_header if this Http2Stream is outgoing
-  http2_init_pseudo_headers(response_header);
+  response_header.create(HTTP_TYPE_RESPONSE, HTTP_2_0);
 
   http_parser_init(&http_parser);
 }
@@ -688,8 +686,7 @@ Http2Stream::update_write_request(bool call_update)
       if (this->response_header.expect_final_response()) {
         this->response_header_done = false;
         response_header.destroy();
-        response_header.create(HTTP_TYPE_RESPONSE);
-        http2_init_pseudo_headers(response_header);
+        response_header.create(HTTP_TYPE_RESPONSE, HTTP_2_0);
         http_parser_clear(&http_parser);
         http_parser_init(&http_parser);
       }
diff --git a/proxy/http2/unit_tests/test_HTTP2.cc b/proxy/http2/unit_tests/test_HTTP2.cc
index 21a772ae9..5ec532031 100644
--- a/proxy/http2/unit_tests/test_HTTP2.cc
+++ b/proxy/http2/unit_tests/test_HTTP2.cc
@@ -47,8 +47,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
 
     HTTPHdr hdr_1;
     ts::PostScript hdr_1_defer([&]() -> void { hdr_1.destroy(); });
-    hdr_1.create(HTTP_TYPE_REQUEST);
-    http2_init_pseudo_headers(hdr_1);
+    hdr_1.create(HTTP_TYPE_REQUEST, HTTP_2_0);
 
     // parse
     const char *start = request;
@@ -61,7 +60,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
     // check pseudo headers
     // :method
     {
-      MIMEField *f = hdr_1.field_find(HTTP2_VALUE_METHOD, HTTP2_LEN_METHOD);
+      MIMEField *f = hdr_1.field_find(PSEUDO_HEADER_METHOD.data(), PSEUDO_HEADER_METHOD.size());
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
       CHECK(v == "GET");
@@ -69,7 +68,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
 
     // :scheme
     {
-      MIMEField *f = hdr_1.field_find(HTTP2_VALUE_SCHEME, HTTP2_LEN_SCHEME);
+      MIMEField *f = hdr_1.field_find(PSEUDO_HEADER_SCHEME.data(), PSEUDO_HEADER_SCHEME.size());
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
       CHECK(v == "https");
@@ -77,7 +76,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
 
     // :authority
     {
-      MIMEField *f = hdr_1.field_find(HTTP2_VALUE_AUTHORITY, HTTP2_LEN_AUTHORITY);
+      MIMEField *f = hdr_1.field_find(PSEUDO_HEADER_AUTHORITY.data(), PSEUDO_HEADER_AUTHORITY.size());
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
       CHECK(v == "trafficserver.apache.org");
@@ -85,7 +84,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
 
     // :path
     {
-      MIMEField *f = hdr_1.field_find(HTTP2_VALUE_PATH, HTTP2_LEN_PATH);
+      MIMEField *f = hdr_1.field_find(PSEUDO_HEADER_PATH.data(), PSEUDO_HEADER_PATH.size());
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
       CHECK(v == "/index.html");
@@ -121,8 +120,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
 
     HTTPHdr hdr_1;
     ts::PostScript hdr_1_defer([&]() -> void { hdr_1.destroy(); });
-    hdr_1.create(HTTP_TYPE_RESPONSE);
-    http2_init_pseudo_headers(hdr_1);
+    hdr_1.create(HTTP_TYPE_RESPONSE, HTTP_2_0);
 
     // parse
     const char *start = response;
@@ -135,7 +133,7 @@ TEST_CASE("Convert HTTPHdr", "[HTTP2]")
     // check pseudo headers
     // :status
     {
-      MIMEField *f = hdr_1.field_find(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS);
+      MIMEField *f = hdr_1.field_find(PSEUDO_HEADER_STATUS.data(), PSEUDO_HEADER_STATUS.size());
       REQUIRE(f != nullptr);
       std::string_view v = f->value_get();
       CHECK(v == "200");
diff --git a/proxy/http3/Http3HeaderFramer.cc b/proxy/http3/Http3HeaderFramer.cc
index 079c5d274..9cd17a17c 100644
--- a/proxy/http3/Http3HeaderFramer.cc
+++ b/proxy/http3/Http3HeaderFramer.cc
@@ -26,7 +26,6 @@
 #include "I_VIO.h"
 
 #include "HTTP.h"
-#include "HTTP2.h"
 
 #include "Http3Frame.h"
 #include "Http3Transaction.h"
@@ -73,12 +72,6 @@ Http3HeaderFramer::is_done() const
   return this->_sent_all_data;
 }
 
-void
-Http3HeaderFramer::_convert_header_from_1_1_to_3(HTTPHdr *hdrs)
-{
-  http2_convert_header_from_1_1_to_2(hdrs);
-}
-
 void
 Http3HeaderFramer::_generate_header_block()
 {
@@ -87,19 +80,17 @@ Http3HeaderFramer::_generate_header_block()
   ParseResult parse_result = PARSE_RESULT_ERROR;
 
   if (this->_transaction->direction() == NET_VCONNECTION_OUT) {
-    this->_header.create(HTTP_TYPE_REQUEST);
-    http2_init_pseudo_headers(this->_header);
+    this->_header.create(HTTP_TYPE_REQUEST, HTTP_3_0);
     parse_result = this->_header.parse_req(&this->_http_parser, this->_source_vio->get_reader(), &bytes_used, false);
   } else {
-    this->_header.create(HTTP_TYPE_RESPONSE);
-    http2_init_pseudo_headers(this->_header);
+    this->_header.create(HTTP_TYPE_RESPONSE, HTTP_3_0);
     parse_result = this->_header.parse_resp(&this->_http_parser, this->_source_vio->get_reader(), &bytes_used, false);
   }
   this->_source_vio->ndone += bytes_used;
 
   switch (parse_result) {
   case PARSE_RESULT_DONE: {
-    this->_convert_header_from_1_1_to_3(&this->_header);
+    this->_hvc.convert(this->_header, 1, 3);
 
     this->_header_block        = new_MIOBuffer(BUFFER_SIZE_INDEX_32K);
     this->_header_block_reader = this->_header_block->alloc_reader();
diff --git a/proxy/http3/Http3HeaderFramer.h b/proxy/http3/Http3HeaderFramer.h
index 9559edba4..98f77ab3c 100644
--- a/proxy/http3/Http3HeaderFramer.h
+++ b/proxy/http3/Http3HeaderFramer.h
@@ -24,6 +24,7 @@
 #pragma once
 
 #include "hdrs/HTTP.h"
+#include "hdrs/VersionConverter.h"
 
 #include "QPACK.h"
 
@@ -54,7 +55,7 @@ private:
   bool _sent_all_data                  = false;
   HTTPParser _http_parser;
   HTTPHdr _header;
+  VersionConverter _hvc;
 
-  void _convert_header_from_1_1_to_3(HTTPHdr *hdrs);
   void _generate_header_block();
 };
diff --git a/proxy/http3/Http3HeaderVIOAdaptor.cc b/proxy/http3/Http3HeaderVIOAdaptor.cc
index 0d246c5ee..e99357abf 100644
--- a/proxy/http3/Http3HeaderVIOAdaptor.cc
+++ b/proxy/http3/Http3HeaderVIOAdaptor.cc
@@ -25,14 +25,6 @@
 
 #include "I_VIO.h"
 #include "HTTP.h"
-#include "HTTP2.h"
-
-// Constant strings for pseudo headers
-const char *HTTP3_VALUE_SCHEME    = ":scheme";
-const char *HTTP3_VALUE_AUTHORITY = ":authority";
-
-const unsigned HTTP3_LEN_SCHEME    = countof(":scheme") - 1;
-const unsigned HTTP3_LEN_AUTHORITY = countof(":authority") - 1;
 
 Http3HeaderVIOAdaptor::Http3HeaderVIOAdaptor(VIO *sink, HTTPType http_type, QPACK *qpack, uint64_t stream_id)
   : _sink_vio(sink), _qpack(qpack), _stream_id(stream_id)
@@ -103,8 +95,8 @@ Http3HeaderVIOAdaptor::event_handler(int event, Event *data)
 int
 Http3HeaderVIOAdaptor::_on_qpack_decode_complete()
 {
-  ParseResult res = this->_convert_header_from_3_to_1_1(&this->_header);
-  if (res == PARSE_RESULT_ERROR) {
+  int res = this->_hvc.convert(this->_header, 3, 1);
+  if (res != 0) {
     Debug("http3", "PARSE_RESULT_ERROR");
     return -1;
   }
@@ -147,29 +139,3 @@ Http3HeaderVIOAdaptor::_on_qpack_decode_complete()
   this->_is_complete = true;
   return 1;
 }
-
-ParseResult
-Http3HeaderVIOAdaptor::_convert_header_from_3_to_1_1(HTTPHdr *hdrs)
-{
-  // TODO: do HTTP/3 specific convert, if there
-
-  if (http_hdr_type_get(hdrs->m_http) == HTTP_TYPE_REQUEST) {
-    // Dirty hack to bypass checks
-    MIMEField *field;
-    if ((field = hdrs->field_find(HTTP3_VALUE_SCHEME, HTTP3_LEN_SCHEME)) == nullptr) {
-      char value_s[]          = "https";
-      MIMEField *scheme_field = hdrs->field_create(HTTP3_VALUE_SCHEME, HTTP3_LEN_SCHEME);
-      scheme_field->value_set(hdrs->m_heap, hdrs->m_mime, value_s, sizeof(value_s) - 1);
-      hdrs->field_attach(scheme_field);
-    }
-
-    if ((field = hdrs->field_find(HTTP3_VALUE_AUTHORITY, HTTP3_LEN_AUTHORITY)) == nullptr) {
-      char value_a[]             = "localhost";
-      MIMEField *authority_field = hdrs->field_create(HTTP3_VALUE_AUTHORITY, HTTP3_LEN_AUTHORITY);
-      authority_field->value_set(hdrs->m_heap, hdrs->m_mime, value_a, sizeof(value_a) - 1);
-      hdrs->field_attach(authority_field);
-    }
-  }
-
-  return http2_convert_header_from_2_to_1_1(hdrs);
-}
diff --git a/proxy/http3/Http3HeaderVIOAdaptor.h b/proxy/http3/Http3HeaderVIOAdaptor.h
index e4cc0ff15..341981180 100644
--- a/proxy/http3/Http3HeaderVIOAdaptor.h
+++ b/proxy/http3/Http3HeaderVIOAdaptor.h
@@ -24,7 +24,7 @@
 #pragma once
 
 #include "QPACK.h"
-
+#include "hdrs/VersionConverter.h"
 #include "Http3FrameHandler.h"
 
 class Http3HeaderVIOAdaptor : public Continuation, public Http3FrameHandler
@@ -47,7 +47,7 @@ private:
   bool _is_complete   = false;
 
   HTTPHdr _header; ///< HTTP header buffer for decoding
+  VersionConverter _hvc;
 
   int _on_qpack_decode_complete();
-  ParseResult _convert_header_from_3_to_1_1(HTTPHdr *hdr);
 };
diff --git a/proxy/http3/Makefile.am b/proxy/http3/Makefile.am
index 7a0ea023a..d48cacc15 100644
--- a/proxy/http3/Makefile.am
+++ b/proxy/http3/Makefile.am
@@ -27,7 +27,6 @@ AM_CPPFLAGS += \
   -I$(abs_top_srcdir)/mgmt/utils \
   -I$(abs_top_srcdir)/proxy \
   -I$(abs_top_srcdir)/proxy/http \
-  -I$(abs_top_srcdir)/proxy/http2 \
   -I$(abs_top_srcdir)/proxy/hdrs \
   -I$(abs_top_srcdir)/proxy/shared \
   -I$(abs_top_srcdir)/proxy/http/remap \
@@ -77,7 +76,6 @@ test_LDADD = \
   $(top_builddir)/mgmt/libmgmt_p.la \
   $(top_builddir)/src/tscore/libtscore.la \
   $(top_builddir)/src/tscpp/util/libtscpputil.la \
-  $(top_builddir)/proxy/http2/libhttp2.a \
   $(top_builddir)/proxy/hdrs/libhdrs.a \
   $(top_builddir)/proxy/shared/libUglyLogStubs.a \
   @LIBPCRE@ \