You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by am...@apache.org on 2018/07/12 13:57:53 UTC

[trafficserver] branch master updated: Refresh upstream connection throttling. Reduce lock contention, add maximum count, rate limit alerts.

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

amc 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 46acdc8  Refresh upstream connection throttling. Reduce lock contention, add maximum count, rate limit alerts.
46acdc8 is described below

commit 46acdc85f405ba480356952ff2979d7385a6c591
Author: Alan M. Carroll <am...@apache.org>
AuthorDate: Fri Jun 8 10:51:34 2018 -0500

    Refresh upstream connection throttling.
    Reduce lock contention, add maximum count, rate limit alerts.
---
 doc/admin-guide/files/records.config.en.rst        |  80 ++-
 .../statistics/core/http-connection.en.rst         |   2 +-
 .../api/functions/TSHttpOverridableConfig.en.rst   |   4 +-
 .../api/types/TSOverridableConfigKey.en.rst        |   4 +-
 lib/ts/apidefs.h.in                                |  16 +-
 mgmt/MgmtDefs.h                                    |  33 +-
 mgmt/RecordsConfig.cc                              |  12 +-
 plugins/lua/ts_lua_http_config.c                   |   8 +-
 proxy/http/HttpConfig.cc                           |  34 +-
 proxy/http/HttpConfig.h                            |   9 +-
 proxy/http/HttpConnectionCount.cc                  | 424 +++++++++++++++-
 proxy/http/HttpConnectionCount.h                   | 558 +++++++++++++--------
 proxy/http/HttpDebugNames.cc                       |   2 +
 proxy/http/HttpProxyAPIEnums.h                     |  10 +
 proxy/http/HttpSM.cc                               | 130 +++--
 proxy/http/HttpServerSession.cc                    |  56 ++-
 proxy/http/HttpServerSession.h                     |  58 +--
 proxy/http/HttpSessionManager.cc                   |   5 +-
 proxy/http/HttpTransact.cc                         |  17 +-
 proxy/http/HttpTransact.h                          |  10 +-
 src/traffic_server/InkAPI.cc                       | 393 ++++++++-------
 src/traffic_server/InkAPITest.cc                   | 265 +++++-----
 22 files changed, 1434 insertions(+), 696 deletions(-)

diff --git a/doc/admin-guide/files/records.config.en.rst b/doc/admin-guide/files/records.config.en.rst
index 65749f9..a474b6f 100644
--- a/doc/admin-guide/files/records.config.en.rst
+++ b/doc/admin-guide/files/records.config.en.rst
@@ -1350,28 +1350,84 @@ Origin Server Connect Attempts
    way up to ts:cv:`proxy.config.net.connections_throttle` connections, which
    in turn can starve incoming requests from available connections.
 
-.. ts:cv:: CONFIG proxy.config.http.origin_max_connections INT 0
+.. ts:cv:: CONFIG proxy.config.http.per_server.connection.max INT 0
    :reloadable:
    :overridable:
 
-   Limits the number of socket connections per origin server to the value specified. To enable, set to one (``1``).
+   Set a limit for the number of concurrent connections to an upstream server group. A value of
+   ``0`` disables checking. If a transaction attempts to connect to a group which already has the
+   maximum number of concurrent connections the transaction either rechecks after a delay or a 503
+   (``HTTP_STATUS_SERVICE_UNAVAILABLE``) error response is sent to the user agent. To configure
 
-.. ts:cv:: CONFIG proxy.config.http.origin_max_connections_queue INT -1
+   Number of transactions that can be delayed concurrently
+      See :ts:cv:`proxy.config.http.per_server.connection.queue_size`.
+
+   How long to delay before rechecking
+      See :ts:cv:`proxy.config.http.per_server.connection.queue_delay`.
+
+   Upstream server group definition
+      See :ts:cv:`proxy.config.http.per_server.connection.match`.
+
+   Frequency of alerts
+      See :ts:cv:`proxy.config.http.per_server.connection.alert_delay`.
+
+.. ts:cv:: CONFIG proxy.config.http.per_server.connection.match STRING ip
    :reloadable:
    :overridable:
 
-   Limits the number of requests to be queued when the :ts:cv:`proxy.config.http.origin_max_connections` is reached.
-   When disabled (``-1``) requests are will wait indefinitely for an available connection. When set to ``0`` all
-   requests past the :ts:cv:`proxy.config.http.origin_max_connections` will immediately fail. When set to ``>0``
-   ATS will queue that many requests to go to the origin, any additional requests past the limit will immediately fail.
+   Control the definition of an upstream server group for
+   :ts:cv:`proxy.config.http.per_server.connection.max`. This must be one of the following keywords.
+
+   ip
+      Group by IP address. Each IP address is a group.
+
+   port
+      Group by IP address and port. Each distinct IP address and port pair is a group.
+
+   host
+      Group by host name. The host name is the post remap FQDN used to resolve the upstream
+      address.
+
+   both
+      Group by IP address, port, and host name. Each distinct combination is a group.
+
+   To disable upstream server grouping, set :ts:cv:`proxy.config.http.per_server.connection.max` to ``0``.
+
+.. ts:cv:: CONFIG proxy.config.http.per_server.connection.queue_size INT 0
+   :reloadable:
+
+   Controls the number of transactions that can be waiting on an upstream server group.
+
+   ``-1``
+      Unlimited.
+
+   ``0``
+      Never wait. If the connection maximum has been reached immediately respond with an error.
+
+   A positive number
+      If there are less than this many waiting transactions, delay this transaction and try again. Otherwise respond immediately with an error.
+
+.. ts:cv:: CONFIG proxy.config.http.per_server.connection.queue_delay INT 100
+   :reloadable:
+   :units: milliseconds
+
+   If a transaction is delayed due to too many connections in an upstream server group, delay this amount of time before checking again.
+
+.. ts:cv:: CONFIG proxy.config.http.per_server.connection.alert_delay INT 60
+   :reloadable:
+   :units: seconds
+
+   Throttle alerts per upstream server group to be no more often than this many seconds. Summary
+   data is provided per alert to allow log scrubbing to generate accurate data.
 
-.. ts:cv:: CONFIG proxy.config.http.origin_min_keep_alive_connections INT 0
+.. ts:cv:: CONFIG proxy.config.http.per_server.min_keep_alive_connections INT 0
    :reloadable:
 
-   As connection to an origin server are opened, keep at least 'n' number of connections open to that origin, even if
-   the connection isn't used for a long time period. Useful when the origin supports keep-alive, removing the time
-   needed to set up a new connection from
-   the next request at the expense of added (inactive) connections. To enable, set to one (``1``).
+   Set a target for the minimum number of active connections to an upstream server group. When an
+   outbound connection is in keep alive state and the inactivity timer expires, if there are fewer
+   than this many connections in the group a new connection the timer is reset instead of closing
+   the connection. Useful when the origin supports keep-alive, removing the time needed to set up a
+   new connection from the next request at the expense of added (inactive) connections.
 
 .. ts:cv:: CONFIG proxy.config.http.connect_attempts_rr_retries INT 3
    :reloadable:
diff --git a/doc/admin-guide/monitoring/statistics/core/http-connection.en.rst b/doc/admin-guide/monitoring/statistics/core/http-connection.en.rst
index 2c55076..c92f1ab 100644
--- a/doc/admin-guide/monitoring/statistics/core/http-connection.en.rst
+++ b/doc/admin-guide/monitoring/statistics/core/http-connection.en.rst
@@ -131,4 +131,4 @@ HTTP Connection
 .. ts:stat:: global proxy.process.http.origin_connections_throttled_out integer
    :type: counter
 
-This tracks the number of origin connections denied due to being over the :ts:cv:`proxy.config.http.origin_max_connections` limit.
+   This tracks the number of origin connections denied due to being over the :ts:cv:`proxy.config.http.per_server.connection.max` limit.
diff --git a/doc/developer-guide/api/functions/TSHttpOverridableConfig.en.rst b/doc/developer-guide/api/functions/TSHttpOverridableConfig.en.rst
index aa352b4..125e111 100644
--- a/doc/developer-guide/api/functions/TSHttpOverridableConfig.en.rst
+++ b/doc/developer-guide/api/functions/TSHttpOverridableConfig.en.rst
@@ -131,8 +131,6 @@ TSOverridableConfigKey Value                                        Configuratio
 :c:macro:`TS_CONFIG_HTTP_NEGATIVE_REVALIDATING_ENABLED`             :ts:cv:`proxy.config.http.negative_revalidating_enabled`
 :c:macro:`TS_CONFIG_HTTP_NEGATIVE_REVALIDATING_LIFETIME`            :ts:cv:`proxy.config.http.negative_revalidating_lifetime`
 :c:macro:`TS_CONFIG_HTTP_NUMBER_OF_REDIRECTIONS`                    :ts:cv:`proxy.config.http.number_of_redirections`
-:c:macro:`TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS`                    :ts:cv:`proxy.config.http.origin_max_connections`
-:c:macro:`TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS_QUEUE`              :ts:cv:`proxy.config.http.origin_max_connections_queue`
 :c:macro:`TS_CONFIG_HTTP_PARENT_PROXY_TOTAL_CONNECT_ATTEMPTS`       :ts:cv:`proxy.config.http.parent_proxy.total_connect_attempts`
 :c:macro:`TS_CONFIG_PARENT_FAILURES_UPDATE_HOSTDB`                  :ts:cv:`proxy.config.http.parent_proxy.mark_down_hostdb`
 :c:macro:`TS_CONFIG_HTTP_POST_CHECK_CONTENT_LENGTH_ENABLED`         :ts:cv:`proxy.config.http.post.check.content_length.enabled`
@@ -171,6 +169,8 @@ TSOverridableConfigKey Value                                        Configuratio
 :c:macro:`TS_CONFIG_HTTP_NORMALIZE_AE`                              :ts:cv:`proxy.config.http.normalize_ae`
 :c:macro:`TS_CONFIG_HTTP_ALLOW_MULTI_RANGE`                         :ts:cv:`proxy.config.http.allow_multi_range`
 :c:macro:`TS_CONFIG_HTTP_ALLOW_HALF_OPEN`                           :ts:cv:`proxy.config.http.allow_half_open`
+:c:macro:`TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX`                 :ts:cv:`proxy.config.http.per_server.connection.max`
+:c:macro:`TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH`               :ts:cv:`proxy.config.http.per_server.connection.match`
 ==================================================================  ====================================================================
 
 Examples
diff --git a/doc/developer-guide/api/types/TSOverridableConfigKey.en.rst b/doc/developer-guide/api/types/TSOverridableConfigKey.en.rst
index a2d7cea..9da76fb 100644
--- a/doc/developer-guide/api/types/TSOverridableConfigKey.en.rst
+++ b/doc/developer-guide/api/types/TSOverridableConfigKey.en.rst
@@ -148,6 +148,8 @@ Enumeration Members
    .. c:macro:: TS_CONFIG_HTTP_INSERT_FORWARDED
    .. c:macro:: TS_CONFIG_HTTP_ALLOW_MULTI_RANGE
    .. c:macro:: TS_CONFIG_HTTP_ALLOW_HALF_OPEN
-   
+   .. c:macro:: TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX
+   .. c:macro:: TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH
+
 Description
 ===========
diff --git a/lib/ts/apidefs.h.in b/lib/ts/apidefs.h.in
index 84a1aee..5b0a953 100644
--- a/lib/ts/apidefs.h.in
+++ b/lib/ts/apidefs.h.in
@@ -477,7 +477,8 @@ typedef enum {
   TS_SRVSTATE_OPEN_RAW_ERROR,
   TS_SRVSTATE_PARSE_ERROR,
   TS_SRVSTATE_TRANSACTION_COMPLETE,
-  TS_SRVSTATE_PARENT_RETRY
+  TS_SRVSTATE_PARENT_RETRY,
+  TS_SRVSTATE_OUTBOUND_CONGESTION
 } TSServerState;
 
 typedef enum {
@@ -578,6 +579,15 @@ typedef enum {
   TS_SERVER_SESSION_SHARING_POOL_GLOBAL,
   TS_SERVER_SESSION_SHARING_POOL_THREAD,
 } TSServerSessionSharingPoolType;
+
+/// Values for per server outbound connection tracking group definition.
+/// See proxy.config.http.per_server.match
+typedef enum {
+  TS_SERVER_OUTBOUND_MATCH_IP,
+  TS_SERVER_OUTBOUND_MATCH_PORT,
+  TS_SERVER_OUTBOUND_MATCH_HOST,
+  TS_SERVER_OUTBOUND_MATCH_BOTH
+} TSOutboundConnectionMatchType;
 #endif
 
 /* librecords types */
@@ -696,7 +706,6 @@ typedef enum {
   TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN,
   TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT,
   TS_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT,
-  TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS,
   TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES,
   TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER,
   TS_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES,
@@ -739,7 +748,6 @@ typedef enum {
   TS_CONFIG_HTTP_CACHE_MAX_OPEN_WRITE_RETRIES,
   TS_CONFIG_HTTP_REDIRECT_USE_ORIG_CACHE_KEY,
   TS_CONFIG_HTTP_ATTACH_SERVER_SESSION_TO_CLIENT,
-  TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS_QUEUE,
   TS_CONFIG_WEBSOCKET_NO_ACTIVITY_TIMEOUT,
   TS_CONFIG_WEBSOCKET_ACTIVE_TIMEOUT,
   TS_CONFIG_HTTP_UNCACHEABLE_REQUESTS_BYPASS_PARENT,
@@ -768,6 +776,8 @@ typedef enum {
   TS_CONFIG_HTTP_ALLOW_MULTI_RANGE,
   TS_CONFIG_HTTP_REQUEST_BUFFER_ENABLED,
   TS_CONFIG_HTTP_ALLOW_HALF_OPEN,
+  TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX,
+  TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH,
   TS_CONFIG_LAST_ENTRY
 } TSOverridableConfigKey;
 
diff --git a/mgmt/MgmtDefs.h b/mgmt/MgmtDefs.h
index 64b7109..7dc3f81 100644
--- a/mgmt/MgmtDefs.h
+++ b/mgmt/MgmtDefs.h
@@ -36,8 +36,10 @@
 /*
  * Type definitions.
  */
+#include <functional>
+#include <string_view>
+
 #include "ts/ink_defs.h"
-//#include "ts/ink_hrtime.h"
 
 typedef int64_t MgmtIntCounter;
 typedef int64_t MgmtInt;
@@ -60,4 +62,33 @@ typedef enum {
  */
 typedef void *(*MgmtCallback)(void *opaque_cb_data, char *data_raw, int data_len);
 
+//-------------------------------------------------------------------------
+// API conversion functions.
+//-------------------------------------------------------------------------
+/** Conversion functions to and from an aribrary type and Management types.
+ *
+ * A type that wants to support conversion in the TS API should create a static instance of this
+ * class and fill in the appropriate members. The TS API set/get functions can then check for a
+ * @c nullptr to see if the conversion is supported and if so, call a function to do that. The
+ * @c void* argument is a raw pointer to the typed object. For instance, if this is for transaction
+ * overrides the pointer will be to the member in the transaction override configuration structure.
+ * Support for the management types is built in, this is only needed for types that aren't defined
+ * in this header.
+ */
+struct MgmtConverter {
+  // MgmtInt conversions.
+  std::function<MgmtInt(void *)> get_int{nullptr};
+  std::function<void(void *, MgmtInt)> set_int{nullptr};
+
+  // MgmtFloat conversions.
+  std::function<MgmtFloat(void *)> get_float{nullptr};
+  std::function<void(void *, MgmtFloat)> set_float{nullptr};
+
+  // MgmtString conversions.
+  // This is a bit different because it takes std::string_view instead of MgmtString but that's
+  // worth the difference.
+  std::function<std::string_view(void *)> get_string{nullptr};
+  std::function<void(void *, std::string_view)> set_string{nullptr};
+};
+
 #define LM_CONNECTION_SERVER "processerver.sock"
diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
index 9d1198c..0ba68b0 100644
--- a/mgmt/RecordsConfig.cc
+++ b/mgmt/RecordsConfig.cc
@@ -381,11 +381,17 @@ static const RecordElement RecordsConfig[] =
   ,
   {RECT_CONFIG, "proxy.config.http.server_tcp_init_cwnd", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_STR, "[0-16]", RECA_NULL}
   ,
-  {RECT_CONFIG, "proxy.config.http.origin_max_connections", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+  {RECT_CONFIG, "proxy.config.http.per_server.connection.max", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
   ,
-  {RECT_CONFIG, "proxy.config.http.origin_max_connections_queue", RECD_INT, "-1", RECU_DYNAMIC, RR_NULL, RECC_STR, "^-?[0-9]+$", RECA_NULL}
+  {RECT_CONFIG, "proxy.config.http.per_server.connection.match", RECD_STRING, "ip", RECU_DYNAMIC, RR_NULL, RECC_STR, "^(?:ip|host|both|none)$", RECA_NULL}
+        ,
+  {RECT_CONFIG, "proxy.config.http.per_server.connection.alert_delay", RECD_INT, "60", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+        ,
+  {RECT_CONFIG, "proxy.config.http.per_server.connection.queue_size", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_STR, "^-?[0-9]+$", RECA_NULL}
     ,
-  {RECT_CONFIG, "proxy.config.http.origin_min_keep_alive_connections", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+  {RECT_CONFIG, "proxy.config.http.per_server.connection.queue_delay", RECD_INT, "100", RECU_DYNAMIC, RR_NULL, RECC_STR, "^-?[0-9]+$", RECA_NULL}
+        ,
+  {RECT_CONFIG, "proxy.config.http.per_server.min_keep_alive", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
   ,
   {RECT_CONFIG, "proxy.config.http.attach_server_session_to_client", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_INT, "[0-1]", RECA_NULL}
   ,
diff --git a/plugins/lua/ts_lua_http_config.c b/plugins/lua/ts_lua_http_config.c
index 6110f76..11f8cca 100644
--- a/plugins/lua/ts_lua_http_config.c
+++ b/plugins/lua/ts_lua_http_config.c
@@ -64,7 +64,8 @@ typedef enum {
   TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN       = TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN,
   TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT      = TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT,
   TS_LUA_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT           = TS_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT,
-  TS_LUA_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS                   = TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS,
+  TS_LUA_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX                = TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX,
+  TS_LUA_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH              = TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH,
   TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES             = TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES,
   TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER = TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER,
   TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES              = TS_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES,
@@ -108,7 +109,6 @@ typedef enum {
   TS_LUA_CONFIG_HTTP_CACHE_MAX_OPEN_WRITE_RETRIES             = TS_CONFIG_HTTP_CACHE_MAX_OPEN_WRITE_RETRIES,
   TS_LUA_CONFIG_HTTP_REDIRECT_USE_ORIG_CACHE_KEY              = TS_CONFIG_HTTP_REDIRECT_USE_ORIG_CACHE_KEY,
   TS_LUA_CONFIG_HTTP_ATTACH_SERVER_SESSION_TO_CLIENT          = TS_CONFIG_HTTP_ATTACH_SERVER_SESSION_TO_CLIENT,
-  TS_LUA_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS_QUEUE             = TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS_QUEUE,
   TS_LUA_CONFIG_WEBSOCKET_NO_ACTIVITY_TIMEOUT                 = TS_CONFIG_WEBSOCKET_NO_ACTIVITY_TIMEOUT,
   TS_LUA_CONFIG_WEBSOCKET_ACTIVE_TIMEOUT                      = TS_CONFIG_WEBSOCKET_ACTIVE_TIMEOUT,
   TS_LUA_CONFIG_HTTP_UNCACHEABLE_REQUESTS_BYPASS_PARENT       = TS_CONFIG_HTTP_UNCACHEABLE_REQUESTS_BYPASS_PARENT,
@@ -191,7 +191,6 @@ ts_lua_var_item ts_lua_http_config_vars[] = {
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT),
-  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES),
@@ -235,7 +234,6 @@ ts_lua_var_item ts_lua_http_config_vars[] = {
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_CACHE_MAX_OPEN_WRITE_RETRIES),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_REDIRECT_USE_ORIG_CACHE_KEY),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ATTACH_SERVER_SESSION_TO_CLIENT),
-  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS_QUEUE),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_WEBSOCKET_NO_ACTIVITY_TIMEOUT),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_WEBSOCKET_ACTIVE_TIMEOUT),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_UNCACHEABLE_REQUESTS_BYPASS_PARENT),
@@ -262,6 +260,8 @@ ts_lua_var_item ts_lua_http_config_vars[] = {
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ALLOW_MULTI_RANGE),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_REQUEST_BUFFER_ENABLED),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_ALLOW_HALF_OPEN),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX),
+  TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH),
   TS_LUA_MAKE_VAR_ITEM(TS_LUA_CONFIG_LAST_ENTRY),
 };
 
diff --git a/proxy/http/HttpConfig.cc b/proxy/http/HttpConfig.cc
index 97ea45e..1e742c1 100644
--- a/proxy/http/HttpConfig.cc
+++ b/proxy/http/HttpConfig.cc
@@ -929,9 +929,7 @@ HttpConfig::startup()
   HttpEstablishStaticConfigLongLong(c.server_max_connections, "proxy.config.http.server_max_connections");
   HttpEstablishStaticConfigLongLong(c.max_websocket_connections, "proxy.config.http.websocket.max_number_of_connections");
   HttpEstablishStaticConfigLongLong(c.oride.server_tcp_init_cwnd, "proxy.config.http.server_tcp_init_cwnd");
-  HttpEstablishStaticConfigLongLong(c.oride.origin_max_connections, "proxy.config.http.origin_max_connections");
-  HttpEstablishStaticConfigLongLong(c.oride.origin_max_connections_queue, "proxy.config.http.origin_max_connections_queue");
-  HttpEstablishStaticConfigLongLong(c.origin_min_keep_alive_connections, "proxy.config.http.origin_min_keep_alive_connections");
+  HttpEstablishStaticConfigLongLong(c.origin_min_keep_alive_connections, "proxy.config.http.per_server.min_keep_alive");
   HttpEstablishStaticConfigByte(c.oride.attach_server_session_to_client, "proxy.config.http.attach_server_session_to_client");
 
   HttpEstablishStaticConfigByte(c.disable_ssl_parenting, "proxy.local.http.parent_proxy.disable_connect_tunneling");
@@ -1171,6 +1169,8 @@ HttpConfig::startup()
   HttpEstablishStaticConfigLongLong(c.oride.number_of_redirections, "proxy.config.http.number_of_redirections");
   HttpEstablishStaticConfigLongLong(c.post_copy_size, "proxy.config.http.post_copy_size");
 
+  OutboundConnTrack::config_init(&c.outbound_conntrack, &c.oride.outbound_conntrack);
+
   http_config_cont->handleEvent(EVENT_NONE, nullptr);
 
   return;
@@ -1208,23 +1208,25 @@ HttpConfig::reconfigure()
   params->disable_ssl_parenting        = INT_TO_BOOL(m_master.disable_ssl_parenting);
   params->oride.forward_connect_method = INT_TO_BOOL(m_master.oride.forward_connect_method);
 
-  params->server_max_connections             = m_master.server_max_connections;
-  params->max_websocket_connections          = m_master.max_websocket_connections;
-  params->oride.server_tcp_init_cwnd         = m_master.oride.server_tcp_init_cwnd;
-  params->oride.origin_max_connections       = m_master.oride.origin_max_connections;
-  params->oride.origin_max_connections_queue = m_master.oride.origin_max_connections_queue;
-  // if origin_max_connections_queue is set without max_connections, it is meaningless, so we'll warn
-  if (params->oride.origin_max_connections_queue > 0 &&
-      !(params->oride.origin_max_connections || params->origin_min_keep_alive_connections)) {
-    Warning("origin_max_connections_queue is set, but neither origin_max_connections nor origin_min_keep_alive_connections are "
-            "set, please correct your records.config");
+  params->server_max_connections     = m_master.server_max_connections;
+  params->max_websocket_connections  = m_master.max_websocket_connections;
+  params->oride.server_tcp_init_cwnd = m_master.oride.server_tcp_init_cwnd;
+  params->oride.outbound_conntrack   = m_master.oride.outbound_conntrack;
+  // If queuing for outbound connection tracking is enabled without enabling max connections, it is meaningless, so we'll warn
+  if (params->outbound_conntrack.queue_size > 0 &&
+      !(params->oride.outbound_conntrack.max > 0 || params->origin_min_keep_alive_connections)) {
+    Warning("'%s' is set, but neither '%s' nor 'per_server.min_keep_alive_connections' are "
+            "set, please correct your records.config",
+            OutboundConnTrack::CONFIG_VAR_QUEUE_SIZE.data(), OutboundConnTrack::CONFIG_VAR_MAX.data());
   }
   params->origin_min_keep_alive_connections     = m_master.origin_min_keep_alive_connections;
   params->oride.attach_server_session_to_client = m_master.oride.attach_server_session_to_client;
 
-  if (params->oride.origin_max_connections && params->oride.origin_max_connections < params->origin_min_keep_alive_connections) {
-    Warning("origin_max_connections < origin_min_keep_alive_connections, setting min=max , please correct your records.config");
-    params->origin_min_keep_alive_connections = params->oride.origin_max_connections;
+  if (params->oride.outbound_conntrack.max > 0 &&
+      params->oride.outbound_conntrack.max < params->origin_min_keep_alive_connections) {
+    Warning("'%s' < origin_min_keep_alive_connections, setting min=max , please correct your records.config",
+            OutboundConnTrack::CONFIG_VAR_MAX.data());
+    params->origin_min_keep_alive_connections = params->oride.outbound_conntrack.max;
   }
 
   params->oride.insert_request_via_string   = m_master.oride.insert_request_via_string;
diff --git a/proxy/http/HttpConfig.h b/proxy/http/HttpConfig.h
index 3518dce..d289699 100644
--- a/proxy/http/HttpConfig.h
+++ b/proxy/http/HttpConfig.h
@@ -49,6 +49,7 @@
 #include "HttpProxyAPIEnums.h"
 #include "ProxyConfig.h"
 #include "P_RecProcess.h"
+#include "HttpConnectionCount.h"
 
 /* Instead of enumerating the stats in DynamicStats.h, each module needs
    to enumerate its stats separately and register them with librecords
@@ -478,8 +479,6 @@ struct OverridableHttpConfigParams {
       transaction_active_timeout_in(900),
       websocket_active_timeout(3600),
       websocket_inactive_timeout(600),
-      origin_max_connections(0),
-      origin_max_connections_queue(0),
       connect_attempts_max_retries(0),
       connect_attempts_max_retries_dead_server(3),
       connect_attempts_rr_retries(3),
@@ -694,8 +693,6 @@ struct OverridableHttpConfigParams {
   MgmtInt transaction_active_timeout_in;
   MgmtInt websocket_active_timeout;
   MgmtInt websocket_inactive_timeout;
-  MgmtInt origin_max_connections;
-  MgmtInt origin_max_connections_queue;
 
   ////////////////////////////////////
   // origin server connect attempts //
@@ -736,6 +733,8 @@ struct OverridableHttpConfigParams {
   MgmtInt default_buffer_water_mark;
   MgmtInt slow_log_threshold;
 
+  OutboundConnTrack::TxnConfig outbound_conntrack;
+
   ///////////////////////////////////////////////////////////////////
   // Server header                                                 //
   ///////////////////////////////////////////////////////////////////
@@ -857,6 +856,8 @@ public:
 
   MgmtByte server_session_sharing_pool = TS_SERVER_SESSION_SHARING_POOL_THREAD;
 
+  OutboundConnTrack::GlobalConfig outbound_conntrack;
+
   // All the overridable configurations goes into this class member, but they
   // are not copied over until needed ("lazy").
   OverridableHttpConfigParams oride;
diff --git a/proxy/http/HttpConnectionCount.cc b/proxy/http/HttpConnectionCount.cc
index 015310a..ca2f495 100644
--- a/proxy/http/HttpConnectionCount.cc
+++ b/proxy/http/HttpConnectionCount.cc
@@ -1,6 +1,6 @@
 /** @file
 
-  A brief file description
+  Outbound connection tracking support.
 
   @section license License
 
@@ -21,43 +21,304 @@
   limitations under the License.
  */
 
+#include <algorithm>
+#include <records/P_RecDefs.h>
 #include "HttpConnectionCount.h"
+#include <ts/bwf_std_format.h>
+#include <ts/BufferWriter.h>
 
-ConnectionCount ConnectionCount::_connectionCount;
-ConnectionCountQueue ConnectionCountQueue::_connectionCount;
+using namespace std::literals;
+
+OutboundConnTrack::Imp OutboundConnTrack::_imp;
+
+OutboundConnTrack::GlobalConfig *OutboundConnTrack::_global_config{nullptr};
+
+const MgmtConverter OutboundConnTrack::MAX_CONV{
+  [](void *data) -> MgmtInt { return static_cast<MgmtInt>(*static_cast<decltype(TxnConfig::max) *>(data)); },
+  [](void *data, MgmtInt i) -> void { *static_cast<decltype(TxnConfig::max) *>(data) = static_cast<decltype(TxnConfig::max)>(i); },
+  nullptr,
+  nullptr,
+  nullptr,
+  nullptr};
+
+// Do integer and string conversions.
+const MgmtConverter OutboundConnTrack::MATCH_CONV{
+  [](void *data) -> MgmtInt { return static_cast<MgmtInt>(*static_cast<decltype(TxnConfig::match) *>(data)); },
+  [](void *data, MgmtInt i) -> void {
+    // Problem - the InkAPITest requires being able to set an arbitrary value, so this can either
+    // correctly clamp or pass the regression tests. Currently it passes the tests.
+    //    *static_cast<decltype(TxnConfig::match) *>(data) = std::clamp(static_cast<decltype(TxnConfig::match)>(i), MATCH_IP,
+    //    MATCH_BOTH);
+    *static_cast<decltype(TxnConfig::match) *>(data) = static_cast<decltype(TxnConfig::match)>(i);
+  },
+  nullptr,
+  nullptr,
+  [](void *data) -> std::string_view {
+    auto t = *static_cast<OutboundConnTrack::MatchType *>(data);
+    return t < 0 || t > OutboundConnTrack::MATCH_BOTH ? "Invalid"sv : OutboundConnTrack::MATCH_TYPE_NAME[t];
+  },
+  [](void *data, std::string_view src) -> void {
+    OutboundConnTrack::MatchType t;
+    if (OutboundConnTrack::lookup_match_type(src, t)) {
+      *static_cast<OutboundConnTrack::MatchType *>(data) = t;
+    } else {
+      OutboundConnTrack::Warning_Bad_Match_Type(src);
+    }
+  }};
+
+const std::array<std::string_view, static_cast<int>(OutboundConnTrack::MATCH_BOTH) + 1> OutboundConnTrack::MATCH_TYPE_NAME{
+  {"ip"sv, "port"sv, "host"sv, "both"sv}};
+
+// Make sure the clock is millisecond resolution or finer.
+static_assert(OutboundConnTrack::Group::Clock::period::num == 1);
+static_assert(OutboundConnTrack::Group::Clock::period::den >= 1000);
+
+// Configuration callback functions.
+namespace
+{
+int
+Config_Update_Conntrack_Max(const char *name, RecDataT dtype, RecData data, void *cookie)
+{
+  auto config = static_cast<OutboundConnTrack::TxnConfig *>(cookie);
+
+  if (RECD_INT == dtype) {
+    config->max = data.rec_int;
+  }
+  return REC_ERR_OKAY;
+}
+
+int
+Config_Update_Conntrack_Queue_Size(const char *name, RecDataT dtype, RecData data, void *cookie)
+{
+  auto config = static_cast<OutboundConnTrack::GlobalConfig *>(cookie);
+
+  if (RECD_INT == dtype) {
+    config->queue_size = data.rec_int;
+  }
+  return REC_ERR_OKAY;
+}
+
+int
+Config_Update_Conntrack_Queue_Delay(const char *name, RecDataT dtype, RecData data, void *cookie)
+{
+  auto config = static_cast<OutboundConnTrack::GlobalConfig *>(cookie);
+
+  if (RECD_INT == dtype && data.rec_int > 0) {
+    config->queue_delay = std::chrono::milliseconds(data.rec_int);
+  }
+  return REC_ERR_OKAY;
+}
+
+int
+Config_Update_Conntrack_Match(const char *name, RecDataT dtype, RecData data, void *cookie)
+{
+  auto config = static_cast<OutboundConnTrack::TxnConfig *>(cookie);
+
+  if (RECD_STRING == dtype) {
+    OutboundConnTrack::MatchType match_type;
+    std::string_view tag{data.rec_string};
+    if (OutboundConnTrack::lookup_match_type(tag, match_type)) {
+      config->match = match_type;
+    } else {
+      OutboundConnTrack::Warning_Bad_Match_Type(tag);
+    }
+  } else {
+    Warning("Invalid type for '%s' - must be 'INT'", OutboundConnTrack::CONFIG_VAR_MATCH.data());
+  }
+  return REC_ERR_OKAY;
+}
+
+int
+Config_Update_Conntrack_Alert_Delay(const char *name, RecDataT dtype, RecData data, void *cookie)
+{
+  auto config = static_cast<OutboundConnTrack::GlobalConfig *>(cookie);
+
+  if (RECD_INT == dtype && data.rec_int >= 0) {
+    config->alert_delay = std::chrono::seconds(data.rec_int);
+  }
+  return REC_ERR_OKAY;
+}
+
+// Do the initial load of a configuration var by grabbing the raw value from the records data
+// and calling the update callback. This must be a function because that's how the records
+// interface works. Everything needed is already in the record @a r.
+void
+Load_Config_Var(RecRecord const *r, void *)
+{
+  for (auto cb = r->config_meta.update_cb_list; nullptr != cb; cb = cb->next) {
+    cb->update_cb(r->name, r->data_type, r->data, cb->update_cookie);
+  }
+}
+
+} // namespace
+
+void
+OutboundConnTrack::config_init(GlobalConfig *global, TxnConfig *txn)
+{
+  _global_config = global; // remember this for later retrieval.
+                           // Per transaction lookup must be done at call time because it changes.
+
+  RecRegisterConfigUpdateCb(CONFIG_VAR_MAX.data(), &Config_Update_Conntrack_Max, txn);
+  RecRegisterConfigUpdateCb(CONFIG_VAR_MATCH.data(), &Config_Update_Conntrack_Match, txn);
+  RecRegisterConfigUpdateCb(CONFIG_VAR_QUEUE_SIZE.data(), &Config_Update_Conntrack_Queue_Size, global);
+  RecRegisterConfigUpdateCb(CONFIG_VAR_QUEUE_DELAY.data(), &Config_Update_Conntrack_Queue_Delay, global);
+  RecRegisterConfigUpdateCb(CONFIG_VAR_ALERT_DELAY.data(), &Config_Update_Conntrack_Alert_Delay, global);
+
+  // Load 'em up by firing off the config update callback.
+  RecLookupRecord(CONFIG_VAR_MAX.data(), &Load_Config_Var, nullptr, true);
+  RecLookupRecord(CONFIG_VAR_MATCH.data(), &Load_Config_Var, nullptr, true);
+  RecLookupRecord(CONFIG_VAR_QUEUE_SIZE.data(), &Load_Config_Var, nullptr, true);
+  RecLookupRecord(CONFIG_VAR_QUEUE_DELAY.data(), &Load_Config_Var, nullptr, true);
+  RecLookupRecord(CONFIG_VAR_ALERT_DELAY.data(), &Load_Config_Var, nullptr, true);
+}
+
+OutboundConnTrack::TxnState
+OutboundConnTrack::obtain(TxnConfig const &txn_cnf, std::string_view fqdn, IpEndpoint const &addr)
+{
+  TxnState zret;
+  CryptoHash hash;
+  CryptoContext().hash_immediate(hash, fqdn.data(), fqdn.size());
+  Group::Key key{addr, hash, txn_cnf.match};
+  std::lock_guard<std::mutex> lock(_imp._mutex); // Table lock
+  auto loc = _imp._table.find(key);
+  if (loc.isValid()) {
+    zret._g = loc;
+  } else {
+    zret._g = new Group(key, fqdn);
+    _imp._table.insert(zret._g);
+  }
+  return zret;
+}
+
+bool
+OutboundConnTrack::Group::equal(const Key &lhs, const Key &rhs)
+{
+  bool zret = false;
+  if (lhs._match_type == rhs._match_type) {
+    switch (lhs._match_type) {
+    case MATCH_IP:
+      zret = ats_ip_addr_eq(&lhs._addr.sa, &rhs._addr.sa);
+      break;
+    case MATCH_PORT:
+      zret = ats_ip_addr_port_eq(&lhs._addr.sa, &rhs._addr.sa);
+      break;
+    case MATCH_HOST:
+      zret = lhs._hash == rhs._hash;
+      break;
+    case MATCH_BOTH:
+      zret = (lhs._hash == rhs._hash && ats_ip_addr_port_eq(&lhs._addr.sa, &rhs._addr.sa));
+      break;
+    }
+  }
+
+  if (is_debug_tag_set(DEBUG_TAG)) {
+    ts::LocalBufferWriter<256> w;
+    w.print("Comparing {} to {} -> {}\0", lhs, rhs, zret ? "match" : "fail");
+    Debug(DEBUG_TAG, "%s", w.data());
+  }
+
+  return zret;
+}
+
+bool
+OutboundConnTrack::Group::should_alert(std::time_t *lat)
+{
+  bool zret = false;
+  // This is a bit clunky because the goal is to store just the tick count as an atomic.
+  // Might check to see if an atomic time_point is really atomic and avoid this.
+  Ticker last_tick{_last_alert};                  // Load the most recent alert time in ticks.
+  TimePoint last{TimePoint::duration{last_tick}}; // Most recent alert time in a time_point.
+  TimePoint now = Clock::now();                   // Current time_point.
+  if (last + _global_config->alert_delay <= now) {
+    // it's been long enough, swap out our time for the last time. The winner of this swap
+    // does the actual alert, leaving its current time as the last alert time.
+    zret = _last_alert.compare_exchange_strong(last_tick, now.time_since_epoch().count());
+    if (zret && lat) {
+      *lat = Clock::to_time_t(last);
+    }
+  }
+  return zret;
+}
+
+std::time_t
+OutboundConnTrack::Group::get_last_alert_epoch_time() const
+{
+  return Clock::to_time_t(TimePoint{TimePoint::duration{Ticker{_last_alert}}});
+}
+
+void
+OutboundConnTrack::get(std::vector<Group const *> &groups)
+{
+  std::lock_guard<std::mutex> lock(_imp._mutex); // TABLE LOCK
+  groups.resize(0);
+  groups.reserve(_imp._table.count());
+  for (Group const &g : _imp._table) {
+    groups.push_back(&g);
+  }
+}
 
 std::string
-ConnectionCount::dumpToJSON()
+OutboundConnTrack::to_json_string()
 {
-  Vec<ConnAddr> keys;
-  ink_mutex_acquire(&_mutex);
-  _hostCount.get_keys(keys);
-  std::ostringstream oss;
-  oss << '{';
-  appendJSONPair(oss, "connectionCountSize", keys.n);
-  oss << ", \"connectionCountList\": [";
-  for (size_t i = 0; i < keys.n; i++) {
-    oss << '{';
+  std::string text;
+  size_t extent = 0;
+  static const ts::BWFormat header_fmt{R"({{"count": {}, "list": [
+)"};
+  static const ts::BWFormat item_fmt{
+    R"(  {{"type": "{}", "ip": "{}", "fqdn": "{}", "current": {}, "max": {}, "blocked": {}, "queued": {}, "alert": {}}},
+)"};
+  static const std::string_view trailer{" \n]}"};
 
-    appendJSONPair(oss, "ip", keys[i].getIpStr());
-    oss << ',';
+  static const auto printer = [](ts::BufferWriter &w, Group const *g) -> ts::BufferWriter & {
+    w.print(item_fmt, g->_match_type, g->_addr, g->_fqdn, g->_count.load(), g->_count_max.load(), g->_blocked.load(),
+            g->_rescheduled.load(), g->get_last_alert_epoch_time());
+    return w;
+  };
 
-    appendJSONPair(oss, "port", keys[i]._addr.host_order_port());
-    oss << ',';
+  ts::FixedBufferWriter null_bw{nullptr}; // Empty buffer for sizing work.
+  std::vector<Group const *> groups;
 
-    appendJSONPair(oss, "hostname_hash", keys[i].getHostnameHashStr());
-    oss << ',';
+  self_type::get(groups);
 
-    appendJSONPair(oss, "connection_count", _hostCount.get(keys[i]));
-    oss << "}";
+  null_bw.print(header_fmt, groups.size()).extent();
+  for (auto g : groups) {
+    printer(null_bw, g);
+  }
+  extent = null_bw.extent() + trailer.size() - 2; // 2 for the trailing comma newline that will get clipped.
 
-    if (i < keys.n - 1) {
-      oss << ',';
+  text.resize(extent);
+  ts::FixedBufferWriter w(const_cast<char *>(text.data()), text.size());
+  w.clip(trailer.size());
+  w.print(header_fmt, groups.size());
+  for (auto g : groups) {
+    printer(w, g);
+  }
+  w.extend(trailer.size());
+  w.write(trailer);
+  return text;
+}
+
+void
+OutboundConnTrack::dump(FILE *f)
+{
+  std::vector<Group const *> groups;
+
+  self_type::get(groups);
+
+  if (groups.size()) {
+    fprintf(f, "\nUpstream Connection Tracking\n%7s | %5s | %10s | %24s | %33s | %8s |\n", "Current", "Block", "Queue", "Address",
+            "Hostname Hash", "Match");
+    fprintf(f, "------|-------|---------|--------------------------|-----------------------------------|----------|\n");
+
+    for (Group const *g : groups) {
+      ts::LocalBufferWriter<128> w;
+      w.print("{:7} | {:5} | {:5} | {:24} | {:33} | {:8} |\n", g->_count.load(), g->_blocked.load(), g->_rescheduled.load(),
+              g->_addr, g->_hash, g->_match_type);
+      fwrite(w.data(), w.size(), 1, f);
     }
+
+    fprintf(f, "------|-------|-------|--------------------------|-----------------------------------|----------|\n");
   }
-  ink_mutex_release(&_mutex);
-  oss << "]}";
-  return oss.str();
 }
 
 struct ShowConnectionCount : public ShowCont {
@@ -65,7 +326,7 @@ struct ShowConnectionCount : public ShowCont {
   int
   showHandler(int event, Event *e)
   {
-    CHECK_SHOW(show(ConnectionCount::getInstance()->dumpToJSON().c_str()));
+    CHECK_SHOW(show(OutboundConnTrack::to_json_string().c_str()));
     return completeJson(event, e);
   }
 };
@@ -77,3 +338,112 @@ register_ShowConnectionCount(Continuation *c, HTTPHdr *h)
   this_ethread()->schedule_imm(s);
   return &s->action;
 }
+
+bool
+OutboundConnTrack::lookup_match_type(std::string_view tag, OutboundConnTrack::MatchType &type)
+{
+  // Search the array for the tag.
+  for (OutboundConnTrack::MatchType idx :
+       {OutboundConnTrack::MATCH_IP, OutboundConnTrack::MATCH_PORT, OutboundConnTrack::MATCH_HOST, OutboundConnTrack::MATCH_BOTH}) {
+    if (tag == MATCH_TYPE_NAME[idx]) {
+      type = idx;
+      return true;
+    }
+  }
+  return false;
+}
+
+void
+OutboundConnTrack::Warning_Bad_Match_Type(std::string_view tag)
+{
+  ts::LocalBufferWriter<256> w;
+  w.print("Invalid value '{}' for '{}' - must be one of", tag, CONFIG_VAR_MATCH);
+  for (auto n : MATCH_TYPE_NAME) {
+    w.write(" '"sv);
+    w.write(n);
+    w.write("',"sv);
+  }
+  w.auxBuffer()[-1] = '\0'; // clip trailing comma and null terminate.
+  Warning("%s", w.data());
+}
+
+void
+OutboundConnTrack::TxnState::Note_Unblocked(TxnConfig *config, int count, sockaddr const *addr)
+{
+  time_t lat; // last alert time (epoch seconds)
+
+  if ((_g->_blocked > 0 || _g->_rescheduled > 0) && _g->should_alert(&lat)) {
+    auto blocked     = _g->_blocked.exchange(0);
+    auto rescheduled = _g->_rescheduled.exchange(0);
+    ts::LocalBufferWriter<256> w;
+    w.print("upstream unblocked: [{}] count={} limit={} group=({}) blocked={} queued={} upstream={}\0",
+            ts::bwf::Date(lat, "%b %d %H:%M:%S"sv), count, config->max, *_g, blocked, rescheduled, addr);
+    Debug(DEBUG_TAG, "%s", w.data());
+    Note("%s", w.data());
+  }
+}
+
+void
+OutboundConnTrack::TxnState::Warn_Blocked(TxnConfig *config, int64_t sm_id, int count, sockaddr const *addr, char const *debug_tag)
+{
+  bool alert_p     = _g->should_alert();
+  auto blocked     = alert_p ? _g->_blocked.exchange(0) : _g->_blocked.load();
+  auto rescheduled = alert_p ? _g->_rescheduled.exchange(0) : _g->_rescheduled.load();
+
+  if (alert_p || debug_tag) {
+    ts::LocalBufferWriter<256> w;
+    w.print("[{}] too many connections: count={} limit={} group=({}) blocked={} queued={} upstream={}\0", sm_id, count, config->max,
+            *_g, blocked, rescheduled, addr);
+
+    if (debug_tag) {
+      Debug(debug_tag, "%s", w.data());
+    }
+    if (alert_p) {
+      Warning("%s", w.data());
+    }
+  }
+}
+
+namespace ts
+{
+BufferWriter &
+bwformat(BufferWriter &w, BWFSpec const &spec, OutboundConnTrack::Group::Key const &key)
+{
+  switch (key._match_type) {
+  case OutboundConnTrack::MATCH_BOTH:
+    w.print("{:s} {},{}", key._match_type, key._addr, key._hash);
+    break;
+  case OutboundConnTrack::MATCH_HOST:
+    w.print("{:s} {}", key._match_type, key._hash);
+    break;
+  case OutboundConnTrack::MATCH_PORT:
+    w.print("{:s} {}", key._match_type, key._addr);
+    break;
+  case OutboundConnTrack::MATCH_IP:
+    w.print("{:s} {::a}", key._match_type, key._addr);
+    break;
+  }
+  return w;
+}
+
+BufferWriter &
+bwformat(BufferWriter &w, BWFSpec const &spec, OutboundConnTrack::Group const &g)
+{
+  switch (g._match_type) {
+  case OutboundConnTrack::MATCH_BOTH:
+    w.print("{:s} {},{}", g._match_type, g._addr, g._fqdn);
+    break;
+  case OutboundConnTrack::MATCH_HOST:
+    w.print("{:s} {}", g._match_type, g._fqdn);
+    break;
+  case OutboundConnTrack::MATCH_PORT:
+    w.print("{:s} {}", g._match_type, g._addr);
+    break;
+  case OutboundConnTrack::MATCH_IP:
+    w.print("{:s} {::a}", g._match_type, g._addr);
+    break;
+  }
+  return w;
+}
+
+} // namespace ts
diff --git a/proxy/http/HttpConnectionCount.h b/proxy/http/HttpConnectionCount.h
index 9567710..2cd7731 100644
--- a/proxy/http/HttpConnectionCount.h
+++ b/proxy/http/HttpConnectionCount.h
@@ -21,237 +21,401 @@
   limitations under the License.
  */
 
-//
-#include "ts/ink_platform.h"
-#include "ts/ink_inet.h"
-#include "ts/ink_mutex.h"
-#include "ts/Map.h"
-#include "ts/Diags.h"
-#include "ts/CryptoHash.h"
-#include "ts/ink_config.h"
+#pragma once
+
+#include <string_view>
+#include <chrono>
+#include <atomic>
+#include <sstream>
+#include <tuple>
+#include <mutex>
+#include <ts/ink_platform.h>
+#include <ts/ink_config.h>
+#include <ts/ink_mutex.h>
+#include <ts/ink_inet.h>
+#include <ts/Map.h>
+#include <ts/Diags.h>
+#include <ts/CryptoHash.h>
+#include <ts/BufferWriterForward.h>
+#include <ts/TextView.h>
+#include <MgmtDefs.h>
 #include "HttpProxyAPIEnums.h"
 #include "Show.h"
-#include <sstream>
-
-#pragma once
 
 /**
- * Singleton class to keep track of the number of connections per host
+ * Singleton class to keep track of the number of outbound connnections.
+ *
+ * Outbound connections are divided in to equivalence classes (called "groups" here) based on the
+ * session matching setting. Tracking data is stored for each group.
  */
-class ConnectionCount
+class OutboundConnTrack
 {
+  using self_type = OutboundConnTrack; ///< Self reference type.
+
 public:
-  /**
-   * Static method to get the instance of the class
-   * @return Returns a pointer to the instance of the class
-   */
-  static ConnectionCount *
-  getInstance()
-  {
-    return &_connectionCount;
-  }
+  // Non-copyable.
+  OutboundConnTrack(const self_type &) = delete;
+  self_type &operator=(const self_type &) = delete;
+
+  /// Definition of an upstream server group equivalence class.
+  enum MatchType {
+    MATCH_IP   = TS_SERVER_OUTBOUND_MATCH_IP,   ///< Match by IP address.
+    MATCH_PORT = TS_SERVER_OUTBOUND_MATCH_PORT, ///< Match by IP address and port.
+    MATCH_HOST = TS_SERVER_OUTBOUND_MATCH_HOST, ///< Match by hostname (FQDN).
+    MATCH_BOTH = TS_SERVER_OUTBOUND_MATCH_BOTH, ///< Hostname, IP Address and port.
+  };
 
-  /**
-   * Gets the number of connections for the host
-   * @param ip IP address of the host
-   * @return Number of connections
-   */
-  int
-  getCount(const IpEndpoint &addr, const CryptoHash &hostname_hash, TSServerSessionSharingMatchType match_type)
-  {
-    if (TS_SERVER_SESSION_SHARING_MATCH_NONE == match_type) {
-      return 0; // We can never match a node if match type is NONE
-    }
+  /// String equivalents for @c MatchType.
+  static const std::array<std::string_view, static_cast<int>(MATCH_BOTH) + 1> MATCH_TYPE_NAME;
 
-    ink_mutex_acquire(&_mutex);
-    int count = _hostCount.get(ConnAddr(addr, hostname_hash, match_type));
-    ink_mutex_release(&_mutex);
-    return count;
-  }
+  /// Per transaction configuration values.
+  struct TxnConfig {
+    int max{0};                ///< Maximum concurrent connections.
+    MatchType match{MATCH_IP}; ///< Match type.
+  };
+
+  /** Static configuration values. */
+  struct GlobalConfig {
+    int queue_size{0};                          ///< Maximum delayed transactions.
+    std::chrono::milliseconds queue_delay{100}; ///< Reschedule / queue delay in ms.
+    std::chrono::seconds alert_delay{60};       ///< Alert delay in seconds.
+  };
+
+  // The names of the configuration values.
+  // Unfortunately these are not used in RecordsConfig.cc so that must be made consistent by hand.
+  // Note: These need to be @c constexpr or there are static initialization ordering risks.
+  static constexpr std::string_view CONFIG_VAR_MAX{"proxy.config.http.per_server.connection.max"_sv};
+  static constexpr std::string_view CONFIG_VAR_MATCH{"proxy.config.http.per_server.connection.match"_sv};
+  static constexpr std::string_view CONFIG_VAR_QUEUE_SIZE{"proxy.config.http.per_server.connection.queue_size"_sv};
+  static constexpr std::string_view CONFIG_VAR_QUEUE_DELAY{"proxy.config.http.per_server.connection.queue_delay"_sv};
+  static constexpr std::string_view CONFIG_VAR_ALERT_DELAY{"proxy.config.http.per_server.connection.alert_delay"_sv};
+
+  /// A record for the outbound connection count.
+  /// These are stored per outbound session equivalence class, as determined by the session matching.
+  struct Group {
+    /// Base clock.
+    using Clock = std::chrono::system_clock;
+    /// Time point type, based on the clock to be used.
+    using TimePoint = Clock::time_point;
+    /// Raw type for clock / time point counts.
+    using Ticker = TimePoint::rep;
+    /// Length of time to suppress alerts for a group.
+    static const std::chrono::seconds ALERT_DELAY;
+
+    /// Equivalence key - two groups are equivalent if their keys are equal.
+    struct Key {
+      IpEndpoint const &_addr;      ///< Remote IP address.
+      CryptoHash const &_hash;      ///< Hash of the FQDN.
+      MatchType const &_match_type; ///< Type of matching.
+    };
+
+    IpEndpoint _addr;      ///< Remote IP address.
+    CryptoHash _hash;      ///< Hash of the FQDN.
+    MatchType _match_type; ///< Type of matching.
+    std::string _fqdn;     ///< Expanded FQDN, set if matching on FQDN.
+    Key _key;              ///< Pre-assembled key which references the following members.
+
+    // Counting data.
+    std::atomic<int> _count{0};         ///< Number of outbound connections.
+    std::atomic<int> _count_max{0};     ///< largest observed @a count value.
+    std::atomic<int> _blocked{0};       ///< Number of outbound connections blocked since last alert.
+    std::atomic<int> _rescheduled{0};   ///< # of connection reschedules.
+    std::atomic<int> _in_queue{0};      ///< # of connections queued, waiting for a connection.
+    std::atomic<Ticker> _last_alert{0}; ///< Absolute time of the last alert.
+
+    LINK(Group, _link); ///< Intrusive hash table support.
+
+    /** Constructor.
+     * Construct from @c Key because the use cases do a table lookup first so the @c Key is already constructed.
+     * @param key A populated @c Key structure - values are copied to the @c Group.
+     * @param fqdn The full FQDN.
+     */
+    Group(Key const &key, std::string_view fqdn);
+    /// Key equality checker.
+    static bool equal(Key const &lhs, Key const &rhs);
+    /// Hashing function.
+    static uint64_t hash(Key const &);
+    /// Check and clear alert enable.
+    /// This is a modifying call - internal state will be updated to prevent too frequent alerts.
+    /// @param lat The last alert time, in epoch seconds, if the method returns @c true.
+    /// @return @c true if an alert should be generated, @c false otherwise.
+    bool should_alert(std::time_t *lat = nullptr);
+    /// Time of the last alert in epoch seconds.
+    std::time_t get_last_alert_epoch_time() const;
+  };
+
+  /// Container for per transaction state and operations.
+  struct TxnState {
+    Group *_g{nullptr};      ///< Active group for this transaction.
+    bool _reserved_p{false}; ///< Set if a connection slot has been reserved.
+    bool _queued_p{false};   ///< Set if the connection is delayed / queued.
+
+    /// Check if tracking is active.
+    bool is_active();
+
+    /// Reserve a connection.
+    int reserve();
+    /// Release a connection reservation.
+    void release();
+    /// Reserve a queue / retry slot.
+    int enqueue();
+    /// Release a block
+    void dequeue();
+    /// Note blocking a transaction.
+    void blocked();
+    /// Note a rescheduling
+    void rescheduled();
+    /// Clear all reservations.
+    void clear();
+    /// Drop the reservation - assume it will be cleaned up elsewhere.
+    /// @return The group for this reservation.
+    Group *drop();
+    /// Update the maximum observed count if needed against @a count.
+    void update_max_count(int count);
+
+    /** Generate a Notice that the group has become unblocked.
+     *
+     * @param config Transaction local configuration.
+     * @param count Current connection count for display in message.
+     * @param addr IP address of the upstream.
+     */
+    void Note_Unblocked(TxnConfig *config, int count, const sockaddr *addr);
+
+    /** Generate a Warning that a connection was blocked.
+     *
+     * @param config Transaction local configuration.
+     * @param sm_id State machine ID to display in Warning.
+     * @param count Count value to display in Warning.
+     * @param addr IP address of the upstream.
+     * @param debug_tag Tag to use for the debug message. If no debug message should be generated set this to @c nullptr.
+     */
+    void Warn_Blocked(TxnConfig *config, int64_t sm_id, int count, const sockaddr *addr, const char *debug_tag = nullptr);
+  };
 
-  /**
-   * Change (increment/decrement) the connection count
-   * @param ip IP address of the host
-   * @param delta Default is +1, can be set to negative to decrement
+  /** Get or create the @c Group for the specified session properties.
+   * @param txn_cnf The transaction local configuration.
+   * @param fqdn The fully qualified domain name of the upstream.
+   * @param addr The IP address of the upstream.
+   * @return A @c Group for the arguments, existing if possible and created if not.
    */
-  void
-  incrementCount(const IpEndpoint &addr, const CryptoHash &hostname_hash, TSServerSessionSharingMatchType match_type,
-                 const int delta = 1)
-  {
-    if (TS_SERVER_SESSION_SHARING_MATCH_NONE == match_type) {
-      return; // We can never match a node if match type is NONE.
-    }
+  static TxnState obtain(TxnConfig const &txn_cnf, std::string_view fqdn, const IpEndpoint &addr);
 
-    ConnAddr caddr(addr, hostname_hash, match_type);
-    ink_mutex_acquire(&_mutex);
-    int count = _hostCount.get(caddr);
-    _hostCount.put(caddr, count + delta);
-    ink_mutex_release(&_mutex);
-  }
-  /**
-   * dump to JSON for stat page.
-   * @return JSON string for _hostCount
+  /** Get the currently existing groups.
+   * @param [out] groups parameter - pointers to the groups are pushed in to this container.
+   *
+   * The groups are loaded in to @a groups, which is cleared before loading. Note the groups returned will remain valid
+   * although data inside the groups is volatile.
+   */
+  static void get(std::vector<Group const *> &groups);
+  /** Write the connection tracking data to JSON.
+   * @return string containing a JSON encoding of the table.
+   */
+  static std::string to_json_string();
+  /** Write the groups to @a f.
+   * @param f Output file.
+   */
+  static void dump(FILE *f);
+  /** Do global initialization.
+   *
+   * This sets up the global configuration and any configuration update callbacks needed. It is presumed
+   * the caller has set up the actual storage where the global configuration data is stored.
+   *
+   * @param config The storage for the global configuration data.
+   * @param txn The storage for the default per transaction data.
    */
-  std::string dumpToJSON();
+  static void config_init(GlobalConfig *global, TxnConfig *txn);
+
+  /// Tag used for debugging otuput.
+  static constexpr char const *const DEBUG_TAG{"conn_track"};
+
+  /** Convert a string to a match type.
+   *
+   * @a type is updated only if this method returns @c true.
+   *
+   * @param [in] tag Tag to look up.
+   * @param [out] type Resulting type.
+   * @return @c true if @a tag was valid and @a type was updated, otherwise @c false.
+   */
+  static bool lookup_match_type(std::string_view tag, MatchType &type);
 
-  struct ConnAddr {
-    IpEndpoint _addr;
-    CryptoHash _hostname_hash;
-    TSServerSessionSharingMatchType _match_type;
+  /** Generate a warning message for a bad @c MatchType tag.
+   *
+   * @param tag The invalid tag.
+   */
+  static void Warning_Bad_Match_Type(std::string_view tag);
 
-    ConnAddr() : _match_type(TS_SERVER_SESSION_SHARING_MATCH_NONE)
-    {
-      ink_zero(_addr);
-      ink_zero(_hostname_hash);
-    }
+  // Converters for overridable values for use in the TS API.
+  static const MgmtConverter MAX_CONV;
+  static const MgmtConverter MATCH_CONV;
 
-    ConnAddr(int x) : _match_type(TS_SERVER_SESSION_SHARING_MATCH_NONE)
-    {
-      ink_release_assert(x == 0);
-      ink_zero(_addr);
-      ink_zero(_hostname_hash);
-    }
+protected:
+  static GlobalConfig *_global_config; ///< Global configuration data.
 
-    ConnAddr(const IpEndpoint &addr, const CryptoHash &hostname_hash, TSServerSessionSharingMatchType match_type)
-      : _addr(addr), _hostname_hash(hostname_hash), _match_type(match_type)
-    {
-    }
+  /// Types and methods for the hash table.
+  struct HashDescriptor {
+    using ID       = uint64_t;
+    using Key      = Group::Key const &;
+    using Value    = Group;
+    using ListHead = DList(Value, _link);
 
-    ConnAddr(const IpEndpoint &addr, const char *hostname, TSServerSessionSharingMatchType match_type)
-      : _addr(addr), _match_type(match_type)
+    static ID
+    hash(Key key)
     {
-      CryptoContext().hash_immediate(_hostname_hash, static_cast<const void *>(hostname), strlen(hostname));
+      return Group::hash(key);
     }
-
-    operator bool() { return ats_is_ip(&_addr); }
-    std::string
-    getIpStr()
+    static Key
+    key(Value *v)
     {
-      std::string str;
-      if (*this) {
-        ip_text_buffer buf;
-        const char *ret = ats_ip_ntop(&_addr.sa, buf, sizeof(buf));
-        if (ret) {
-          str.assign(ret);
-        }
-      }
-      return str;
+      return v->_key;
     }
-
-    std::string
-    getHostnameHashStr()
+    static bool
+    equal(Key lhs, Key rhs)
     {
-      char hashBuffer[CRYPTO_HEX_SIZE];
-      return std::string(_hostname_hash.toHexStr(hashBuffer));
+      return Group::equal(lhs, rhs);
     }
   };
 
-  class ConnAddrHashFns
-  {
-  public:
-    static uintptr_t
-    hash(ConnAddr &addr)
-    {
-      if (addr._match_type == TS_SERVER_SESSION_SHARING_MATCH_IP) {
-        return (uintptr_t)ats_ip_port_hash(&addr._addr.sa);
-      } else if (addr._match_type == TS_SERVER_SESSION_SHARING_MATCH_HOST) {
-        return (uintptr_t)addr._hostname_hash.u64[0];
-      } else if (addr._match_type == TS_SERVER_SESSION_SHARING_MATCH_BOTH) {
-        return ((uintptr_t)ats_ip_port_hash(&addr._addr.sa) ^ (uintptr_t)addr._hostname_hash.u64[0]);
-      } else {
-        return 0; // they will never be equal() because of it returns false for NONE matches.
-      }
-    }
-
-    static int
-    equal(ConnAddr &a, ConnAddr &b)
-    {
-      char addrbuf1[INET6_ADDRSTRLEN];
-      char addrbuf2[INET6_ADDRSTRLEN];
-      char crypto_hashbuf1[CRYPTO_HEX_SIZE];
-      char crypto_hashbuf2[CRYPTO_HEX_SIZE];
-      a._hostname_hash.toHexStr(crypto_hashbuf1);
-      b._hostname_hash.toHexStr(crypto_hashbuf2);
-      Debug("conn_count", "Comparing hostname hash %s dest %s match method %d to hostname hash %s dest %s match method %d",
-            crypto_hashbuf1, ats_ip_nptop(&a._addr.sa, addrbuf1, sizeof(addrbuf1)), a._match_type, crypto_hashbuf2,
-            ats_ip_nptop(&b._addr.sa, addrbuf2, sizeof(addrbuf2)), b._match_type);
-
-      if (a._match_type != b._match_type || a._match_type == TS_SERVER_SESSION_SHARING_MATCH_NONE) {
-        Debug("conn_count", "result = 0, a._match_type != b._match_type || a._match_type == TS_SERVER_SESSION_SHARING_MATCH_NONE");
-        return 0;
-      }
-
-      if (a._match_type == TS_SERVER_SESSION_SHARING_MATCH_IP) {
-        if (ats_ip_addr_port_eq(&a._addr.sa, &b._addr.sa)) {
-          Debug("conn_count", "result = 1, a._match_type == TS_SERVER_SESSION_SHARING_MATCH_IP");
-          return 1;
-        } else {
-          Debug("conn_count", "result = 0, a._match_type == TS_SERVER_SESSION_SHARING_MATCH_IP");
-          return 0;
-        }
-      }
-
-      if (a._match_type == TS_SERVER_SESSION_SHARING_MATCH_HOST) {
-        if ((a._hostname_hash.u64[0] == b._hostname_hash.u64[0] && a._hostname_hash.u64[1] == b._hostname_hash.u64[1])) {
-          Debug("conn_count", "result = 1, a._match_type == TS_SERVER_SESSION_SHARING_MATCH_HOST");
-          return 1;
-        } else {
-          Debug("conn_count", "result = 0, a._match_type == TS_SERVER_SESSION_SHARING_MATCH_HOST");
-          return 0;
-        }
-      }
-
-      if (a._match_type == TS_SERVER_SESSION_SHARING_MATCH_BOTH) {
-        if ((ats_ip_addr_port_eq(&a._addr.sa, &b._addr.sa)) &&
-            (a._hostname_hash.u64[0] == b._hostname_hash.u64[0] && a._hostname_hash.u64[1] == b._hostname_hash.u64[1])) {
-          Debug("conn_count", "result = 1, a._match_type == TS_SERVER_SESSION_SHARING_MATCH_BOTH");
-
-          return 1;
-        }
-      }
-
-      Debug("conn_count", "result = 0, a._match_type == TS_SERVER_SESSION_SHARING_MATCH_BOTH");
-      return 0;
-    }
+  /// Internal implementation class instance.
+  struct Imp {
+    TSHashTable<HashDescriptor> _table; ///< Hash table of upstream groups.
+    std::mutex _mutex;                  ///< Lock for insert & find.
   };
+  static Imp _imp;
 
-protected:
-  // Hide the constructor and copy constructor
-  ConnectionCount() { ink_mutex_init(&_mutex); }
-  ConnectionCount(const ConnectionCount & /* x ATS_UNUSED */) {}
-  static ConnectionCount _connectionCount;
-  HashMap<ConnAddr, ConnAddrHashFns, int> _hostCount;
-  ink_mutex _mutex;
-
-private:
-  void
-  appendJSONPair(std::ostringstream &oss, const std::string &key, const int value)
-  {
-    oss << '\"' << key << "\": " << value;
+  /// Get the implementation instance.
+  /// @note This is done purely to allow subclasses to reuse methods in this class.
+  Imp &instance();
+};
+
+inline OutboundConnTrack::Imp &
+OutboundConnTrack::instance()
+{
+  return _imp;
+}
+
+inline OutboundConnTrack::Group::Group(Key const &key, std::string_view fqdn)
+  : _hash(key._hash), _match_type(key._match_type), _key{_addr, _hash, _match_type}
+{
+  // store the host name if relevant.
+  if (MATCH_HOST == _match_type || MATCH_BOTH == _match_type) {
+    _fqdn.assign(fqdn);
   }
+  // store the IP address if relevant.
+  if (MATCH_HOST == _match_type) {
+    _addr.setToAnyAddr(AF_INET);
+  } else {
+    ats_ip_copy(_addr, key._addr);
+  }
+}
 
-  void
-  appendJSONPair(std::ostringstream &oss, const std::string &key, const std::string &value)
-  {
-    oss << '\"' << key << "\": \"" << value << '"';
+inline uint64_t
+OutboundConnTrack::Group::hash(const Key &key)
+{
+  switch (key._match_type) {
+  case MATCH_IP:
+    return ats_ip_hash(&key._addr.sa);
+  case MATCH_PORT:
+    return ats_ip_port_hash(&key._addr.sa);
+  case MATCH_HOST:
+    return key._hash.fold();
+  case MATCH_BOTH:
+    return ats_ip_port_hash(&key._addr.sa) ^ key._hash.fold();
+  default:
+    return 0;
   }
-};
+}
 
-class ConnectionCountQueue : public ConnectionCount
+inline bool
+OutboundConnTrack::TxnState::is_active()
 {
-public:
-  /**
-   * Static method to get the instance of the class
-   * @return Returns a pointer to the instance of the class
-   */
-  static ConnectionCountQueue *
-  getInstance()
-  {
-    return &_connectionCount;
+  return nullptr != _g;
+}
+
+inline int
+OutboundConnTrack::TxnState::reserve()
+{
+  _reserved_p = true;
+  return ++_g->_count;
+}
+
+inline void
+OutboundConnTrack::TxnState::release()
+{
+  if (_reserved_p) {
+    _reserved_p = false;
+    --_g->_count;
   }
+}
 
-private:
-  static ConnectionCountQueue _connectionCount;
-};
+inline OutboundConnTrack::Group *
+OutboundConnTrack::TxnState::drop()
+{
+  _reserved_p = false;
+  return _g;
+}
+
+inline int
+OutboundConnTrack::TxnState::enqueue()
+{
+  _queued_p = true;
+  return ++_g->_in_queue;
+}
+
+inline void
+OutboundConnTrack::TxnState::dequeue()
+{
+  if (_queued_p) {
+    _queued_p = false;
+    --_g->_in_queue;
+  }
+}
+
+inline void
+OutboundConnTrack::TxnState::clear()
+{
+  if (_g) {
+    this->dequeue();
+    this->release();
+    _g = nullptr;
+  }
+}
+
+inline void
+OutboundConnTrack::TxnState::update_max_count(int count)
+{
+  auto cmax = _g->_count_max.load();
+  if (count > cmax) {
+    _g->_count_max.compare_exchange_weak(cmax, count);
+  }
+}
+
+inline void
+OutboundConnTrack::TxnState::blocked()
+{
+  ++_g->_blocked;
+}
+
+inline void
+OutboundConnTrack::TxnState::rescheduled()
+{
+  ++_g->_rescheduled;
+}
 
 Action *register_ShowConnectionCount(Continuation *, HTTPHdr *);
+
+namespace ts
+{
+inline BufferWriter &
+bwformat(BufferWriter &w, BWFSpec const &spec, OutboundConnTrack::MatchType type)
+{
+  if (spec.has_numeric_type()) {
+    bwformat(w, spec, static_cast<unsigned int>(type));
+  } else {
+    bwformat(w, spec, OutboundConnTrack::MATCH_TYPE_NAME[type]);
+  }
+  return w;
+}
+
+BufferWriter &bwformat(BufferWriter &w, BWFSpec const &spec, OutboundConnTrack::Group::Key const &key);
+BufferWriter &bwformat(BufferWriter &w, BWFSpec const &spec, OutboundConnTrack::Group const &g);
+} // namespace ts
diff --git a/proxy/http/HttpDebugNames.cc b/proxy/http/HttpDebugNames.cc
index 8f270a8..31c8f9f 100644
--- a/proxy/http/HttpDebugNames.cc
+++ b/proxy/http/HttpDebugNames.cc
@@ -56,6 +56,8 @@ HttpDebugNames::get_server_state_name(HttpTransact::ServerState_t state)
     return "TRANSACTION_COMPLETE";
   case HttpTransact::PARENT_RETRY:
     return "PARENT_RETRY";
+  case HttpTransact::OUTBOUND_CONGESTION:
+    return "OUTBOUND_CONGESTION";
   }
 
   return ("unknown state name");
diff --git a/proxy/http/HttpProxyAPIEnums.h b/proxy/http/HttpProxyAPIEnums.h
index dc18bb8..5a94a10 100644
--- a/proxy/http/HttpProxyAPIEnums.h
+++ b/proxy/http/HttpProxyAPIEnums.h
@@ -29,6 +29,7 @@
 
 #pragma once
 
+// This is use to signal apidefs.h to not define these again.
 #ifndef _HTTP_PROXY_API_ENUMS_H_
 #define _HTTP_PROXY_API_ENUMS_H_
 
@@ -45,4 +46,13 @@ typedef enum {
   TS_SERVER_SESSION_SHARING_POOL_GLOBAL,
   TS_SERVER_SESSION_SHARING_POOL_THREAD,
 } TSServerSessionSharingPoolType;
+
+/// Values for per server outbound connection tracking group definition.
+/// See proxy.config.http.per_server.match
+typedef enum {
+  TS_SERVER_OUTBOUND_MATCH_IP,
+  TS_SERVER_OUTBOUND_MATCH_PORT,
+  TS_SERVER_OUTBOUND_MATCH_HOST,
+  TS_SERVER_OUTBOUND_MATCH_BOTH
+} TSOutboundConnectionMatchType;
 #endif
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 297275e..cc832f1 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -42,6 +42,7 @@
 #include "HttpPages.h"
 #include "IPAllow.h"
 #include "ts/I_Layout.h"
+#include "ts/bwf_std_format.h"
 
 #include <openssl/ossl_typ.h>
 #include <openssl/ssl.h>
@@ -81,6 +82,9 @@ static const int boundary_size   = 2 + sizeof("RANGE_SEPARATOR") - 1 + 2;
 static const char *str_100_continue_response = "HTTP/1.1 100 Continue\r\n\r\n";
 static const int len_100_continue_response   = strlen(str_100_continue_response);
 
+// Handy typedef for short (single line) message generation.
+using lbw = ts::LocalBufferWriter<256>;
+
 namespace
 {
 /// Update the milestone state given the milestones and timer.
@@ -1720,23 +1724,19 @@ HttpSM::state_http_server_open(int event, void *data)
                 httpServerSessionAllocator.alloc();
     session->sharing_pool  = static_cast<TSServerSessionSharingPoolType>(t_state.http_config_param->server_session_sharing_pool);
     session->sharing_match = static_cast<TSServerSessionSharingMatchType>(t_state.txn_conf->server_session_sharing_match);
-    // If origin_max_connections or origin_min_keep_alive_connections is
-    // set then we are metering the max and or min number
-    // of connections per host.  Set enable_origin_connection_limiting
-    // to true in the server session so it will increment and decrement
-    // the connection count.
-    if (t_state.txn_conf->origin_max_connections > 0 || t_state.http_config_param->origin_min_keep_alive_connections > 0) {
-      SMDebug("http_ss", "[%" PRId64 "] max number of connections: %" PRIu64, sm_id, t_state.txn_conf->origin_max_connections);
-      session->enable_origin_connection_limiting = true;
-    }
-    /*UnixNetVConnection * vc = (UnixNetVConnection*)(ua_txn->client_vc);
-       UnixNetVConnection *server_vc = (UnixNetVConnection*)data;
-       printf("client fd is :%d , server fd is %d\n",vc->con.fd,
-       server_vc->con.fd); */
+
     session->attach_hostname(t_state.current.server->name);
     session->new_connection(static_cast<NetVConnection *>(data));
     session->state = HSS_ACTIVE;
 
+    // If origin_max_connections or origin_min_keep_alive_connections is set then we are metering
+    // the max and or min number of connections per host. Transfer responsibility for this to the
+    // session object.
+    if (t_state.outbound_conn_track_state.is_active()) {
+      SMDebug("http_ss", "[%" PRId64 "] max number of outbound connections: %d", sm_id, t_state.txn_conf->outbound_conntrack.max);
+      session->enable_outbound_connection_tracking(t_state.outbound_conn_track_state.drop());
+    }
+
     attach_server_session(session);
     if (t_state.current.request_to == HttpTransact::PARENT_PROXY) {
       session->to_parent_proxy = true;
@@ -1759,6 +1759,7 @@ HttpSM::state_http_server_open(int event, void *data)
     t_state.current.state = HttpTransact::CONNECTION_ERROR;
     // save the errno from the connect fail for future use (passed as negative value, flip back)
     t_state.current.server->set_connect_fail(event == NET_EVENT_OPEN_FAILED ? -reinterpret_cast<intptr_t>(data) : ECONNABORTED);
+    t_state.outbound_conn_track_state.clear();
 
     /* If we get this error in transparent mode, then we simply can't bind to the 4-tuple to make the connection.  There's no hope
        of retries succeeding in the near future. The best option is to just shut down the connection without further comment. The
@@ -4620,7 +4621,7 @@ void
 HttpSM::send_origin_throttled_response()
 {
   t_state.current.attempts = t_state.txn_conf->connect_attempts_max_retries;
-  t_state.current.state    = HttpTransact::CONNECTION_ERROR;
+  t_state.current.state    = HttpTransact::OUTBOUND_CONGESTION;
   call_transact_and_set_next_state(HttpTransact::HandleResponse);
 }
 
@@ -4645,6 +4646,10 @@ HttpSM::do_http_server_open(bool raw)
   pending_action = nullptr;
   ink_assert(server_entry == nullptr);
 
+  // Clean up connection tracking info if any. Need to do it now so the selected group
+  // is consistent with the actual upstream in case of retry.
+  t_state.outbound_conn_track_state.clear();
+
   // ua_entry can be null if a scheduled update is also a reverse proxy
   // request. Added REVPROXY to the assert below, and then changed checks
   // to be based on ua_txn != NULL instead of req_flavor value.
@@ -4845,56 +4850,59 @@ HttpSM::do_http_server_open(bool raw)
       return;
     }
   }
-  // Check to see if we have reached the max number of connections on this
-  // host.
-  if (t_state.txn_conf->origin_max_connections > 0) {
-    ConnectionCount *connections = ConnectionCount::getInstance();
-
-    CryptoHash hostname_hash;
-    CryptoContext().hash_immediate(hostname_hash, static_cast<const void *>(t_state.current.server->name),
-                                   static_cast<int>(strlen(t_state.current.server->name)));
-
-    auto ccount = connections->getCount(t_state.current.server->dst_addr, hostname_hash,
-                                        (TSServerSessionSharingMatchType)t_state.txn_conf->server_session_sharing_match);
-    if (ccount >= t_state.txn_conf->origin_max_connections) {
-      ip_port_text_buffer addrbuf;
-      ats_ip_nptop(&t_state.current.server->dst_addr.sa, addrbuf, sizeof(addrbuf));
-      SMDebug("http", "[%" PRId64 "] too many connections (%d) for this host (%" PRId64 "): %s", sm_id, ccount,
-              t_state.txn_conf->origin_max_connections, addrbuf);
-      Warning("[%" PRId64 "] too many connections (%d) for this host (%" PRId64 "): %s", sm_id, ccount,
-              t_state.txn_conf->origin_max_connections, addrbuf);
-      ink_assert(pending_action == nullptr);
 
-      // if we were previously queued, or the queue is disabled-- just reschedule
-      if (t_state.origin_request_queued || t_state.txn_conf->origin_max_connections_queue < 0) {
-        pending_action = eventProcessor.schedule_in(this, HRTIME_MSECONDS(100));
-        return;
-      } else if (t_state.txn_conf->origin_max_connections_queue > 0) { // If we have a queue, lets see if there is a slot
-        ConnectionCountQueue *waiting_connections = ConnectionCountQueue::getInstance();
-        // if there is space in the queue
-        if (waiting_connections->getCount(t_state.current.server->dst_addr, hostname_hash,
-                                          (TSServerSessionSharingMatchType)t_state.txn_conf->server_session_sharing_match) <
-            t_state.txn_conf->origin_max_connections_queue) {
-          t_state.origin_request_queued = true;
-          Debug("http", "[%" PRId64 "] queued for this host: %s", sm_id,
-                ats_ip_ntop(&t_state.current.server->dst_addr.sa, addrbuf, sizeof(addrbuf)));
-          waiting_connections->incrementCount(t_state.current.server->dst_addr, hostname_hash,
-                                              (TSServerSessionSharingMatchType)t_state.txn_conf->server_session_sharing_match, 1);
-          pending_action = eventProcessor.schedule_in(this, HRTIME_MSECONDS(100));
-        } else { // the queue is full
+  // See if the outbound connection tracker data is needed. If so, get it here for consistency.
+  if (t_state.txn_conf->outbound_conntrack.max > 0 || t_state.http_config_param->origin_min_keep_alive_connections > 0) {
+    t_state.outbound_conn_track_state = OutboundConnTrack::obtain(
+      t_state.txn_conf->outbound_conntrack, std::string_view{t_state.current.server->name}, t_state.current.server->dst_addr);
+  }
+
+  // Check to see if we have reached the max number of connections on this upstream host.
+  if (t_state.txn_conf->outbound_conntrack.max > 0) {
+    auto &ct_state = t_state.outbound_conn_track_state;
+    auto ccount    = ct_state.reserve();
+    if (ccount > t_state.txn_conf->outbound_conntrack.max) {
+      ct_state.release();
+
+      ink_assert(pending_action == nullptr); // in case of reschedule must not have already pending.
+
+      // If the queue is disabled, reschedule.
+      if (t_state.http_config_param->outbound_conntrack.queue_size < 0) {
+        ct_state.enqueue();
+        ct_state.rescheduled();
+        pending_action =
+          eventProcessor.schedule_in(this, HRTIME_MSECONDS(t_state.http_config_param->outbound_conntrack.queue_delay.count()));
+      } else if (t_state.http_config_param->outbound_conntrack.queue_size > 0) { // queue enabled, check for a slot
+        auto wcount = ct_state.enqueue();
+        if (wcount < t_state.http_config_param->outbound_conntrack.queue_size) {
+          ct_state.rescheduled();
+          SMDebug("http", "%s", lbw().print("[{}] queued for {}\0", sm_id, t_state.current.server->dst_addr).data());
+          pending_action =
+            eventProcessor.schedule_in(this, HRTIME_MSECONDS(t_state.http_config_param->outbound_conntrack.queue_delay.count()));
+        } else {              // the queue is full
+          ct_state.dequeue(); // release the queue slot
+          ct_state.blocked(); // note the blockage.
           HTTP_INCREMENT_DYN_STAT(http_origin_connections_throttled_stat);
           send_origin_throttled_response();
         }
-      } else { // the queue is set to 0
+      } else { // queue size is 0, always block.
+        ct_state.blocked();
         HTTP_INCREMENT_DYN_STAT(http_origin_connections_throttled_stat);
         send_origin_throttled_response();
       }
+
+      ct_state.Warn_Blocked(&t_state.txn_conf->outbound_conntrack, sm_id, ccount - 1, &t_state.current.server->dst_addr.sa,
+                            debug_on && is_debug_tag_set("http") ? "http" : nullptr);
+
       return;
+    } else {
+      ct_state.Note_Unblocked(&t_state.txn_conf->outbound_conntrack, ccount, &t_state.current.server->dst_addr.sa);
     }
+
+    ct_state.update_max_count(ccount);
   }
 
-  // We did not manage to get an existing session
-  //  and need to open a new connection
+  // We did not manage to get an existing session and need to open a new connection
   Action *connect_action_handle;
 
   NetVCOptions opt;
@@ -4908,7 +4916,7 @@ HttpSM::do_http_server_open(bool raw)
   if (ua_txn) {
     opt.local_port = ua_txn->get_outbound_port();
 
-    const IpAddr &outbound_ip = AF_INET6 == ip_family ? ua_txn->get_outbound_ip6() : ua_txn->get_outbound_ip4();
+    const IpAddr &outbound_ip = AF_INET6 == opt.ip_family ? ua_txn->get_outbound_ip6() : ua_txn->get_outbound_ip4();
     if (outbound_ip.isValid()) {
       opt.addr_binding = NetVCOptions::INTF_ADDR;
       opt.local_ip     = outbound_ip;
@@ -5332,18 +5340,8 @@ HttpSM::handle_post_failure()
 void
 HttpSM::handle_http_server_open()
 {
-  // if we were a queued request, we need to decrement the queue size-- as we got a connection
-  if (t_state.origin_request_queued) {
-    CryptoHash hostname_hash;
-    CryptoContext().hash_immediate(hostname_hash, static_cast<const void *>(t_state.current.server->name),
-                                   strlen(t_state.current.server->name));
-
-    ConnectionCountQueue *waiting_connections = ConnectionCountQueue::getInstance();
-    waiting_connections->incrementCount(t_state.current.server->dst_addr, hostname_hash,
-                                        (TSServerSessionSharingMatchType)t_state.txn_conf->server_session_sharing_match, -1);
-    // The request is now not queued. This is important if the request will ever retry, the t_state is re-used
-    t_state.origin_request_queued = false;
-  }
+  // The request is now not queued. This is important because server retries reuse the t_state.
+  t_state.outbound_conn_track_state.dequeue();
 
   // [bwyatt] applying per-transaction OS netVC options here
   //          IFF they differ from the netVC's current options.
diff --git a/proxy/http/HttpServerSession.cc b/proxy/http/HttpServerSession.cc
index c70cf44..9f946a5 100644
--- a/proxy/http/HttpServerSession.cc
+++ b/proxy/http/HttpServerSession.cc
@@ -29,6 +29,8 @@
 
  ****************************************************************************/
 #include "ts/ink_config.h"
+#include "ts/BufferWriter.h"
+#include "ts/bwf_std_format.h"
 #include "ts/Allocator.h"
 #include "HttpServerSession.h"
 #include "HttpSessionManager.h"
@@ -72,18 +74,6 @@ HttpServerSession::new_connection(NetVConnection *new_vc)
   magic = HTTP_SS_MAGIC_ALIVE;
   HTTP_SUM_GLOBAL_DYN_STAT(http_current_server_connections_stat, 1); // Update the true global stat
   HTTP_INCREMENT_DYN_STAT(http_total_server_connections_stat);
-  // Check to see if we are limiting the number of connections
-  // per host
-  if (enable_origin_connection_limiting == true) {
-    if (connection_count == nullptr) {
-      connection_count = ConnectionCount::getInstance();
-    }
-    connection_count->incrementCount(get_server_ip(), hostname_hash, sharing_match);
-    ip_port_text_buffer addrbuf;
-    Debug("http_ss", "[%" PRId64 "] new connection, ip: %s, count: %u", con_id,
-          ats_ip_nptop(&get_server_ip().sa, addrbuf, sizeof(addrbuf)),
-          connection_count->getCount(get_server_ip(), hostname_hash, sharing_match));
-  }
 #ifdef LAZY_BUF_ALLOC
   read_buffer = new_empty_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
 #else
@@ -96,6 +86,18 @@ HttpServerSession::new_connection(NetVConnection *new_vc)
   new_vc->set_tcp_congestion_control(SERVER_SIDE);
 }
 
+void
+HttpServerSession::enable_outbound_connection_tracking(OutboundConnTrack::Group *group)
+{
+  ink_assert(nullptr == conn_track_group);
+  conn_track_group = group;
+  if (is_debug_tag_set("http_ss")) {
+    ts::LocalBufferWriter<256> w;
+    w.print("[{}] new connection, ip: {}, group ({}), count: {}\0", con_id, get_server_ip(), *group, group->_count);
+    Debug("http_ss", "%s", w.data());
+  }
+}
+
 VIO *
 HttpServerSession::do_io_read(Continuation *c, int64_t nbytes, MIOBuffer *buf)
 {
@@ -117,30 +119,36 @@ HttpServerSession::do_io_shutdown(ShutdownHowTo_t howto)
 void
 HttpServerSession::do_io_close(int alerrno)
 {
+  ts::LocalBufferWriter<256> w;
+  bool debug_p = is_debug_tag_set("http_ss");
+
   if (state == HSS_ACTIVE) {
     HTTP_DECREMENT_DYN_STAT(http_current_server_transactions_stat);
     this->server_trans_stat--;
   }
 
-  Debug("http_ss", "[%" PRId64 "] session closing, netvc %p", con_id, server_vc);
+  if (debug_p)
+    w.print("[{}] session close: nevtc {:x}", con_id, server_vc);
 
   HTTP_SUM_GLOBAL_DYN_STAT(http_current_server_connections_stat, -1); // Make sure to work on the global stat
   HTTP_SUM_DYN_STAT(http_transactions_per_server_con, transact_count);
 
-  // Check to see if we are limiting the number of connections
-  // per host
-  if (enable_origin_connection_limiting == true) {
-    if (connection_count->getCount(get_server_ip(), hostname_hash, sharing_match) >= 0) {
-      connection_count->incrementCount(get_server_ip(), hostname_hash, sharing_match, -1);
-      ip_port_text_buffer addrbuf;
-      Debug("http_ss", "[%" PRId64 "] connection closed, ip: %s, count: %u", con_id,
-            ats_ip_nptop(&get_server_ip().sa, addrbuf, sizeof(addrbuf)),
-            connection_count->getCount(get_server_ip(), hostname_hash, sharing_match));
+  // Update upstream connection tracking data if present.
+  if (conn_track_group) {
+    if (conn_track_group->_count >= 0) {
+      auto n = (conn_track_group->_count)--;
+      if (debug_p) {
+        w.print(" conn track group ({}) count {}", conn_track_group->_key, n);
+      }
     } else {
-      Error("[%" PRId64 "] number of connections should be greater than or equal to zero: %u", con_id,
-            connection_count->getCount(get_server_ip(), hostname_hash, sharing_match));
+      // A bit dubious, as there's no guarantee it's still negative, but even that would be interesting to know.
+      Error("[http_ss] [%" PRId64 "] number of connections should be greater than or equal to zero: %u", con_id,
+            conn_track_group->_count.load());
     }
   }
+  if (debug_p) {
+    Debug("http_ss", "%.*s", static_cast<int>(w.size()), w.data());
+  }
 
   if (server_vc) {
     server_vc->do_io_close(alerrno);
diff --git a/proxy/http/HttpServerSession.h b/proxy/http/HttpServerSession.h
index b3a134e..491e2ca 100644
--- a/proxy/http/HttpServerSession.h
+++ b/proxy/http/HttpServerSession.h
@@ -66,30 +66,23 @@ enum {
 
 class HttpServerSession : public VConnection
 {
+  using super_type = VConnection;
+
 public:
-  HttpServerSession()
-    : VConnection(nullptr),
-      hostname_hash(),
-      con_id(0),
-      transact_count(0),
-      state(HSS_INIT),
-      to_parent_proxy(false),
-      server_trans_stat(0),
-      private_session(false),
-      sharing_match(TS_SERVER_SESSION_SHARING_MATCH_BOTH),
-      sharing_pool(TS_SERVER_SESSION_SHARING_POOL_GLOBAL),
-      enable_origin_connection_limiting(false),
-      connection_count(nullptr),
-      read_buffer(nullptr),
-      server_vc(nullptr),
-      magic(HTTP_SS_MAGIC_DEAD),
-      buf_reader(nullptr)
-  {
-  }
+  HttpServerSession() : super_type(nullptr) {}
 
   void destroy();
   void new_connection(NetVConnection *new_vc);
 
+  /** Enable tracking the number of outbound session.
+   *
+   * @param group The connection tracking group.
+   *
+   * The @a group must have already incremented the connection count. It will be cleaned up when the
+   * session terminates.
+   */
+  void enable_outbound_connection_tracking(OutboundConnTrack::Group *group);
+
   void
   reset_read_buffer(void)
   {
@@ -139,28 +132,28 @@ public:
 
   CryptoHash hostname_hash;
 
-  int64_t con_id;
-  int transact_count;
-  HSS_State state;
+  int64_t con_id     = 0;
+  int transact_count = 0;
+  HSS_State state    = HSS_INIT;
 
   // Used to determine whether the session is for parent proxy
   // it is session to orgin server
   // We need to determine whether a closed connection was to
   // close parent proxy to update the
   // proxy.process.http.current_parent_proxy_connections
-  bool to_parent_proxy;
+  bool to_parent_proxy = false;
 
   // Used to verify we are recording the server
   //   transaction stat properly
-  int server_trans_stat;
+  int server_trans_stat = 0;
 
   // Sessions become if authentication headers
   //  are sent over them
-  bool private_session;
+  bool private_session = false;
 
   // Copy of the owning SM's server session sharing settings
-  TSServerSessionSharingMatchType sharing_match;
-  TSServerSessionSharingPoolType sharing_pool;
+  TSServerSessionSharingMatchType sharing_match{TS_SERVER_SESSION_SHARING_MATCH_BOTH};
+  TSServerSessionSharingPoolType sharing_pool{TS_SERVER_SESSION_SHARING_POOL_GLOBAL};
   //  int share_session;
 
   LINK(HttpServerSession, ip_hash_link);
@@ -168,8 +161,7 @@ public:
 
   // Keep track of connection limiting and a pointer to the
   // singleton that keeps track of the connection counts.
-  bool enable_origin_connection_limiting;
-  ConnectionCount *connection_count;
+  OutboundConnTrack::Group *conn_track_group = nullptr;
 
   // The ServerSession owns the following buffer which use
   //   for parsing the headers.  The server session needs to
@@ -178,7 +170,7 @@ public:
   //   changing the buffer we are doing I/O on.  We can
   //   not change the buffer for I/O without issuing a
   //   an asyncronous cancel on NT
-  MIOBuffer *read_buffer;
+  MIOBuffer *read_buffer = nullptr;
 
   virtual int
   populate_protocol(std::string_view *result, int size) const
@@ -197,10 +189,10 @@ public:
 private:
   HttpServerSession(HttpServerSession &);
 
-  NetVConnection *server_vc;
-  int magic;
+  NetVConnection *server_vc = nullptr;
+  int magic                 = HTTP_SS_MAGIC_DEAD;
 
-  IOBufferReader *buf_reader;
+  IOBufferReader *buf_reader = nullptr;
 };
 
 extern ClassAllocator<HttpServerSession> httpServerSessionAllocator;
diff --git a/proxy/http/HttpSessionManager.cc b/proxy/http/HttpSessionManager.cc
index 5d573cc..07f7c41 100644
--- a/proxy/http/HttpSessionManager.cc
+++ b/proxy/http/HttpSessionManager.cc
@@ -197,9 +197,8 @@ ServerSessionPool::eventHandler(int event, void *data)
       // to the origin and we are below the min number of keep alive connections to this
       // origin, then reset the timeouts on our end and do not close the connection
       if ((event == VC_EVENT_INACTIVITY_TIMEOUT || event == VC_EVENT_ACTIVE_TIMEOUT) && s->state == HSS_KA_SHARED &&
-          s->enable_origin_connection_limiting) {
-        bool connection_count_below_min = s->connection_count->getCount(s->get_server_ip(), s->hostname_hash, s->sharing_match) <=
-                                          http_config_params->origin_min_keep_alive_connections;
+          s->conn_track_group) {
+        bool connection_count_below_min = s->conn_track_group->_count <= http_config_params->origin_min_keep_alive_connections;
 
         if (connection_count_below_min) {
           Debug("http_ss",
diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
index f9ee1f7..24171cb 100644
--- a/proxy/http/HttpTransact.cc
+++ b/proxy/http/HttpTransact.cc
@@ -3080,6 +3080,8 @@ HttpTransact::OriginServerRawOpen(State *s)
   /* fall through */
   case CONNECTION_CLOSED:
     /* fall through */
+  case OUTBOUND_CONGESTION:
+    /* fall through */
     handle_server_died(s);
 
     ink_assert(s->cache_info.action == CACHE_DO_NO_ACTION);
@@ -3458,6 +3460,12 @@ HttpTransact::handle_response_from_server(State *s)
     s->current.server->clear_connect_fail();
     handle_forward_server_connection_open(s);
     break;
+  case OUTBOUND_CONGESTION:
+    TxnDebug("http_trans", "[handle_response_from_server] Error. congestion control -- congested.");
+    SET_VIA_STRING(VIA_DETAIL_SERVER_CONNECT, VIA_DETAIL_SERVER_FAILURE);
+    s->current.server->set_connect_fail(EUSERS); // too many users
+    handle_server_connection_not_open(s);
+    break;
   case OPEN_RAW_ERROR:
   /* fall through */
   case CONNECTION_ERROR:
@@ -6308,7 +6316,8 @@ HttpTransact::is_response_valid(State *s, HTTPHdr *incoming_response)
   if (s->current.state != CONNECTION_ALIVE) {
     ink_assert((s->current.state == CONNECTION_ERROR) || (s->current.state == OPEN_RAW_ERROR) ||
                (s->current.state == PARSE_ERROR) || (s->current.state == CONNECTION_CLOSED) ||
-               (s->current.state == INACTIVE_TIMEOUT) || (s->current.state == ACTIVE_TIMEOUT));
+               (s->current.state == INACTIVE_TIMEOUT) || (s->current.state == ACTIVE_TIMEOUT) ||
+               s->current.state == OUTBOUND_CONGESTION);
 
     s->hdr_info.response_error = CONNECTION_OPEN_FAILED;
     return false;
@@ -7409,6 +7418,12 @@ HttpTransact::handle_server_died(State *s)
     reason    = "Invalid HTTP Response";
     body_type = "response#bad_response";
     break;
+  case OUTBOUND_CONGESTION:
+    status                     = HTTP_STATUS_SERVICE_UNAVAILABLE;
+    reason                     = "Origin server congested";
+    body_type                  = "congestion#retryAfter";
+    s->hdr_info.response_error = TOTAL_RESPONSE_ERROR_TYPES;
+    break;
   case STATE_UNDEFINED:
   case TRANSACTION_COMPLETE:
   default: /* unknown death */
diff --git a/proxy/http/HttpTransact.h b/proxy/http/HttpTransact.h
index 962c864..c447e11 100644
--- a/proxy/http/HttpTransact.h
+++ b/proxy/http/HttpTransact.h
@@ -73,6 +73,8 @@ struct HttpConfigParams;
 class HttpSM;
 
 #include "ts/InkErrno.h"
+#include "HttpConnectionCount.h"
+
 #define UNKNOWN_INTERNAL_ERROR (INK_START_ERRNO - 1)
 
 enum ViaStringIndex_t {
@@ -351,7 +353,8 @@ public:
     OPEN_RAW_ERROR,
     PARSE_ERROR,
     TRANSACTION_COMPLETE,
-    PARENT_RETRY
+    PARENT_RETRY,
+    OUTBOUND_CONGESTION
   };
 
   enum CacheWriteStatus_t {
@@ -692,6 +695,7 @@ public:
     CacheLookupInfo cache_info;
     DNSLookupInfo dns_info;
     RedirectInfo redirect_info;
+    OutboundConnTrack::TxnState outbound_conn_track_state;
     unsigned int updated_server_version   = HostDBApplicationInfo::HTTP_VERSION_UNDEFINED;
     bool force_dns                        = false;
     MgmtByte cache_open_write_fail_action = 0;
@@ -741,9 +745,6 @@ public:
     bool is_websocket        = false;
     bool did_upgrade_succeed = false;
 
-    // Some queue info
-    bool origin_request_queued = false;
-
     char *internal_msg_buffer                       = nullptr; // out
     char *internal_msg_buffer_type                  = nullptr; // out
     int64_t internal_msg_buffer_size                = 0;       // out
@@ -899,6 +900,7 @@ public:
       arena.reset();
       unmapped_url.clear();
       hostdb_entry.clear();
+      outbound_conn_track_state.clear();
 
       delete[] ranges;
       ranges      = nullptr;
diff --git a/src/traffic_server/InkAPI.cc b/src/traffic_server/InkAPI.cc
index 4b22185..6fef4fc 100644
--- a/src/traffic_server/InkAPI.cc
+++ b/src/traffic_server/InkAPI.cc
@@ -7801,367 +7801,408 @@ TSSkipRemappingSet(TSHttpTxn txnp, int flag)
 
 template <typename T>
 inline void *
-_memberp_to_generic(T *ptr, std::type_info const *&type)
-{
-  type = &typeid(T);
+_memberp_to_generic(T *ptr, MgmtConverter const *&conv)
+{
+  static const MgmtConverter IntConverter{
+    [](void *data) -> MgmtInt { return *static_cast<MgmtInt *>(data); },
+    [](void *data, MgmtInt i) -> void { *static_cast<MgmtInt *>(data) = i; },
+    nullptr,
+    nullptr, // float
+    nullptr,
+    nullptr // string
+  };
+
+  static const MgmtConverter ByteConverter{
+    [](void *data) -> MgmtInt { return *static_cast<MgmtByte *>(data); },
+    [](void *data, MgmtInt i) -> void { *static_cast<MgmtByte *>(data) = i; },
+    nullptr,
+    nullptr, // float
+    nullptr,
+    nullptr // string
+  };
+
+  static const MgmtConverter FloatConverter{
+    nullptr, // int
+    nullptr,
+    [](void *data) -> MgmtFloat { return *static_cast<MgmtFloat *>(data); },
+    [](void *data, MgmtFloat f) -> void { *static_cast<MgmtFloat *>(data) = f; },
+    nullptr,
+    nullptr // string
+  };
+
+  // For now, strings are special.
+
+  auto type = &typeid(T);
+  if (*type == TYPE_INFO_MGMT_INT) {
+    conv = &IntConverter;
+  } else if (*type == TYPE_INFO_MGMT_BYTE) {
+    conv = &ByteConverter;
+  } else if (*type == TYPE_INFO_MGMT_FLOAT) {
+    conv = &FloatConverter;
+  } else {
+    conv = nullptr;
+  }
+
   return ptr;
 }
 
 // Little helper function to find the struct member
 static void *
-_conf_to_memberp(TSOverridableConfigKey conf, OverridableHttpConfigParams *overridableHttpConfig, std::type_info const *&typep)
+_conf_to_memberp(TSOverridableConfigKey conf, OverridableHttpConfigParams *overridableHttpConfig, MgmtConverter const *&conv)
 {
   void *ret = nullptr;
-  typep     = &typeid(void);
+  conv      = nullptr;
 
   switch (conf) {
   case TS_CONFIG_URL_REMAP_PRISTINE_HOST_HDR:
-    ret = _memberp_to_generic(&overridableHttpConfig->maintain_pristine_host_hdr, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->maintain_pristine_host_hdr, conv);
     break;
   case TS_CONFIG_HTTP_CHUNKING_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->chunking_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->chunking_enabled, conv);
     break;
   case TS_CONFIG_HTTP_NEGATIVE_CACHING_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->negative_caching_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->negative_caching_enabled, conv);
     break;
   case TS_CONFIG_HTTP_NEGATIVE_CACHING_LIFETIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->negative_caching_lifetime, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->negative_caching_lifetime, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_WHEN_TO_REVALIDATE:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_when_to_revalidate, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_when_to_revalidate, conv);
     break;
   case TS_CONFIG_HTTP_KEEP_ALIVE_ENABLED_IN:
-    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_enabled_in, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_enabled_in, conv);
     break;
   case TS_CONFIG_HTTP_KEEP_ALIVE_ENABLED_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_enabled_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_enabled_out, conv);
     break;
   case TS_CONFIG_HTTP_KEEP_ALIVE_POST_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_post_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_post_out, conv);
     break;
   case TS_CONFIG_HTTP_SERVER_SESSION_SHARING_MATCH:
-    ret = _memberp_to_generic(&overridableHttpConfig->server_session_sharing_match, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->server_session_sharing_match, conv);
     break;
   case TS_CONFIG_NET_SOCK_RECV_BUFFER_SIZE_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->sock_recv_buffer_size_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->sock_recv_buffer_size_out, conv);
     break;
   case TS_CONFIG_NET_SOCK_SEND_BUFFER_SIZE_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->sock_send_buffer_size_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->sock_send_buffer_size_out, conv);
     break;
   case TS_CONFIG_NET_SOCK_OPTION_FLAG_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->sock_option_flag_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->sock_option_flag_out, conv);
     break;
   case TS_CONFIG_HTTP_FORWARD_PROXY_AUTH_TO_PARENT:
-    ret = _memberp_to_generic(&overridableHttpConfig->fwd_proxy_auth_to_parent, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->fwd_proxy_auth_to_parent, conv);
     break;
   case TS_CONFIG_HTTP_ANONYMIZE_REMOVE_FROM:
-    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_from, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_from, conv);
     break;
   case TS_CONFIG_HTTP_ANONYMIZE_REMOVE_REFERER:
-    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_referer, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_referer, conv);
     break;
   case TS_CONFIG_HTTP_ANONYMIZE_REMOVE_USER_AGENT:
-    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_user_agent, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_user_agent, conv);
     break;
   case TS_CONFIG_HTTP_ANONYMIZE_REMOVE_COOKIE:
-    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_cookie, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_cookie, conv);
     break;
   case TS_CONFIG_HTTP_ANONYMIZE_REMOVE_CLIENT_IP:
-    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_client_ip, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_remove_client_ip, conv);
     break;
   case TS_CONFIG_HTTP_ANONYMIZE_INSERT_CLIENT_IP:
-    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_insert_client_ip, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->anonymize_insert_client_ip, conv);
     break;
   case TS_CONFIG_HTTP_RESPONSE_SERVER_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_server_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_server_enabled, conv);
     break;
   case TS_CONFIG_HTTP_INSERT_SQUID_X_FORWARDED_FOR:
-    ret = _memberp_to_generic(&overridableHttpConfig->insert_squid_x_forwarded_for, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->insert_squid_x_forwarded_for, conv);
     break;
   case TS_CONFIG_HTTP_INSERT_FORWARDED:
-    ret = _memberp_to_generic(&overridableHttpConfig->insert_forwarded, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->insert_forwarded, conv);
     break;
   case TS_CONFIG_HTTP_SERVER_TCP_INIT_CWND:
-    ret = _memberp_to_generic(&overridableHttpConfig->server_tcp_init_cwnd, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->server_tcp_init_cwnd, conv);
     break;
   case TS_CONFIG_HTTP_SEND_HTTP11_REQUESTS:
-    ret = _memberp_to_generic(&overridableHttpConfig->send_http11_requests, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->send_http11_requests, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_HTTP:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_http, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_http, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_CLIENT_NO_CACHE:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_client_no_cache, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_client_no_cache, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_CLIENT_CC_MAX_AGE:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_client_cc_max_age, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_client_cc_max_age, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IMS_ON_CLIENT_NO_CACHE:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_ims_on_client_no_cache, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_ims_on_client_no_cache, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_SERVER_NO_CACHE:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_server_no_cache, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_server_no_cache, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_CACHE_RESPONSES_TO_COOKIES:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_responses_to_cookies, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_responses_to_cookies, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_AUTHENTICATION:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_auth, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_ignore_auth, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_CACHE_URLS_THAT_LOOK_DYNAMIC:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_urls_that_look_dynamic, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_urls_that_look_dynamic, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_REQUIRED_HEADERS:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_required_headers, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_required_headers, conv);
     break;
   case TS_CONFIG_HTTP_INSERT_REQUEST_VIA_STR:
-    ret = _memberp_to_generic(&overridableHttpConfig->insert_request_via_string, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->insert_request_via_string, conv);
     break;
   case TS_CONFIG_HTTP_INSERT_RESPONSE_VIA_STR:
-    ret = _memberp_to_generic(&overridableHttpConfig->insert_response_via_string, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->insert_response_via_string, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_HEURISTIC_MIN_LIFETIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_heuristic_min_lifetime, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_heuristic_min_lifetime, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_HEURISTIC_MAX_LIFETIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_heuristic_max_lifetime, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_heuristic_max_lifetime, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_GUARANTEED_MIN_LIFETIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_guaranteed_min_lifetime, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_guaranteed_min_lifetime, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_GUARANTEED_MAX_LIFETIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_guaranteed_max_lifetime, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_guaranteed_max_lifetime, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_MAX_STALE_AGE:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_max_stale_age, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_max_stale_age, conv);
     break;
   case TS_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_IN:
-    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_no_activity_timeout_in, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_no_activity_timeout_in, conv);
     break;
   case TS_CONFIG_HTTP_KEEP_ALIVE_NO_ACTIVITY_TIMEOUT_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_no_activity_timeout_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->keep_alive_no_activity_timeout_out, conv);
     break;
   case TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_IN:
-    ret = _memberp_to_generic(&overridableHttpConfig->transaction_no_activity_timeout_in, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->transaction_no_activity_timeout_in, conv);
     break;
   case TS_CONFIG_HTTP_TRANSACTION_NO_ACTIVITY_TIMEOUT_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->transaction_no_activity_timeout_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->transaction_no_activity_timeout_out, conv);
     break;
   case TS_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->transaction_active_timeout_out, typep);
-    break;
-  case TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS:
-    ret = _memberp_to_generic(&overridableHttpConfig->origin_max_connections, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->transaction_active_timeout_out, conv);
     break;
   case TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES:
-    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_max_retries, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_max_retries, conv);
     break;
   case TS_CONFIG_HTTP_CONNECT_ATTEMPTS_MAX_RETRIES_DEAD_SERVER:
-    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_max_retries_dead_server, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_max_retries_dead_server, conv);
     break;
   case TS_CONFIG_HTTP_CONNECT_ATTEMPTS_RR_RETRIES:
-    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_rr_retries, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_rr_retries, conv);
     break;
   case TS_CONFIG_HTTP_CONNECT_ATTEMPTS_TIMEOUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_timeout, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->connect_attempts_timeout, conv);
     break;
   case TS_CONFIG_HTTP_POST_CONNECT_ATTEMPTS_TIMEOUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->post_connect_attempts_timeout, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->post_connect_attempts_timeout, conv);
     break;
   case TS_CONFIG_HTTP_DOWN_SERVER_CACHE_TIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->down_server_timeout, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->down_server_timeout, conv);
     break;
   case TS_CONFIG_HTTP_DOWN_SERVER_ABORT_THRESHOLD:
-    ret = _memberp_to_generic(&overridableHttpConfig->client_abort_threshold, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->client_abort_threshold, conv);
     break;
   case TS_CONFIG_HTTP_DOC_IN_CACHE_SKIP_DNS:
-    ret = _memberp_to_generic(&overridableHttpConfig->doc_in_cache_skip_dns, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->doc_in_cache_skip_dns, conv);
     break;
   case TS_CONFIG_HTTP_BACKGROUND_FILL_ACTIVE_TIMEOUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->background_fill_active_timeout, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->background_fill_active_timeout, conv);
     break;
   case TS_CONFIG_HTTP_RESPONSE_SERVER_STR:
-    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_server_string, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_server_string, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_HEURISTIC_LM_FACTOR:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_heuristic_lm_factor, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_heuristic_lm_factor, conv);
     break;
   case TS_CONFIG_HTTP_BACKGROUND_FILL_COMPLETED_THRESHOLD:
-    ret = _memberp_to_generic(&overridableHttpConfig->background_fill_threshold, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->background_fill_threshold, conv);
     break;
   case TS_CONFIG_NET_SOCK_PACKET_MARK_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->sock_packet_mark_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->sock_packet_mark_out, conv);
     break;
   case TS_CONFIG_NET_SOCK_PACKET_TOS_OUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->sock_packet_tos_out, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->sock_packet_tos_out, conv);
     break;
   case TS_CONFIG_HTTP_INSERT_AGE_IN_RESPONSE:
-    ret = _memberp_to_generic(&overridableHttpConfig->insert_age_in_response, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->insert_age_in_response, conv);
     break;
   case TS_CONFIG_HTTP_CHUNKING_SIZE:
-    ret = _memberp_to_generic(&overridableHttpConfig->http_chunking_size, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->http_chunking_size, conv);
     break;
   case TS_CONFIG_HTTP_FLOW_CONTROL_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->flow_control_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->flow_control_enabled, conv);
     break;
   case TS_CONFIG_HTTP_FLOW_CONTROL_LOW_WATER_MARK:
-    ret = _memberp_to_generic(&overridableHttpConfig->flow_low_water_mark, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->flow_low_water_mark, conv);
     break;
   case TS_CONFIG_HTTP_FLOW_CONTROL_HIGH_WATER_MARK:
-    ret = _memberp_to_generic(&overridableHttpConfig->flow_high_water_mark, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->flow_high_water_mark, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_RANGE_LOOKUP:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_range_lookup, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_range_lookup, conv);
     break;
   case TS_CONFIG_HTTP_NORMALIZE_AE:
-    ret = _memberp_to_generic(&overridableHttpConfig->normalize_ae, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->normalize_ae, conv);
     break;
   case TS_CONFIG_HTTP_DEFAULT_BUFFER_SIZE:
-    ret = _memberp_to_generic(&overridableHttpConfig->default_buffer_size_index, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->default_buffer_size_index, conv);
     break;
   case TS_CONFIG_HTTP_DEFAULT_BUFFER_WATER_MARK:
-    ret = _memberp_to_generic(&overridableHttpConfig->default_buffer_water_mark, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->default_buffer_water_mark, conv);
     break;
   case TS_CONFIG_HTTP_REQUEST_HEADER_MAX_SIZE:
-    ret = _memberp_to_generic(&overridableHttpConfig->request_hdr_max_size, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->request_hdr_max_size, conv);
     break;
   case TS_CONFIG_HTTP_RESPONSE_HEADER_MAX_SIZE:
-    ret = _memberp_to_generic(&overridableHttpConfig->response_hdr_max_size, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->response_hdr_max_size, conv);
     break;
   case TS_CONFIG_HTTP_NEGATIVE_REVALIDATING_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->negative_revalidating_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->negative_revalidating_enabled, conv);
     break;
   case TS_CONFIG_HTTP_NEGATIVE_REVALIDATING_LIFETIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->negative_revalidating_lifetime, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->negative_revalidating_lifetime, conv);
     break;
   case TS_CONFIG_SSL_HSTS_MAX_AGE:
-    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_hsts_max_age, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_hsts_max_age, conv);
     break;
   case TS_CONFIG_SSL_HSTS_INCLUDE_SUBDOMAINS:
-    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_hsts_include_subdomains, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->proxy_response_hsts_include_subdomains, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_OPEN_READ_RETRY_TIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_open_read_retry_time, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_open_read_retry_time, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_MAX_OPEN_READ_RETRIES:
-    ret = _memberp_to_generic(&overridableHttpConfig->max_cache_open_read_retries, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->max_cache_open_read_retries, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_RANGE_WRITE:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_range_write, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_range_write, conv);
     break;
   case TS_CONFIG_HTTP_POST_CHECK_CONTENT_LENGTH_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->post_check_content_length_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->post_check_content_length_enabled, conv);
     break;
   case TS_CONFIG_HTTP_REQUEST_BUFFER_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->request_buffer_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->request_buffer_enabled, conv);
     break;
   case TS_CONFIG_HTTP_GLOBAL_USER_AGENT_HEADER:
-    ret = _memberp_to_generic(&overridableHttpConfig->global_user_agent_header, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->global_user_agent_header, conv);
     break;
   case TS_CONFIG_HTTP_AUTH_SERVER_SESSION_PRIVATE:
-    ret = _memberp_to_generic(&overridableHttpConfig->auth_server_session_private, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->auth_server_session_private, conv);
     break;
   case TS_CONFIG_HTTP_SLOW_LOG_THRESHOLD:
-    ret = _memberp_to_generic(&overridableHttpConfig->slow_log_threshold, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->slow_log_threshold, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_GENERATION:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_generation_number, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_generation_number, conv);
     break;
   case TS_CONFIG_BODY_FACTORY_TEMPLATE_BASE:
-    ret = _memberp_to_generic(&overridableHttpConfig->body_factory_template_base, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->body_factory_template_base, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_OPEN_WRITE_FAIL_ACTION:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_open_write_fail_action, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_open_write_fail_action, conv);
     break;
   case TS_CONFIG_HTTP_NUMBER_OF_REDIRECTIONS:
-    ret = _memberp_to_generic(&overridableHttpConfig->number_of_redirections, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->number_of_redirections, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_MAX_OPEN_WRITE_RETRIES:
-    ret = _memberp_to_generic(&overridableHttpConfig->max_cache_open_write_retries, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->max_cache_open_write_retries, conv);
     break;
   case TS_CONFIG_HTTP_REDIRECT_USE_ORIG_CACHE_KEY:
-    ret = _memberp_to_generic(&overridableHttpConfig->redirect_use_orig_cache_key, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->redirect_use_orig_cache_key, conv);
     break;
   case TS_CONFIG_HTTP_ATTACH_SERVER_SESSION_TO_CLIENT:
-    ret = _memberp_to_generic(&overridableHttpConfig->attach_server_session_to_client, typep);
-    break;
-  case TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS_QUEUE:
-    ret = _memberp_to_generic(&overridableHttpConfig->origin_max_connections_queue, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->attach_server_session_to_client, conv);
     break;
   case TS_CONFIG_WEBSOCKET_NO_ACTIVITY_TIMEOUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->websocket_inactive_timeout, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->websocket_inactive_timeout, conv);
     break;
   case TS_CONFIG_WEBSOCKET_ACTIVE_TIMEOUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->websocket_active_timeout, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->websocket_active_timeout, conv);
     break;
   case TS_CONFIG_HTTP_UNCACHEABLE_REQUESTS_BYPASS_PARENT:
-    ret = _memberp_to_generic(&overridableHttpConfig->uncacheable_requests_bypass_parent, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->uncacheable_requests_bypass_parent, conv);
     break;
   case TS_CONFIG_HTTP_PARENT_PROXY_TOTAL_CONNECT_ATTEMPTS:
-    ret = _memberp_to_generic(&overridableHttpConfig->parent_connect_attempts, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->parent_connect_attempts, conv);
     break;
   case TS_CONFIG_HTTP_TRANSACTION_ACTIVE_TIMEOUT_IN:
-    ret = _memberp_to_generic(&overridableHttpConfig->transaction_active_timeout_in, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->transaction_active_timeout_in, conv);
     break;
   case TS_CONFIG_SRV_ENABLED:
-    ret = _memberp_to_generic(&overridableHttpConfig->srv_enabled, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->srv_enabled, conv);
     break;
   case TS_CONFIG_HTTP_FORWARD_CONNECT_METHOD:
-    ret = _memberp_to_generic(&overridableHttpConfig->forward_connect_method, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->forward_connect_method, conv);
     break;
   case TS_CONFIG_SSL_CERT_FILENAME:
-    ret = _memberp_to_generic(&overridableHttpConfig->client_cert_filename, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->client_cert_filename, conv);
     break;
   case TS_CONFIG_SSL_CERT_FILEPATH:
-    ret = _memberp_to_generic(&overridableHttpConfig->client_cert_filepath, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->client_cert_filepath, conv);
     break;
   case TS_CONFIG_PARENT_FAILURES_UPDATE_HOSTDB:
-    ret = _memberp_to_generic(&overridableHttpConfig->parent_failures_update_hostdb, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->parent_failures_update_hostdb, conv);
     break;
   case TS_CONFIG_SSL_CLIENT_VERIFY_SERVER:
-    ret = _memberp_to_generic(&overridableHttpConfig->ssl_client_verify_server, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->ssl_client_verify_server, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_ENABLE_DEFAULT_VARY_HEADER:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_enable_default_vary_headers, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_enable_default_vary_headers, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_VARY_DEFAULT_TEXT:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_vary_default_text, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_vary_default_text, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_VARY_DEFAULT_IMAGES:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_vary_default_images, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_vary_default_images, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_VARY_DEFAULT_OTHER:
-    ret = _memberp_to_generic(&overridableHttpConfig->cache_vary_default_other, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->cache_vary_default_other, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_ACCEPT_MISMATCH:
-    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_mismatch, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_mismatch, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_ACCEPT_LANGUAGE_MISMATCH:
-    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_language_mismatch, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_language_mismatch, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_ACCEPT_ENCODING_MISMATCH:
-    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_encoding_mismatch, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_encoding_mismatch, conv);
     break;
   case TS_CONFIG_HTTP_CACHE_IGNORE_ACCEPT_CHARSET_MISMATCH:
-    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_charset_mismatch, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->ignore_accept_charset_mismatch, conv);
     break;
   case TS_CONFIG_HTTP_PARENT_PROXY_FAIL_THRESHOLD:
-    ret = _memberp_to_generic(&overridableHttpConfig->parent_fail_threshold, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->parent_fail_threshold, conv);
     break;
   case TS_CONFIG_HTTP_PARENT_PROXY_RETRY_TIME:
-    ret = _memberp_to_generic(&overridableHttpConfig->parent_retry_time, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->parent_retry_time, conv);
     break;
   case TS_CONFIG_HTTP_PER_PARENT_CONNECT_ATTEMPTS:
-    ret = _memberp_to_generic(&overridableHttpConfig->per_parent_connect_attempts, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->per_parent_connect_attempts, conv);
     break;
   case TS_CONFIG_HTTP_PARENT_CONNECT_ATTEMPT_TIMEOUT:
-    ret = _memberp_to_generic(&overridableHttpConfig->parent_connect_timeout, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->parent_connect_timeout, conv);
     break;
   case TS_CONFIG_HTTP_ALLOW_MULTI_RANGE:
-    ret = _memberp_to_generic(&overridableHttpConfig->allow_multi_range, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->allow_multi_range, conv);
     break;
   case TS_CONFIG_HTTP_ALLOW_HALF_OPEN:
-    ret = _memberp_to_generic(&overridableHttpConfig->allow_half_open, typep);
+    ret = _memberp_to_generic(&overridableHttpConfig->allow_half_open, conv);
+    break;
+  case TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX:
+    ret  = &overridableHttpConfig->outbound_conntrack.max;
+    conv = &OutboundConnTrack::MAX_CONV;
+    break;
+  case TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH:
+    ret  = &overridableHttpConfig->outbound_conntrack.match;
+    conv = &OutboundConnTrack::MATCH_CONV;
     break;
   // This helps avoiding compiler warnings, yet detect unhandled enum members.
   case TS_CONFIG_NULL:
@@ -8180,23 +8221,17 @@ TSHttpTxnConfigIntSet(TSHttpTxn txnp, TSOverridableConfigKey conf, TSMgmtInt val
   sdk_assert(sdk_sanity_check_txn(txnp) == TS_SUCCESS);
 
   HttpSM *s = reinterpret_cast<HttpSM *>(txnp);
-  std::type_info const *type;
+  MgmtConverter const *conv;
 
   s->t_state.setup_per_txn_configs();
 
-  void *dest = _conf_to_memberp(conf, s->t_state.txn_conf, type);
+  void *dest = _conf_to_memberp(conf, s->t_state.txn_conf, conv);
 
-  if (!dest) {
+  if (!dest || !conv->set_int) {
     return TS_ERROR;
   }
 
-  if (*type == TYPE_INFO_MGMT_INT) {
-    *(static_cast<TSMgmtInt *>(dest)) = value;
-  } else if (*type == TYPE_INFO_MGMT_BYTE) {
-    *(static_cast<TSMgmtByte *>(dest)) = static_cast<TSMgmtByte>(value);
-  } else {
-    return TS_ERROR;
-  }
+  conv->set_int(dest, value);
 
   return TS_SUCCESS;
 }
@@ -8208,19 +8243,15 @@ TSHttpTxnConfigIntGet(TSHttpTxn txnp, TSOverridableConfigKey conf, TSMgmtInt *va
   sdk_assert(sdk_sanity_check_null_ptr((void *)value) == TS_SUCCESS);
 
   HttpSM *s = reinterpret_cast<HttpSM *>(txnp);
-  std::type_info const *type;
-  void *src = _conf_to_memberp(conf, s->t_state.txn_conf, type);
+  MgmtConverter const *conv;
+  void *src = _conf_to_memberp(conf, s->t_state.txn_conf, conv);
 
-  if (!src) {
-    return TS_ERROR;
-  } else if (*type == TYPE_INFO_MGMT_INT) {
-    *value = *(static_cast<TSMgmtInt *>(src));
-  } else if (*type == TYPE_INFO_MGMT_BYTE) {
-    *value = *(static_cast<TSMgmtByte *>(src));
-  } else {
+  if (!src || !conv->get_int) {
     return TS_ERROR;
   }
 
+  *value = conv->get_int(src);
+
   return TS_SUCCESS;
 }
 
@@ -8230,18 +8261,18 @@ TSHttpTxnConfigFloatSet(TSHttpTxn txnp, TSOverridableConfigKey conf, TSMgmtFloat
   sdk_assert(sdk_sanity_check_txn(txnp) == TS_SUCCESS);
 
   HttpSM *s = reinterpret_cast<HttpSM *>(txnp);
-  std::type_info const *type;
+  MgmtConverter const *conv;
 
   s->t_state.setup_per_txn_configs();
 
-  TSMgmtFloat *dest = static_cast<TSMgmtFloat *>(_conf_to_memberp(conf, s->t_state.txn_conf, type));
+  void *dest = _conf_to_memberp(conf, s->t_state.txn_conf, conv);
 
-  if (!dest || *type != TYPE_INFO_MGMT_FLOAT) {
+  if (!dest || !conv->set_float) {
     return TS_ERROR;
-  } else {
-    *dest = value;
   }
 
+  conv->set_float(dest, value);
+
   return TS_SUCCESS;
 }
 
@@ -8249,16 +8280,15 @@ TSReturnCode
 TSHttpTxnConfigFloatGet(TSHttpTxn txnp, TSOverridableConfigKey conf, TSMgmtFloat *value)
 {
   sdk_assert(sdk_sanity_check_txn(txnp) == TS_SUCCESS);
-  sdk_assert(sdk_sanity_check_null_ptr((void *)value) == TS_SUCCESS);
+  sdk_assert(sdk_sanity_check_null_ptr(static_cast<void *>(value)) == TS_SUCCESS);
 
-  std::type_info const *type;
-  TSMgmtFloat *dest = static_cast<TSMgmtFloat *>(_conf_to_memberp(conf, ((HttpSM *)txnp)->t_state.txn_conf, type));
+  MgmtConverter const *conv;
+  void *src = _conf_to_memberp(conf, reinterpret_cast<HttpSM *>(txnp)->t_state.txn_conf, conv);
 
-  if (dest && *type == TYPE_INFO_MGMT_FLOAT) {
-    *value = *dest;
-  } else {
+  if (!src || !conv->get_float) {
     return TS_ERROR;
   }
+  *value = conv->get_float(src);
 
   return TS_SUCCESS;
 }
@@ -8272,7 +8302,7 @@ TSHttpTxnConfigStringSet(TSHttpTxn txnp, TSOverridableConfigKey conf, const char
     length = strlen(value);
   }
 
-  HttpSM *s = (HttpSM *)txnp;
+  HttpSM *s = reinterpret_cast<HttpSM *>(txnp);
 
   s->t_state.setup_per_txn_configs();
 
@@ -8325,10 +8355,17 @@ TSHttpTxnConfigStringSet(TSHttpTxn txnp, TSOverridableConfigKey conf, const char
       }
     }
     break;
-  default:
-    return TS_ERROR;
+  default: {
+    MgmtConverter const *conv;
+    void *dest = _conf_to_memberp(conf, s->t_state.txn_conf, conv);
+    if (dest != nullptr && conv != nullptr && conv->set_string) {
+      conv->set_string(dest, std::string_view(value, length));
+    } else {
+      return TS_ERROR;
+    }
     break;
   }
+  }
 
   return TS_SUCCESS;
 }
@@ -8340,7 +8377,7 @@ TSHttpTxnConfigStringGet(TSHttpTxn txnp, TSOverridableConfigKey conf, const char
   sdk_assert(sdk_sanity_check_null_ptr((void **)value) == TS_SUCCESS);
   sdk_assert(sdk_sanity_check_null_ptr((void *)length) == TS_SUCCESS);
 
-  HttpSM *sm = (HttpSM *)txnp;
+  HttpSM *sm = reinterpret_cast<HttpSM *>(txnp);
 
   switch (conf) {
   case TS_CONFIG_HTTP_RESPONSE_SERVER_STR:
@@ -8355,10 +8392,19 @@ TSHttpTxnConfigStringGet(TSHttpTxn txnp, TSOverridableConfigKey conf, const char
     *value  = sm->t_state.txn_conf->body_factory_template_base;
     *length = sm->t_state.txn_conf->body_factory_template_base_len;
     break;
-  default:
-    return TS_ERROR;
+  default: {
+    MgmtConverter const *conv;
+    void *src = _conf_to_memberp(conf, sm->t_state.txn_conf, conv);
+    if (src != nullptr && conv != nullptr && conv->get_string) {
+      auto sv = conv->get_string(src);
+      *value  = sv.data();
+      *length = sv.size();
+    } else {
+      return TS_ERROR;
+    }
     break;
   }
+  }
 
   return TS_SUCCESS;
 }
@@ -8367,15 +8413,18 @@ TSHttpTxnConfigStringGet(TSHttpTxn txnp, TSOverridableConfigKey conf, const char
 TSReturnCode
 TSHttpTxnConfigFind(const char *name, int length, TSOverridableConfigKey *conf, TSRecordDataType *type)
 {
-  sdk_assert(sdk_sanity_check_null_ptr((void *)name) == TS_SUCCESS);
-  sdk_assert(sdk_sanity_check_null_ptr((void *)conf) == TS_SUCCESS);
+  sdk_assert(sdk_sanity_check_null_ptr(name) == TS_SUCCESS);
+  sdk_assert(sdk_sanity_check_null_ptr(conf) == TS_SUCCESS);
 
   TSOverridableConfigKey cnf = TS_CONFIG_NULL;
   TSRecordDataType typ       = TS_RECORDDATATYPE_INT;
 
-  if (length == -1) {
+  if (length < 0) {
     length = strlen(name);
   }
+
+  std::string_view name_sv(name, length);
+
   // Lots of string comparisons here, but we avoid quite a few by checking lengths
   switch (length) {
   case 24:
@@ -8561,9 +8610,7 @@ TSHttpTxnConfigFind(const char *name, int length, TSOverridableConfigKey *conf,
       }
       break;
     case 's':
-      if (!strncmp(name, "proxy.config.http.origin_max_connections", length)) {
-        cnf = TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS;
-      } else if (!strncmp(name, "proxy.config.http.cache.required_headers", length)) {
+      if (!strncmp(name, "proxy.config.http.cache.required_headers", length)) {
         cnf = TS_CONFIG_HTTP_CACHE_REQUIRED_HEADERS;
       } else if (!strncmp(name, "proxy.config.ssl.hsts_include_subdomains", length)) {
         cnf = TS_CONFIG_SSL_HSTS_INCLUDE_SUBDOMAINS;
@@ -8674,6 +8721,11 @@ TSHttpTxnConfigFind(const char *name, int length, TSOverridableConfigKey *conf,
         typ = TS_RECORDDATATYPE_STRING;
       }
       break;
+    case 'x':
+      if (name_sv == OutboundConnTrack::CONFIG_VAR_MAX) {
+        cnf = TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MAX;
+      }
+      break;
     }
     break;
 
@@ -8701,6 +8753,11 @@ TSHttpTxnConfigFind(const char *name, int length, TSOverridableConfigKey *conf,
         cnf = TS_CONFIG_HTTP_PARENT_PROXY_FAIL_THRESHOLD;
       }
       break;
+    case 'h':
+      if (name_sv == OutboundConnTrack::CONFIG_VAR_MATCH) {
+        cnf = TS_CONFIG_HTTP_PER_SERVER_CONNECTION_MATCH;
+      }
+      break;
     case 'n':
       if (!strncmp(name, "proxy.config.http.cache.ignore_authentication", length)) {
         cnf = TS_CONFIG_HTTP_CACHE_IGNORE_AUTHENTICATION;
@@ -8744,8 +8801,6 @@ TSHttpTxnConfigFind(const char *name, int length, TSOverridableConfigKey *conf,
         cnf = TS_CONFIG_HTTP_CACHE_HEURISTIC_MIN_LIFETIME;
       } else if (!strncmp(name, "proxy.config.http.cache.heuristic_max_lifetime", length)) {
         cnf = TS_CONFIG_HTTP_CACHE_HEURISTIC_MAX_LIFETIME;
-      } else if (!strncmp(name, "proxy.config.http.origin_max_connections_queue", length)) {
-        cnf = TS_CONFIG_HTTP_ORIGIN_MAX_CONNECTIONS_QUEUE;
       }
       break;
     case 'h':
diff --git a/src/traffic_server/InkAPITest.cc b/src/traffic_server/InkAPITest.cc
index 688cbe3..fb47d87 100644
--- a/src/traffic_server/InkAPITest.cc
+++ b/src/traffic_server/InkAPITest.cc
@@ -8572,126 +8572,126 @@ EXCLUSIVE_REGRESSION_TEST(SDK_API_TSHttpConnectServerIntercept)(RegressionTest *
 ////////////////////////////////////////////////
 
 // The order of these should be the same as TSOverridableConfigKey
-const char *SDK_Overridable_Configs[TS_CONFIG_LAST_ENTRY] = {"proxy.config.url_remap.pristine_host_hdr",
-                                                             "proxy.config.http.chunking_enabled",
-                                                             "proxy.config.http.negative_caching_enabled",
-                                                             "proxy.config.http.negative_caching_lifetime",
-                                                             "proxy.config.http.cache.when_to_revalidate",
-                                                             "proxy.config.http.keep_alive_enabled_in",
-                                                             "proxy.config.http.keep_alive_enabled_out",
-                                                             "proxy.config.http.keep_alive_post_out",
-                                                             "proxy.config.http.server_session_sharing.match",
-                                                             "proxy.config.net.sock_recv_buffer_size_out",
-                                                             "proxy.config.net.sock_send_buffer_size_out",
-                                                             "proxy.config.net.sock_option_flag_out",
-                                                             "proxy.config.http.forward.proxy_auth_to_parent",
-                                                             "proxy.config.http.anonymize_remove_from",
-                                                             "proxy.config.http.anonymize_remove_referer",
-                                                             "proxy.config.http.anonymize_remove_user_agent",
-                                                             "proxy.config.http.anonymize_remove_cookie",
-                                                             "proxy.config.http.anonymize_remove_client_ip",
-                                                             "proxy.config.http.insert_client_ip",
-                                                             "proxy.config.http.response_server_enabled",
-                                                             "proxy.config.http.insert_squid_x_forwarded_for",
-                                                             "proxy.config.http.server_tcp_init_cwnd",
-                                                             "proxy.config.http.send_http11_requests",
-                                                             "proxy.config.http.cache.http",
-                                                             "proxy.config.http.cache.ignore_client_no_cache",
-                                                             "proxy.config.http.cache.ignore_client_cc_max_age",
-                                                             "proxy.config.http.cache.ims_on_client_no_cache",
-                                                             "proxy.config.http.cache.ignore_server_no_cache",
-                                                             "proxy.config.http.cache.cache_responses_to_cookies",
-                                                             "proxy.config.http.cache.ignore_authentication",
-                                                             "proxy.config.http.cache.cache_urls_that_look_dynamic",
-                                                             "proxy.config.http.cache.required_headers",
-                                                             "proxy.config.http.insert_request_via_str",
-                                                             "proxy.config.http.insert_response_via_str",
-                                                             "proxy.config.http.cache.heuristic_min_lifetime",
-                                                             "proxy.config.http.cache.heuristic_max_lifetime",
-                                                             "proxy.config.http.cache.guaranteed_min_lifetime",
-                                                             "proxy.config.http.cache.guaranteed_max_lifetime",
-                                                             "proxy.config.http.cache.max_stale_age",
-                                                             "proxy.config.http.keep_alive_no_activity_timeout_in",
-                                                             "proxy.config.http.keep_alive_no_activity_timeout_out",
-                                                             "proxy.config.http.transaction_no_activity_timeout_in",
-                                                             "proxy.config.http.transaction_no_activity_timeout_out",
-                                                             "proxy.config.http.transaction_active_timeout_out",
-                                                             "proxy.config.http.origin_max_connections",
-                                                             "proxy.config.http.connect_attempts_max_retries",
-                                                             "proxy.config.http.connect_attempts_max_retries_dead_server",
-                                                             "proxy.config.http.connect_attempts_rr_retries",
-                                                             "proxy.config.http.connect_attempts_timeout",
-                                                             "proxy.config.http.post_connect_attempts_timeout",
-                                                             "proxy.config.http.down_server.cache_time",
-                                                             "proxy.config.http.down_server.abort_threshold",
-                                                             "proxy.config.http.doc_in_cache_skip_dns",
-                                                             "proxy.config.http.background_fill_active_timeout",
-                                                             "proxy.config.http.response_server_str",
-                                                             "proxy.config.http.cache.heuristic_lm_factor",
-                                                             "proxy.config.http.background_fill_completed_threshold",
-                                                             "proxy.config.net.sock_packet_mark_out",
-                                                             "proxy.config.net.sock_packet_tos_out",
-                                                             "proxy.config.http.insert_age_in_response",
-                                                             "proxy.config.http.chunking.size",
-                                                             "proxy.config.http.flow_control.enabled",
-                                                             "proxy.config.http.flow_control.low_water",
-                                                             "proxy.config.http.flow_control.high_water",
-                                                             "proxy.config.http.cache.range.lookup",
-                                                             "proxy.config.http.default_buffer_size",
-                                                             "proxy.config.http.default_buffer_water_mark",
-                                                             "proxy.config.http.request_header_max_size",
-                                                             "proxy.config.http.response_header_max_size",
-                                                             "proxy.config.http.negative_revalidating_enabled",
-                                                             "proxy.config.http.negative_revalidating_lifetime",
-                                                             "proxy.config.ssl.hsts_max_age",
-                                                             "proxy.config.ssl.hsts_include_subdomains",
-                                                             "proxy.config.http.cache.open_read_retry_time",
-                                                             "proxy.config.http.cache.max_open_read_retries",
-                                                             "proxy.config.http.cache.range.write",
-                                                             "proxy.config.http.post.check.content_length.enabled",
-                                                             "proxy.config.http.global_user_agent_header",
-                                                             "proxy.config.http.auth_server_session_private",
-                                                             "proxy.config.http.slow.log.threshold",
-                                                             "proxy.config.http.cache.generation",
-                                                             "proxy.config.body_factory.template_base",
-                                                             "proxy.config.http.cache.open_write_fail_action",
-                                                             "proxy.config.http.number_of_redirections",
-                                                             "proxy.config.http.cache.max_open_write_retries",
-                                                             "proxy.config.http.redirect_use_orig_cache_key",
-                                                             "proxy.config.http.attach_server_session_to_client",
-                                                             "proxy.config.http.origin_max_connections_queue",
-                                                             "proxy.config.websocket.no_activity_timeout",
-                                                             "proxy.config.websocket.active_timeout",
-                                                             "proxy.config.http.uncacheable_requests_bypass_parent",
-                                                             "proxy.config.http.parent_proxy.total_connect_attempts",
-                                                             "proxy.config.http.transaction_active_timeout_in",
-                                                             "proxy.config.srv_enabled",
-                                                             "proxy.config.http.forward_connect_method",
-                                                             "proxy.config.ssl.client.cert.filename",
-                                                             "proxy.config.ssl.client.cert.path",
-                                                             "proxy.config.http.parent_proxy.mark_down_hostdb",
-                                                             "proxy.config.ssl.client.verify.server",
-                                                             "proxy.config.http.cache.enable_default_vary_headers",
-                                                             "proxy.config.http.cache.vary_default_text",
-                                                             "proxy.config.http.cache.vary_default_images",
-                                                             "proxy.config.http.cache.vary_default_other",
-                                                             "proxy.config.http.cache.ignore_accept_mismatch",
-                                                             "proxy.config.http.cache.ignore_accept_language_mismatch",
-                                                             "proxy.config.http.cache.ignore_accept_encoding_mismatch",
-                                                             "proxy.config.http.cache.ignore_accept_charset_mismatch",
-                                                             "proxy.config.http.parent_proxy.fail_threshold",
-                                                             "proxy.config.http.parent_proxy.retry_time",
-                                                             "proxy.config.http.parent_proxy.per_parent_connect_attempts",
-                                                             "proxy.config.http.parent_proxy.connect_attempts_timeout",
-                                                             "proxy.config.http.normalize_ae",
-                                                             "proxy.config.http.insert_forwarded",
-                                                             "proxy.config.http.allow_multi_range",
-                                                             "proxy.config.http.request_buffer_enabled",
-                                                             "proxy.config.http.allow_half_open"};
+std::array<std::string_view, TS_CONFIG_LAST_ENTRY> SDK_Overridable_Configs = {
+  {"proxy.config.url_remap.pristine_host_hdr",
+   "proxy.config.http.chunking_enabled",
+   "proxy.config.http.negative_caching_enabled",
+   "proxy.config.http.negative_caching_lifetime",
+   "proxy.config.http.cache.when_to_revalidate",
+   "proxy.config.http.keep_alive_enabled_in",
+   "proxy.config.http.keep_alive_enabled_out",
+   "proxy.config.http.keep_alive_post_out",
+   "proxy.config.http.server_session_sharing.match",
+   "proxy.config.net.sock_recv_buffer_size_out",
+   "proxy.config.net.sock_send_buffer_size_out",
+   "proxy.config.net.sock_option_flag_out",
+   "proxy.config.http.forward.proxy_auth_to_parent",
+   "proxy.config.http.anonymize_remove_from",
+   "proxy.config.http.anonymize_remove_referer",
+   "proxy.config.http.anonymize_remove_user_agent",
+   "proxy.config.http.anonymize_remove_cookie",
+   "proxy.config.http.anonymize_remove_client_ip",
+   "proxy.config.http.insert_client_ip",
+   "proxy.config.http.response_server_enabled",
+   "proxy.config.http.insert_squid_x_forwarded_for",
+   "proxy.config.http.server_tcp_init_cwnd",
+   "proxy.config.http.send_http11_requests",
+   "proxy.config.http.cache.http",
+   "proxy.config.http.cache.ignore_client_no_cache",
+   "proxy.config.http.cache.ignore_client_cc_max_age",
+   "proxy.config.http.cache.ims_on_client_no_cache",
+   "proxy.config.http.cache.ignore_server_no_cache",
+   "proxy.config.http.cache.cache_responses_to_cookies",
+   "proxy.config.http.cache.ignore_authentication",
+   "proxy.config.http.cache.cache_urls_that_look_dynamic",
+   "proxy.config.http.cache.required_headers",
+   "proxy.config.http.insert_request_via_str",
+   "proxy.config.http.insert_response_via_str",
+   "proxy.config.http.cache.heuristic_min_lifetime",
+   "proxy.config.http.cache.heuristic_max_lifetime",
+   "proxy.config.http.cache.guaranteed_min_lifetime",
+   "proxy.config.http.cache.guaranteed_max_lifetime",
+   "proxy.config.http.cache.max_stale_age",
+   "proxy.config.http.keep_alive_no_activity_timeout_in",
+   "proxy.config.http.keep_alive_no_activity_timeout_out",
+   "proxy.config.http.transaction_no_activity_timeout_in",
+   "proxy.config.http.transaction_no_activity_timeout_out",
+   "proxy.config.http.transaction_active_timeout_out",
+   "proxy.config.http.connect_attempts_max_retries",
+   "proxy.config.http.connect_attempts_max_retries_dead_server",
+   "proxy.config.http.connect_attempts_rr_retries",
+   "proxy.config.http.connect_attempts_timeout",
+   "proxy.config.http.post_connect_attempts_timeout",
+   "proxy.config.http.down_server.cache_time",
+   "proxy.config.http.down_server.abort_threshold",
+   "proxy.config.http.doc_in_cache_skip_dns",
+   "proxy.config.http.background_fill_active_timeout",
+   "proxy.config.http.response_server_str",
+   "proxy.config.http.cache.heuristic_lm_factor",
+   "proxy.config.http.background_fill_completed_threshold",
+   "proxy.config.net.sock_packet_mark_out",
+   "proxy.config.net.sock_packet_tos_out",
+   "proxy.config.http.insert_age_in_response",
+   "proxy.config.http.chunking.size",
+   "proxy.config.http.flow_control.enabled",
+   "proxy.config.http.flow_control.low_water",
+   "proxy.config.http.flow_control.high_water",
+   "proxy.config.http.cache.range.lookup",
+   "proxy.config.http.default_buffer_size",
+   "proxy.config.http.default_buffer_water_mark",
+   "proxy.config.http.request_header_max_size",
+   "proxy.config.http.response_header_max_size",
+   "proxy.config.http.negative_revalidating_enabled",
+   "proxy.config.http.negative_revalidating_lifetime",
+   "proxy.config.ssl.hsts_max_age",
+   "proxy.config.ssl.hsts_include_subdomains",
+   "proxy.config.http.cache.open_read_retry_time",
+   "proxy.config.http.cache.max_open_read_retries",
+   "proxy.config.http.cache.range.write",
+   "proxy.config.http.post.check.content_length.enabled",
+   "proxy.config.http.global_user_agent_header",
+   "proxy.config.http.auth_server_session_private",
+   "proxy.config.http.slow.log.threshold",
+   "proxy.config.http.cache.generation",
+   "proxy.config.body_factory.template_base",
+   "proxy.config.http.cache.open_write_fail_action",
+   "proxy.config.http.number_of_redirections",
+   "proxy.config.http.cache.max_open_write_retries",
+   "proxy.config.http.redirect_use_orig_cache_key",
+   "proxy.config.http.attach_server_session_to_client",
+   "proxy.config.websocket.no_activity_timeout",
+   "proxy.config.websocket.active_timeout",
+   "proxy.config.http.uncacheable_requests_bypass_parent",
+   "proxy.config.http.parent_proxy.total_connect_attempts",
+   "proxy.config.http.transaction_active_timeout_in",
+   "proxy.config.srv_enabled",
+   "proxy.config.http.forward_connect_method",
+   "proxy.config.ssl.client.cert.filename",
+   "proxy.config.ssl.client.cert.path",
+   "proxy.config.http.parent_proxy.mark_down_hostdb",
+   "proxy.config.ssl.client.verify.server",
+   "proxy.config.http.cache.enable_default_vary_headers",
+   "proxy.config.http.cache.vary_default_text",
+   "proxy.config.http.cache.vary_default_images",
+   "proxy.config.http.cache.vary_default_other",
+   "proxy.config.http.cache.ignore_accept_mismatch",
+   "proxy.config.http.cache.ignore_accept_language_mismatch",
+   "proxy.config.http.cache.ignore_accept_encoding_mismatch",
+   "proxy.config.http.cache.ignore_accept_charset_mismatch",
+   "proxy.config.http.parent_proxy.fail_threshold",
+   "proxy.config.http.parent_proxy.retry_time",
+   "proxy.config.http.parent_proxy.per_parent_connect_attempts",
+   "proxy.config.http.parent_proxy.connect_attempts_timeout",
+   "proxy.config.http.normalize_ae",
+   "proxy.config.http.insert_forwarded",
+   "proxy.config.http.allow_multi_range",
+   "proxy.config.http.request_buffer_enabled",
+   "proxy.config.http.allow_half_open",
+   OutboundConnTrack::CONFIG_VAR_MAX,
+   OutboundConnTrack::CONFIG_VAR_MATCH}};
 
 REGRESSION_TEST(SDK_API_OVERRIDABLE_CONFIGS)(RegressionTest *test, int /* atype ATS_UNUSED */, int *pstatus)
 {
-  const char *conf;
   TSOverridableConfigKey key;
   TSRecordDataType type;
   HttpSM *s      = HttpSM::allocate();
@@ -8707,17 +8707,29 @@ REGRESSION_TEST(SDK_API_OVERRIDABLE_CONFIGS)(RegressionTest *test, int /* atype
   s->init();
 
   *pstatus = REGRESSION_TEST_INPROGRESS;
-  for (int i = TS_CONFIG_NULL + 1; i < TS_CONFIG_LAST_ENTRY; ++i) {
-    conf = SDK_Overridable_Configs[i];
+  for (int i = 0; i < static_cast<int>(SDK_Overridable_Configs.size()); ++i) {
+    std::string_view conf{SDK_Overridable_Configs[i]};
 
-    if (TS_SUCCESS == TSHttpTxnConfigFind(conf, -1, &key, &type)) {
+    if (TS_SUCCESS == TSHttpTxnConfigFind(conf.data(), -1, &key, &type)) {
       if (key != i) {
-        SDK_RPRINT(test, "TSHttpTxnConfigFind", "TestCase1", TC_FAIL, "Failed on %s, expected %d, got %d", conf, i, key);
+        SDK_RPRINT(test, "TSHttpTxnConfigFind", "TestCase1", TC_FAIL, "Failed on %s, expected %d, got %d", conf.data(), i, key);
         success = false;
         continue;
       }
     } else {
-      SDK_RPRINT(test, "TSHttpTxnConfigFind", "TestCase1", TC_FAIL, "Call returned unexpected TS_ERROR for %s", conf);
+      SDK_RPRINT(test, "TSHttpTxnConfigFind", "TestCase1", TC_FAIL, "Call returned unexpected TS_ERROR for %s", conf.data());
+      success = false;
+      continue;
+    }
+
+    if (TS_SUCCESS == TSHttpTxnConfigFind(conf.data(), conf.size(), &key, &type)) {
+      if (key != i) {
+        SDK_RPRINT(test, "TSHttpTxnConfigFind", "TestCase1", TC_FAIL, "Failed on %s, expected %d, got %d", conf.data(), i, key);
+        success = false;
+        continue;
+      }
+    } else {
+      SDK_RPRINT(test, "TSHttpTxnConfigFind", "TestCase1", TC_FAIL, "Call returned unexpected TS_ERROR for %s", conf.data());
       success = false;
       continue;
     }
@@ -8729,7 +8741,8 @@ REGRESSION_TEST(SDK_API_OVERRIDABLE_CONFIGS)(RegressionTest *test, int /* atype
       TSHttpTxnConfigIntSet(txnp, key, ival_rand);
       TSHttpTxnConfigIntGet(txnp, key, &ival_read);
       if (ival_rand != ival_read) {
-        SDK_RPRINT(test, "TSHttpTxnConfigIntSet", "TestCase1", TC_FAIL, "Failed on %s, %d != %d", conf, ival_read, ival_rand);
+        SDK_RPRINT(test, "TSHttpTxnConfigIntSet", "TestCase1", TC_FAIL, "Failed on %s, %d != %d", conf.data(), ival_read,
+                   ival_rand);
         success = false;
         continue;
       }
@@ -8740,7 +8753,8 @@ REGRESSION_TEST(SDK_API_OVERRIDABLE_CONFIGS)(RegressionTest *test, int /* atype
       TSHttpTxnConfigFloatSet(txnp, key, fval_rand);
       TSHttpTxnConfigFloatGet(txnp, key, &fval_read);
       if (fval_rand != fval_read) {
-        SDK_RPRINT(test, "TSHttpTxnConfigFloatSet", "TestCase1", TC_FAIL, "Failed on %s, %f != %f", conf, fval_read, fval_rand);
+        SDK_RPRINT(test, "TSHttpTxnConfigFloatSet", "TestCase1", TC_FAIL, "Failed on %s, %f != %f", conf.data(), fval_read,
+                   fval_rand);
         success = false;
         continue;
       }
@@ -8750,7 +8764,8 @@ REGRESSION_TEST(SDK_API_OVERRIDABLE_CONFIGS)(RegressionTest *test, int /* atype
       TSHttpTxnConfigStringSet(txnp, key, test_string, -1);
       TSHttpTxnConfigStringGet(txnp, key, &sval_read, &len);
       if (test_string != sval_read) {
-        SDK_RPRINT(test, "TSHttpTxnConfigStringSet", "TestCase1", TC_FAIL, "Failed on %s, %s != %s", conf, sval_read, test_string);
+        SDK_RPRINT(test, "TSHttpTxnConfigStringSet", "TestCase1", TC_FAIL, "Failed on %s, %s != %s", conf.data(), sval_read,
+                   test_string);
         success = false;
         continue;
       }