You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by bc...@apache.org on 2019/04/11 23:26:32 UTC

[trafficserver] branch 8.1.x updated: Add hooks for outbound TLS start and close.

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

bcall pushed a commit to branch 8.1.x
in repository https://gitbox.apache.org/repos/asf/trafficserver.git


The following commit(s) were added to refs/heads/8.1.x by this push:
     new a6ce406  Add hooks for outbound TLS start and close.
a6ce406 is described below

commit a6ce406126d510d431baa80b3a09ebd46a792cd4
Author: Susan Hinrichs <sh...@apache.org>
AuthorDate: Thu Oct 4 23:33:00 2018 +0000

    Add hooks for outbound TLS start and close.
    
    (cherry picked from commit 8e309ff9be0931f69503b3b71806e55cf8fcfddd)
---
 .../api/functions/TSNetInvokingGet.en.rst          |  37 ++++++
 doc/developer-guide/api/types/TSHttpHookID.en.rst  |   4 +
 .../hooks-and-transactions/ssl-hooks.en.rst        |  30 ++++-
 include/ts/apidefs.h.in                            |  11 +-
 include/ts/ts.h                                    |  10 ++
 iocore/net/P_SSLNetVConnection.h                   |  11 ++
 iocore/net/SSLNetVConnection.cc                    | 132 +++++++++++++++-----
 proxy/InkAPIInternal.h                             |   2 +
 proxy/http/HttpDebugNames.cc                       |   4 +
 src/traffic_server/InkAPI.cc                       |  28 +++++
 src/traffic_server/InkAPITest.cc                   |   6 +-
 .../tls_hooks/gold/ts-close-out-close.gold         |   3 +
 .../tls_hooks/gold/ts-out-delay-start-2.gold       |   5 +
 .../tls_hooks/gold/ts-out-start-close-2.gold       |   4 +
 tests/gold_tests/tls_hooks/tls_hooks10.test.py     |   7 --
 tests/gold_tests/tls_hooks/tls_hooks12.test.py     |  15 ---
 .../{tls_hooks10.test.py => tls_hooks13.test.py}   |  24 ++--
 .../{tls_hooks10.test.py => tls_hooks14.test.py}   |  26 ++--
 .../{tls_hooks10.test.py => tls_hooks15.test.py}   |  24 ++--
 tests/tools/plugins/ssl_hook_test.cc               | 134 ++++++++++++++++++++-
 20 files changed, 398 insertions(+), 119 deletions(-)

diff --git a/doc/developer-guide/api/functions/TSNetInvokingGet.en.rst b/doc/developer-guide/api/functions/TSNetInvokingGet.en.rst
new file mode 100644
index 0000000..876196e
--- /dev/null
+++ b/doc/developer-guide/api/functions/TSNetInvokingGet.en.rst
@@ -0,0 +1,37 @@
+.. Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed
+   with this work for additional information regarding copyright
+   ownership.  The ASF licenses this file to you under the Apache
+   License, Version 2.0 (the "License"); you may not use this file
+   except in compliance with the License.  You may obtain a copy of
+   the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+   implied.  See the License for the specific language governing
+   permissions and limitations under the License.
+
+.. include:: ../../../common.defs
+
+.. default-domain:: c
+
+TSNetInvokingContGet
+********************
+
+Synopsis
+========
+
+`#include <ts/ts.h>`
+
+.. function:: TSCont TSNetInvokingContGet(TSVConn conn)
+
+.. function:: TSHttpTxn TSNetInvokingTxnGet(TSVConn conn)
+
+Description
+===========
+
+The TSNetInvokingContGet and TSNetInvokingTxnGet returns the continuation or transaction that
+started the connection request associated with the conn parameter.
diff --git a/doc/developer-guide/api/types/TSHttpHookID.en.rst b/doc/developer-guide/api/types/TSHttpHookID.en.rst
index 852b672..dd09fbc 100644
--- a/doc/developer-guide/api/types/TSHttpHookID.en.rst
+++ b/doc/developer-guide/api/types/TSHttpHookID.en.rst
@@ -72,8 +72,12 @@ Enumeration Members
 
 .. c:macro:: TSHttpHookID TS_VCONN_START_HOOK
 
+.. c:macro:: TSHttpHookID TS_VCONN_OUTBOUND_START_HOOK
+
 .. c:macro:: TSHttpHookID TS_VCONN_CLOSE_HOOK
 
+.. c:macro:: TSHttpHookID TS_VCONN_OUTBOUND_CLOSE_HOOK
+
 .. c:macro:: TSHttpHookID TS_SSL_SNI_HOOK
 
 .. c:macro:: TSHttpHookID TS_SSL_CERT_HOOK
diff --git a/doc/developer-guide/plugins/hooks-and-transactions/ssl-hooks.en.rst b/doc/developer-guide/plugins/hooks-and-transactions/ssl-hooks.en.rst
index 1fa1a20..bb56447 100644
--- a/doc/developer-guide/plugins/hooks-and-transactions/ssl-hooks.en.rst
+++ b/doc/developer-guide/plugins/hooks-and-transactions/ssl-hooks.en.rst
@@ -58,7 +58,7 @@ callback or from another piece of code.
 TS_VCONN_CLOSE_HOOK
 ------------------------
 
-This hook is invoked after the SSL handshake is done and when the IO is closing. The TSVConnArgs should be cleaned up here.
+This hook is invoked after the SSL handshake is done and when the IO is closing. The TSVConnArgs should be cleaned up here. A callback at this point must reenable.
 
 TS_SSL_SERVERNAME_HOOK
 ----------------------
@@ -107,11 +107,27 @@ Processing will continue regardless of whether the hook callback executes
 :c:func:`TSSslVConnReenable()` since the openssl implementation does not allow
 for pausing processing during the certificate verify callback.
 
+TS_VCONN_OUTBOUND_START_HOOK
+----------------------------
+
+This hook is invoked after ATS has connected to the upstream server and before the SSL handshake has started.  This gives the plugin the option of 
+overriding the default SSL connection options on the SSL object.
+
+In theory this hook could apply and be useful for non-SSL connections as well, but at this point this hook is only called in the SSL sequence.
+
+The TLS handshake processing will not proceed until :c:func:`TSSslVConnReenable()` is called either from within the hook
+callback or from another piece of code.
+
+TS_VCONN_OUTBOUND_CLOSE_HOOK
+-----------------------------
+
+This hook is invoked after the SSL handshake is done and right before the outbound connection closes.  A callback at this point must reenable.
+
 TLS Hook State Diagram
 ----------------------
 
 .. graphviz::
-   :alt: TLS Hook State Diagram
+   :alt: TLS Inbound Hook State Diagram
 
    digraph tls_hook_state_diagram{
      HANDSHAKE_HOOKS_PRE -> TS_VCONN_START_HOOK;
@@ -144,4 +160,14 @@ TLS Hook State Diagram
      HANDSHAKE_HOOKS_DONE [shape=box];
    }
 
+.. graphviz::
+   :alt: TLS Outbound Hook State Diagram
+
+   digraph tls_hook_state_diagram{
+     HANDSHAKE_HOOKS_OUTBOUND_PRE -> HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE;
+     HANDSHAKE_HOOKS_PRE_INVOKE -> TSSslVConnReenable;
+     TSSslVConnReenable -> HANDSHAKE_HOOKS_OUTBOUND_PRE;
+     HANDSHAKE_HOOKS_OUTBOUND_PRE -> HANDSHAKE_HOOKS_DONE;
+     HANDSHAKE_HOOKS_DONE -> HANDSHAKE_HOOKS_OUTBOUND_CLOSE;
+   }
 
diff --git a/include/ts/apidefs.h.in b/include/ts/apidefs.h.in
index 862ac31..60a9bae 100644
--- a/include/ts/apidefs.h.in
+++ b/include/ts/apidefs.h.in
@@ -293,8 +293,10 @@ typedef enum {
   TS_SSL_VERIFY_SERVER_HOOK = TS_SSL_SERVER_VERIFY_HOOK,
   TS_SSL_VERIFY_CLIENT_HOOK,
   TS_SSL_SESSION_HOOK,
-  TS_SSL_LAST_HOOK = TS_SSL_SESSION_HOOK,
-  TS_HTTP_REQUEST_BUFFER_READ_COMPLETE_HOOK = 24,
+  TS_VCONN_OUTBOUND_START_HOOK,
+  TS_VCONN_OUTBOUND_CLOSE_HOOK,
+  TS_SSL_LAST_HOOK = TS_VCONN_OUTBOUND_CLOSE_HOOK,
+  TS_HTTP_REQUEST_BUFFER_READ_COMPLETE_HOOK,
   TS_HTTP_LAST_HOOK
 } TSHttpHookID;
 
@@ -462,9 +464,10 @@ typedef enum {
   TS_EVENT_INTERNAL_60202                       = 60202,
   TS_EVENT_SSL_CERT                             = 60203,
   TS_EVENT_SSL_SERVERNAME                       = 60204,
-  TS_EVENT_SSL_SERVER_VERIFY_HOOK               = 60205,
   TS_EVENT_SSL_VERIFY_SERVER                    = 60205,
-  TS_EVENT_SSL_VERIFY_CLIENT                    = 60206
+  TS_EVENT_SSL_VERIFY_CLIENT                    = 60206,
+  TS_EVENT_VCONN_OUTBOUND_START                 = 60207,
+  TS_EVENT_VCONN_OUTBOUND_CLOSE                 = 60208
 } TSEvent;
 #define TS_EVENT_HTTP_READ_REQUEST_PRE_REMAP TS_EVENT_HTTP_PRE_REMAP /* backwards compat */
 
diff --git a/include/ts/ts.h b/include/ts/ts.h
index 3db5637..4fbe2d1 100644
--- a/include/ts/ts.h
+++ b/include/ts/ts.h
@@ -1742,6 +1742,16 @@ tsapi TSAction TSNetConnect(
   struct sockaddr const *to /**< Address to which to connect. */
 );
 
+/**
+ * Retrieves the continuation associated with creating the TSVConn
+ */
+tsapi TSCont TSNetInvokingContGet(TSVConn conn);
+
+/**
+ * Retrieves the transaction associated with creating the TSVConn
+ */
+tsapi TSHttpTxn TSNetInvokingTxnGet(TSVConn conn);
+
 tsapi TSAction TSNetAccept(TSCont contp, int port, int domain, int accept_threads);
 
 /**
diff --git a/iocore/net/P_SSLNetVConnection.h b/iocore/net/P_SSLNetVConnection.h
index 66505fd..7afc515 100644
--- a/iocore/net/P_SSLNetVConnection.h
+++ b/iocore/net/P_SSLNetVConnection.h
@@ -261,6 +261,15 @@ public:
       }
       break;
 
+    case HANDSHAKE_HOOKS_OUTBOUND_PRE:
+    case HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE:
+      if (eventId == TS_EVENT_VCONN_OUTBOUND_START) {
+        if (curHook) {
+          retval = true;
+        }
+      }
+      break;
+
     case HANDSHAKE_HOOKS_DONE:
       retval = true;
       break;
@@ -343,6 +352,8 @@ private:
     HANDSHAKE_HOOKS_CERT_INVOKE,
     HANDSHAKE_HOOKS_CLIENT_CERT,
     HANDSHAKE_HOOKS_CLIENT_CERT_INVOKE,
+    HANDSHAKE_HOOKS_OUTBOUND_PRE,
+    HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE,
     HANDSHAKE_HOOKS_DONE
   } sslHandshakeHookState = HANDSHAKE_HOOKS_PRE;
 
diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
index d801550..5687d48 100644
--- a/iocore/net/SSLNetVConnection.cc
+++ b/iocore/net/SSLNetVConnection.cc
@@ -878,35 +878,42 @@ SSLNetVConnection::SSLNetVConnection() {}
 void
 SSLNetVConnection::do_io_close(int lerrno)
 {
-  if (this->ssl != nullptr && sslHandShakeComplete) {
-    callHooks(TS_EVENT_VCONN_CLOSE);
-    int shutdown_mode = SSL_get_shutdown(ssl);
-    Debug("ssl-shutdown", "previous shutdown state 0x%x", shutdown_mode);
-    int new_shutdown_mode = shutdown_mode | SSL_RECEIVED_SHUTDOWN;
-
-    if (new_shutdown_mode != shutdown_mode) {
-      // We do not need to sit around and wait for the client's close-notify if
-      // they have not already sent it.  We will still be standards compliant
-      Debug("ssl-shutdown", "new SSL_set_shutdown 0x%x", new_shutdown_mode);
-      SSL_set_shutdown(ssl, new_shutdown_mode);
+  if (this->ssl != nullptr) {
+    if (get_context() == NET_VCONNECTION_OUT) {
+      callHooks(TS_EVENT_VCONN_OUTBOUND_CLOSE);
+    } else {
+      callHooks(TS_EVENT_VCONN_CLOSE);
     }
 
-    // If the peer has already sent a FIN, don't bother with the shutdown
-    // They will just send us a RST for our troubles
-    // This test is not foolproof.  The client's fin could be on the wire
-    // at the same time we send the close-notify.  If so, the client will likely
-    // send RST anyway
-    char c;
-    ssize_t x = recv(this->con.fd, &c, 1, MSG_PEEK);
-    // x < 0 means error.  x == 0 means fin sent
-    bool do_shutdown = (x > 0);
-    if (x < 0) {
-      do_shutdown = (errno == EAGAIN || errno == EWOULDBLOCK);
-    }
-    if (do_shutdown) {
-      // Send the close-notify
-      int ret = SSL_shutdown(ssl);
-      Debug("ssl-shutdown", "SSL_shutdown %s", (ret) ? "success" : "failed");
+    if (sslHandShakeComplete) {
+      int shutdown_mode = SSL_get_shutdown(ssl);
+      Debug("ssl-shutdown", "previous shutdown state 0x%x", shutdown_mode);
+      int new_shutdown_mode = shutdown_mode | SSL_RECEIVED_SHUTDOWN;
+
+      if (new_shutdown_mode != shutdown_mode) {
+        // We do not need to sit around and wait for the client's close-notify if
+        // they have not already sent it.  We will still be standards compliant
+        Debug("ssl-shutdown", "new SSL_set_shutdown 0x%x", new_shutdown_mode);
+        SSL_set_shutdown(ssl, new_shutdown_mode);
+      }
+
+      // If the peer has already sent a FIN, don't bother with the shutdown
+      // They will just send us a RST for our troubles
+      // This test is not foolproof.  The client's fin could be on the wire
+      // at the same time we send the close-notify.  If so, the client will likely
+      // send RST anyway
+      char c;
+      ssize_t x = recv(this->con.fd, &c, 1, MSG_PEEK);
+      // x < 0 means error.  x == 0 means fin sent
+      bool do_shutdown = (x > 0);
+      if (x < 0) {
+        do_shutdown = (errno == EAGAIN || errno == EWOULDBLOCK);
+      }
+      if (do_shutdown) {
+        // Send the close-notify
+        int ret = SSL_shutdown(ssl);
+        Debug("ssl-shutdown", "SSL_shutdown %s", (ret) ? "success" : "failed");
+      }
     }
   }
   // Go on and do the unix socket cleanups
@@ -1366,6 +1373,33 @@ SSLNetVConnection::sslClientHandShakeEvent(int &err)
 
   ink_assert(SSLNetVCAccess(ssl) == this);
 
+  // Initialize properly for a client connection
+  if (sslHandshakeHookState == HANDSHAKE_HOOKS_PRE) {
+    sslHandshakeHookState = HANDSHAKE_HOOKS_OUTBOUND_PRE;
+  }
+
+  // Do outbound hook processing here
+  // Continue on if we are in the invoked state.  The hook has not yet reenabled
+  if (sslHandshakeHookState == HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE) {
+    return SSL_WAIT_FOR_HOOK;
+  }
+
+  // Go do the preaccept hooks
+  if (sslHandshakeHookState == HANDSHAKE_HOOKS_OUTBOUND_PRE) {
+    if (!curHook) {
+      Debug("ssl", "Initialize outbound connect curHook from NULL");
+      curHook = ssl_hooks->get(TS_VCONN_OUTBOUND_START_INTERNAL_HOOK);
+    } else {
+      curHook = curHook->next();
+    }
+    // If no more hooks, carry on
+    if (nullptr != curHook) {
+      sslHandshakeHookState = HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE;
+      ContWrapper::wrap(nh->mutex.get(), curHook->m_cont, TS_EVENT_VCONN_OUTBOUND_START, this);
+      return SSL_WAIT_FOR_HOOK;
+    }
+  }
+
   ssl_error = SSLConnect(ssl);
   switch (ssl_error) {
   case SSL_ERROR_NONE:
@@ -1521,6 +1555,9 @@ SSLNetVConnection::reenable(NetHandler *nh)
   case HANDSHAKE_HOOKS_PRE_INVOKE:
     sslHandshakeHookState = HANDSHAKE_HOOKS_PRE;
     break;
+  case HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE:
+    sslHandshakeHookState = HANDSHAKE_HOOKS_OUTBOUND_PRE;
+    break;
   case HANDSHAKE_HOOKS_CERT_INVOKE:
     sslHandshakeHookState = HANDSHAKE_HOOKS_CERT;
     break;
@@ -1553,6 +1590,16 @@ SSLNetVConnection::reenable(NetHandler *nh)
       Debug("ssl", "Reenable preaccept");
       sslHandshakeHookState = HANDSHAKE_HOOKS_PRE_INVOKE;
       ContWrapper::wrap(nh->mutex.get(), curHook->m_cont, TS_EVENT_VCONN_START, this);
+    } else if (sslHandshakeHookState == HANDSHAKE_HOOKS_OUTBOUND_PRE) {
+      Debug("ssl", "Reenable outbound connect");
+      sslHandshakeHookState = HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE;
+      ContWrapper::wrap(nh->mutex.get(), curHook->m_cont, TS_EVENT_VCONN_OUTBOUND_START, this);
+    } else if (sslHandshakeHookState == HANDSHAKE_HOOKS_DONE) {
+      if (this->get_context() == NET_VCONNECTION_OUT) {
+        ContWrapper::wrap(nh->mutex.get(), curHook->m_cont, TS_EVENT_VCONN_OUTBOUND_CLOSE, this);
+      } else {
+        ContWrapper::wrap(nh->mutex.get(), curHook->m_cont, TS_EVENT_VCONN_CLOSE, this);
+      }
     }
     return;
   } else {
@@ -1569,6 +1616,13 @@ SSLNetVConnection::reenable(NetHandler *nh)
     case HANDSHAKE_HOOKS_CERT_INVOKE:
       sslHandshakeHookState = HANDSHAKE_HOOKS_CLIENT_CERT;
       break;
+    case HANDSHAKE_HOOKS_OUTBOUND_PRE:
+    case HANDSHAKE_HOOKS_OUTBOUND_PRE_INVOKE:
+      this->write.triggered = true;
+      this->write.enabled   = true;
+      this->writeReschedule(nh);
+      sslHandshakeHookState = HANDSHAKE_HOOKS_DONE;
+      break;
     case HANDSHAKE_HOOKS_CLIENT_CERT:
     case HANDSHAKE_HOOKS_CLIENT_CERT_INVOKE:
       sslHandshakeHookState = HANDSHAKE_HOOKS_DONE;
@@ -1600,7 +1654,7 @@ SSLNetVConnection::callHooks(TSEvent eventId)
 {
   // Only dealing with the SNI/CERT hook so far.
   ink_assert(eventId == TS_EVENT_SSL_CERT || eventId == TS_EVENT_SSL_SERVERNAME || eventId == TS_EVENT_SSL_VERIFY_SERVER ||
-             eventId == TS_EVENT_SSL_VERIFY_CLIENT || eventId == TS_EVENT_VCONN_CLOSE);
+             eventId == TS_EVENT_SSL_VERIFY_CLIENT || eventId == TS_EVENT_VCONN_CLOSE || eventId == TS_EVENT_VCONN_OUTBOUND_CLOSE);
   Debug("ssl", "callHooks sslHandshakeHookState=%d", this->sslHandshakeHookState);
 
   // Move state if it is appropriate
@@ -1663,8 +1717,21 @@ SSLNetVConnection::callHooks(TSEvent eventId)
     }
   // fallthrough
   case HANDSHAKE_HOOKS_DONE:
+  case HANDSHAKE_HOOKS_OUTBOUND_PRE:
     if (eventId == TS_EVENT_VCONN_CLOSE) {
-      curHook = ssl_hooks->get(TS_VCONN_CLOSE_INTERNAL_HOOK);
+      sslHandshakeHookState = HANDSHAKE_HOOKS_DONE;
+      if (curHook == nullptr) {
+        curHook = ssl_hooks->get(TS_VCONN_CLOSE_INTERNAL_HOOK);
+      } else {
+        curHook = curHook->next();
+      }
+    } else if (eventId == TS_EVENT_VCONN_OUTBOUND_CLOSE) {
+      sslHandshakeHookState = HANDSHAKE_HOOKS_DONE;
+      if (curHook == nullptr) {
+        curHook = ssl_hooks->get(TS_VCONN_OUTBOUND_CLOSE_INTERNAL_HOOK);
+      } else {
+        curHook = curHook->next();
+      }
     }
     break;
   default:
@@ -1675,13 +1742,15 @@ SSLNetVConnection::callHooks(TSEvent eventId)
 
   Debug("ssl", "callHooks iterated to curHook=%p", curHook);
 
+  bool reenabled = true;
+
   this->serverName = const_cast<char *>(SSL_get_servername(this->ssl, TLSEXT_NAMETYPE_host_name));
   if (this->serverName) {
     auto *hs = TunnelMap.find(this->serverName);
     if (hs != nullptr) {
       this->SNIMapping = true;
       this->attributes = HttpProxyPort::TRANSPORT_BLIND_TUNNEL;
-      return EVENT_DONE;
+      return reenabled;
     }
   }
 
@@ -1692,10 +1761,9 @@ SSLNetVConnection::callHooks(TSEvent eventId)
     // we get out of this callback, and then will shuffle
     // over the buffered handshake packets to the O.S.
     // sslHandShakeComplete = 1;
-    return EVENT_DONE;
+    return reenabled;
   }
 
-  bool reenabled = true;
   if (curHook != nullptr) {
     curHook->invoke(eventId, this);
     reenabled =
diff --git a/proxy/InkAPIInternal.h b/proxy/InkAPIInternal.h
index 0ed5672..9c24fe8 100644
--- a/proxy/InkAPIInternal.h
+++ b/proxy/InkAPIInternal.h
@@ -282,6 +282,8 @@ typedef enum {
   TS_SSL_VERIFY_SERVER_INTERNAL_HOOK,
   TS_SSL_VERIFY_CLIENT_INTERNAL_HOOK,
   TS_SSL_SESSION_INTERNAL_HOOK,
+  TS_VCONN_OUTBOUND_START_INTERNAL_HOOK,
+  TS_VCONN_OUTBOUND_CLOSE_INTERNAL_HOOK,
   TS_SSL_INTERNAL_LAST_HOOK
 } TSSslHookInternalID;
 
diff --git a/proxy/http/HttpDebugNames.cc b/proxy/http/HttpDebugNames.cc
index e460f82..7492a3a 100644
--- a/proxy/http/HttpDebugNames.cc
+++ b/proxy/http/HttpDebugNames.cc
@@ -470,6 +470,10 @@ HttpDebugNames::get_api_hook_name(TSHttpHookID t)
     return "TS_SSL_VERIFY_CLIENT_HOOK";
   case TS_SSL_SESSION_HOOK:
     return "TS_SSL_SESSION_HOOK";
+  case TS_VCONN_OUTBOUND_START_HOOK:
+    return "TS_VCONN_OUTBOUND_START_HOOK";
+  case TS_VCONN_OUTBOUND_CLOSE_HOOK:
+    return "TS_VCONN_OUTBOUND_CLOSE_HOOK";
   }
 
   return "unknown hook";
diff --git a/src/traffic_server/InkAPI.cc b/src/traffic_server/InkAPI.cc
index 372b855..d055317 100644
--- a/src/traffic_server/InkAPI.cc
+++ b/src/traffic_server/InkAPI.cc
@@ -6806,6 +6806,34 @@ TSNetConnectTransparent(TSCont contp, sockaddr const *client_addr, sockaddr cons
   return reinterpret_cast<TSAction>(netProcessor.connect_re(reinterpret_cast<INKContInternal *>(contp), server_addr, &opt));
 }
 
+TSCont
+TSNetInvokingContGet(TSVConn conn)
+{
+  NetVConnection *vc         = reinterpret_cast<NetVConnection *>(conn);
+  UnixNetVConnection *net_vc = dynamic_cast<UnixNetVConnection *>(vc);
+  TSCont ret                 = nullptr;
+  if (net_vc) {
+    const Action *action = net_vc->get_action();
+    ret                  = reinterpret_cast<TSCont>(action->continuation);
+  }
+  return ret;
+}
+
+TSHttpTxn
+TSNetInvokingTxnGet(TSVConn conn)
+{
+  TSCont cont   = TSNetInvokingContGet(conn);
+  TSHttpTxn ret = nullptr;
+  if (cont) {
+    Continuation *contobj = reinterpret_cast<Continuation *>(cont);
+    HttpSM *sm            = dynamic_cast<HttpSM *>(contobj);
+    if (sm) {
+      ret = reinterpret_cast<TSHttpTxn>(sm);
+    }
+  }
+  return ret;
+}
+
 TSAction
 TSNetAccept(TSCont contp, int port, int domain, int accept_threads)
 {
diff --git a/src/traffic_server/InkAPITest.cc b/src/traffic_server/InkAPITest.cc
index 2769d49..f17e761 100644
--- a/src/traffic_server/InkAPITest.cc
+++ b/src/traffic_server/InkAPITest.cc
@@ -6625,8 +6625,10 @@ typedef enum {
   ORIG_TS_SSL_VERIFY_SERVER_HOOK,
   ORIG_TS_SSL_VERIFY_CLIENT_HOOK,
   ORIG_TS_SSL_SESSION_HOOK,
-  ORIG_TS_SSL_LAST_HOOK                          = ORIG_TS_SSL_SESSION_HOOK,
-  ORIG_TS_HTTP_REQUEST_BUFFER_READ_COMPLETE_HOOK = 24,
+  ORIG_TS_VCONN_OUTBOUND_START_HOOK,
+  ORIG_TS_VCONN_OUTBOUND_CLOSE_HOOK,
+  ORIG_TS_SSL_LAST_HOOK = ORIG_TS_VCONN_OUTBOUND_CLOSE_HOOK,
+  ORIG_TS_HTTP_REQUEST_BUFFER_READ_COMPLETE_HOOK,
   ORIG_TS_HTTP_LAST_HOOK
 } ORIG_TSHttpHookID;
 
diff --git a/tests/gold_tests/tls_hooks/gold/ts-close-out-close.gold b/tests/gold_tests/tls_hooks/gold/ts-close-out-close.gold
new file mode 100644
index 0000000..c03962e
--- /dev/null
+++ b/tests/gold_tests/tls_hooks/gold/ts-close-out-close.gold
@@ -0,0 +1,3 @@
+`` DIAG: (ssl_hook_test) Outbound close callback `` - event is good
+`` DIAG: (ssl_hook_test) Close callback 0 `` - event is good
+`` DIAG: (ssl_hook_test) Close callback 1 `` - event is good
diff --git a/tests/gold_tests/tls_hooks/gold/ts-out-delay-start-2.gold b/tests/gold_tests/tls_hooks/gold/ts-out-delay-start-2.gold
new file mode 100644
index 0000000..21c5d61
--- /dev/null
+++ b/tests/gold_tests/tls_hooks/gold/ts-out-delay-start-2.gold
@@ -0,0 +1,5 @@
+`` DIAG: (ssl_hook_test) Setup callbacks pa=0 sni=0 cert=0 cert_imm=0 pa_delay=0
+`` DIAG: (ssl_hook_test) Outbound delay start callback 0 ``
+`` DIAG: (ssl_hook_test) Callback reenable ssl_vc=``
+`` DIAG: (ssl_hook_test) Outbound delay start callback 1 ``
+`` DIAG: (ssl_hook_test) Callback reenable ssl_vc=``
diff --git a/tests/gold_tests/tls_hooks/gold/ts-out-start-close-2.gold b/tests/gold_tests/tls_hooks/gold/ts-out-start-close-2.gold
new file mode 100644
index 0000000..d13e43d
--- /dev/null
+++ b/tests/gold_tests/tls_hooks/gold/ts-out-start-close-2.gold
@@ -0,0 +1,4 @@
+`` DIAG: (ssl_hook_test) Outbound start callback 0 ``
+`` DIAG: (ssl_hook_test) Outbound close callback 0 ``
+`` DIAG: (ssl_hook_test) Outbound close callback 1 ``
+
diff --git a/tests/gold_tests/tls_hooks/tls_hooks10.test.py b/tests/gold_tests/tls_hooks/tls_hooks10.test.py
index 59cccf5..b4aa252 100644
--- a/tests/gold_tests/tls_hooks/tls_hooks10.test.py
+++ b/tests/gold_tests/tls_hooks/tls_hooks10.test.py
@@ -68,15 +68,8 @@ tr.StillRunningAfter = ts
 tr.StillRunningAfter = server
 tr.Processes.Default.Command = 'curl -k -H \'host:example.com:{0}\' https://127.0.0.1:{0}'.format(ts.Variables.ssl_port)
 tr.Processes.Default.ReturnCode = 0
-tr.Processes.Default.Streams.stdout = "gold/preaccept-1.gold"
 
 ts.Streams.stderr = "gold/ts-cert-1-im-2.gold"
 
-certstring0 = "Cert callback 0"
-certstring1 = "Cert callback 1"
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}.*{0}(?!.*{0}).*\Z".format(certstring0), "Cert message appears twicd", reflags=re.S | re.M)
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}(?!.*{0}).*\Z".format(certstring1), "Cert message appears only once", reflags=re.S | re.M)
 tr.Processes.Default.TimeOut = 5
 tr.TimeOut = 5
diff --git a/tests/gold_tests/tls_hooks/tls_hooks12.test.py b/tests/gold_tests/tls_hooks/tls_hooks12.test.py
index 252f92d..ea66c33 100644
--- a/tests/gold_tests/tls_hooks/tls_hooks12.test.py
+++ b/tests/gold_tests/tls_hooks/tls_hooks12.test.py
@@ -65,23 +65,8 @@ tr.StillRunningAfter = ts
 tr.StillRunningAfter = server
 tr.Processes.Default.Command = 'curl -k -H \'host:example.com:{0}\' https://127.0.0.1:{0}'.format(ts.Variables.ssl_port)
 tr.Processes.Default.ReturnCode = 0
-tr.Processes.Default.Streams.stdout = "gold/preaccept-1.gold"
 
 ts.Streams.stderr = "gold/ts-preaccept-delayed-1-immdate-2.gold"
 
-# Not going to check for number of times the message appears.  With the current test framework
-# a probing TCP connection is made to test that the port is listening.  The entire preaccept hook
-# sequence may appear on that probe.  Or it may not.  If we move away from the probe connection
-# we can check for the right number of each message.
-#preacceptstring0 = "Pre accept delay callback 0"
-# ts.Streams.All = Testers.ContainsExpression(
-#    "\A(?:(?!{0}).)*{0}.*({0})?(?!.*{0}).*\Z".format(preacceptstring0), "Pre accept message appears only once or twice", reflags=re.S | re.M)
-#preacceptstring1 = "Pre accept callback 0"
-# ts.Streams.All = Testers.ContainsExpression(
-#    "\A(?:(?!{0}).)*{0}.*({0})?(?!.*{0}).*\Z".format(preacceptstring1), "Pre accept message appears only once or twice", reflags=re.S | re.M)
-#preacceptstring2 = "Pre accept callback 1"
-# ts.Streams.All = Testers.ContainsExpression(
-#    "\A(?:(?!{0}).)*{0}.*({0})?(?!.*{0}).*\Z".format(preacceptstring2), "Pre accept message appears only once or twice", reflags=re.S | re.M)
-
 tr.Processes.Default.TimeOut = 5
 tr.TimeOut = 5
diff --git a/tests/gold_tests/tls_hooks/tls_hooks10.test.py b/tests/gold_tests/tls_hooks/tls_hooks13.test.py
similarity index 74%
copy from tests/gold_tests/tls_hooks/tls_hooks10.test.py
copy to tests/gold_tests/tls_hooks/tls_hooks13.test.py
index 59cccf5..9388221 100644
--- a/tests/gold_tests/tls_hooks/tls_hooks10.test.py
+++ b/tests/gold_tests/tls_hooks/tls_hooks13.test.py
@@ -1,5 +1,5 @@
 '''
-Test one delayed cert callback and two immediate cert callbacks
+Test one outbound start and two outbound close
 '''
 #  Licensed to the Apache Software Foundation (ASF) under one
 #  or more contributor license agreements.  See the NOTICE file
@@ -24,13 +24,10 @@ Test.Summary = '''
 Test different combinations of TLS handshake hooks to ensure they are applied consistently.
 '''
 
-Test.SkipUnless(
-    Condition.HasProgram("grep", "grep needs to be installed on system for this test to work"),
-    Condition.HasOpenSSLVersion("1.0.2")
-    )
+Test.SkipUnless(Condition.HasProgram("grep", "grep needs to be installed on system for this test to work"))
 
 ts = Test.MakeATSProcess("ts", select_ports=False)
-server = Test.MakeOriginServer("server")
+server = Test.MakeOriginServer("server", ssl=True)
 request_header = {"headers": "GET / HTTP/1.1\r\nHost: www.example.com\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
 # desired response form the origin server
 response_header = {"headers": "HTTP/1.1 200 OK\r\nConnection: close\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
@@ -56,27 +53,20 @@ ts.Disk.ssl_multicert_config.AddLine(
 )
 
 ts.Disk.remap_config.AddLine(
-    'map https://example.com:4443 http://127.0.0.1:{0}'.format(server.Variables.Port)
+    'map https://example.com:4443 https://127.0.0.1:{0}'.format(server.Variables.Port)
 )
 
-Test.PreparePlugin(os.path.join(Test.Variables.AtsTestToolsDir, 'plugins', 'ssl_hook_test.cc'), ts, '-cert=1 -i=2')
+Test.PreparePlugin(os.path.join(Test.Variables.AtsTestToolsDir, 'plugins', 'ssl_hook_test.cc'), ts, '-out_start=1 -out_close=2')
 
-tr = Test.AddTestRun("Test a combination of delayed and immediate cert hooks")
+tr = Test.AddTestRun("Test outbound start and close")
 tr.Processes.Default.StartBefore(server)
 tr.Processes.Default.StartBefore(Test.Processes.ts, ready=When.PortOpen(ts.Variables.ssl_port))
 tr.StillRunningAfter = ts
 tr.StillRunningAfter = server
 tr.Processes.Default.Command = 'curl -k -H \'host:example.com:{0}\' https://127.0.0.1:{0}'.format(ts.Variables.ssl_port)
 tr.Processes.Default.ReturnCode = 0
-tr.Processes.Default.Streams.stdout = "gold/preaccept-1.gold"
 
-ts.Streams.stderr = "gold/ts-cert-1-im-2.gold"
+ts.Streams.stderr = "gold/ts-out-start-close-2.gold"
 
-certstring0 = "Cert callback 0"
-certstring1 = "Cert callback 1"
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}.*{0}(?!.*{0}).*\Z".format(certstring0), "Cert message appears twicd", reflags=re.S | re.M)
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}(?!.*{0}).*\Z".format(certstring1), "Cert message appears only once", reflags=re.S | re.M)
 tr.Processes.Default.TimeOut = 5
 tr.TimeOut = 5
diff --git a/tests/gold_tests/tls_hooks/tls_hooks10.test.py b/tests/gold_tests/tls_hooks/tls_hooks14.test.py
similarity index 74%
copy from tests/gold_tests/tls_hooks/tls_hooks10.test.py
copy to tests/gold_tests/tls_hooks/tls_hooks14.test.py
index 59cccf5..b411f65 100644
--- a/tests/gold_tests/tls_hooks/tls_hooks10.test.py
+++ b/tests/gold_tests/tls_hooks/tls_hooks14.test.py
@@ -1,5 +1,5 @@
 '''
-Test one delayed cert callback and two immediate cert callbacks
+Test two outbound start delayed hooks
 '''
 #  Licensed to the Apache Software Foundation (ASF) under one
 #  or more contributor license agreements.  See the NOTICE file
@@ -24,13 +24,10 @@ Test.Summary = '''
 Test different combinations of TLS handshake hooks to ensure they are applied consistently.
 '''
 
-Test.SkipUnless(
-    Condition.HasProgram("grep", "grep needs to be installed on system for this test to work"),
-    Condition.HasOpenSSLVersion("1.0.2")
-    )
+Test.SkipUnless(Condition.HasProgram("grep", "grep needs to be installed on system for this test to work"))
 
 ts = Test.MakeATSProcess("ts", select_ports=False)
-server = Test.MakeOriginServer("server")
+server = Test.MakeOriginServer("server", ssl=True)
 request_header = {"headers": "GET / HTTP/1.1\r\nHost: www.example.com\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
 # desired response form the origin server
 response_header = {"headers": "HTTP/1.1 200 OK\r\nConnection: close\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
@@ -56,27 +53,20 @@ ts.Disk.ssl_multicert_config.AddLine(
 )
 
 ts.Disk.remap_config.AddLine(
-    'map https://example.com:4443 http://127.0.0.1:{0}'.format(server.Variables.Port)
+    'map https://example.com:4443 https://127.0.0.1:{0}'.format(server.Variables.Port)
 )
 
-Test.PreparePlugin(os.path.join(Test.Variables.AtsTestToolsDir, 'plugins', 'ssl_hook_test.cc'), ts, '-cert=1 -i=2')
-
-tr = Test.AddTestRun("Test a combination of delayed and immediate cert hooks")
+Test.PreparePlugin(os.path.join(Test.Variables.AtsTestToolsDir, 'plugins', 'ssl_hook_test.cc'), ts, '-out_start_delay=2')
+ 
+tr = Test.AddTestRun("Test outbound delay start")
 tr.Processes.Default.StartBefore(server)
 tr.Processes.Default.StartBefore(Test.Processes.ts, ready=When.PortOpen(ts.Variables.ssl_port))
 tr.StillRunningAfter = ts
 tr.StillRunningAfter = server
 tr.Processes.Default.Command = 'curl -k -H \'host:example.com:{0}\' https://127.0.0.1:{0}'.format(ts.Variables.ssl_port)
 tr.Processes.Default.ReturnCode = 0
-tr.Processes.Default.Streams.stdout = "gold/preaccept-1.gold"
 
-ts.Streams.stderr = "gold/ts-cert-1-im-2.gold"
+ts.Streams.stderr = "gold/ts-out-delay-start-2.gold"
 
-certstring0 = "Cert callback 0"
-certstring1 = "Cert callback 1"
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}.*{0}(?!.*{0}).*\Z".format(certstring0), "Cert message appears twicd", reflags=re.S | re.M)
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}(?!.*{0}).*\Z".format(certstring1), "Cert message appears only once", reflags=re.S | re.M)
 tr.Processes.Default.TimeOut = 5
 tr.TimeOut = 5
diff --git a/tests/gold_tests/tls_hooks/tls_hooks10.test.py b/tests/gold_tests/tls_hooks/tls_hooks15.test.py
similarity index 74%
copy from tests/gold_tests/tls_hooks/tls_hooks10.test.py
copy to tests/gold_tests/tls_hooks/tls_hooks15.test.py
index 59cccf5..ed6e687 100644
--- a/tests/gold_tests/tls_hooks/tls_hooks10.test.py
+++ b/tests/gold_tests/tls_hooks/tls_hooks15.test.py
@@ -1,5 +1,5 @@
 '''
-Test one delayed cert callback and two immediate cert callbacks
+Test one delayed preaccept callback
 '''
 #  Licensed to the Apache Software Foundation (ASF) under one
 #  or more contributor license agreements.  See the NOTICE file
@@ -24,13 +24,10 @@ Test.Summary = '''
 Test different combinations of TLS handshake hooks to ensure they are applied consistently.
 '''
 
-Test.SkipUnless(
-    Condition.HasProgram("grep", "grep needs to be installed on system for this test to work"),
-    Condition.HasOpenSSLVersion("1.0.2")
-    )
+Test.SkipUnless(Condition.HasProgram("grep", "grep needs to be installed on system for this test to work"))
 
 ts = Test.MakeATSProcess("ts", select_ports=False)
-server = Test.MakeOriginServer("server")
+server = Test.MakeOriginServer("server", ssl=True)
 request_header = {"headers": "GET / HTTP/1.1\r\nHost: www.example.com\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
 # desired response form the origin server
 response_header = {"headers": "HTTP/1.1 200 OK\r\nConnection: close\r\n\r\n", "timestamp": "1469733493.993", "body": ""}
@@ -56,27 +53,20 @@ ts.Disk.ssl_multicert_config.AddLine(
 )
 
 ts.Disk.remap_config.AddLine(
-    'map https://example.com:4443 http://127.0.0.1:{0}'.format(server.Variables.Port)
+    'map https://example.com:4443 https://127.0.0.1:{0}'.format(server.Variables.Port)
 )
 
-Test.PreparePlugin(os.path.join(Test.Variables.AtsTestToolsDir, 'plugins', 'ssl_hook_test.cc'), ts, '-cert=1 -i=2')
+Test.PreparePlugin(os.path.join(Test.Variables.AtsTestToolsDir, 'plugins', 'ssl_hook_test.cc'), ts, '-close=2 -out_close=1')
 
-tr = Test.AddTestRun("Test a combination of delayed and immediate cert hooks")
+tr = Test.AddTestRun("Test one delayed preaccept hook")
 tr.Processes.Default.StartBefore(server)
 tr.Processes.Default.StartBefore(Test.Processes.ts, ready=When.PortOpen(ts.Variables.ssl_port))
 tr.StillRunningAfter = ts
 tr.StillRunningAfter = server
 tr.Processes.Default.Command = 'curl -k -H \'host:example.com:{0}\' https://127.0.0.1:{0}'.format(ts.Variables.ssl_port)
 tr.Processes.Default.ReturnCode = 0
-tr.Processes.Default.Streams.stdout = "gold/preaccept-1.gold"
 
-ts.Streams.stderr = "gold/ts-cert-1-im-2.gold"
+ts.Streams.stderr = "gold/ts-close-out-close.gold"
 
-certstring0 = "Cert callback 0"
-certstring1 = "Cert callback 1"
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}.*{0}(?!.*{0}).*\Z".format(certstring0), "Cert message appears twicd", reflags=re.S | re.M)
-ts.Streams.All = Testers.ContainsExpression(
-    "\A(?:(?!{0}).)*{0}(?!.*{0}).*\Z".format(certstring1), "Cert message appears only once", reflags=re.S | re.M)
 tr.Processes.Default.TimeOut = 5
 tr.TimeOut = 5
diff --git a/tests/tools/plugins/ssl_hook_test.cc b/tests/tools/plugins/ssl_hook_test.cc
index 7321b9c..f7fde1c 100644
--- a/tests/tools/plugins/ssl_hook_test.cc
+++ b/tests/tools/plugins/ssl_hook_test.cc
@@ -30,6 +30,7 @@
 #include <getopt.h>
 #include <openssl/ssl.h>
 #include <strings.h>
+#include <cstring>
 
 #define PN "ssl_hook_test"
 #define PCP "[" PN " Plugin] "
@@ -78,6 +79,70 @@ CB_Pre_Accept_Delay(TSCont cont, TSEvent event, void *edata)
 }
 
 int
+CB_out_start(TSCont cont, TSEvent event, void *edata)
+{
+  TSVConn ssl_vc = reinterpret_cast<TSVConn>(edata);
+
+  int count = reinterpret_cast<intptr_t>(TSContDataGet(cont));
+
+  TSDebug(PN, "Outbound start callback %d %p - event is %s", count, ssl_vc,
+          event == TS_EVENT_VCONN_OUTBOUND_START ? "good" : "bad");
+
+  // All done, reactivate things
+  TSVConnReenable(ssl_vc);
+  return TS_SUCCESS;
+}
+
+int
+CB_out_start_delay(TSCont cont, TSEvent event, void *edata)
+{
+  TSVConn ssl_vc = reinterpret_cast<TSVConn>(edata);
+
+  int count = reinterpret_cast<intptr_t>(TSContDataGet(cont));
+
+  TSDebug(PN, "Outbound delay start callback %d %p - event is %s", count, ssl_vc,
+          event == TS_EVENT_VCONN_OUTBOUND_START ? "good" : "bad");
+
+  TSCont cb = TSContCreate(&ReenableSSL, TSMutexCreate());
+
+  TSContDataSet(cb, ssl_vc);
+
+  // Schedule to reenable in a bit
+  TSContSchedule(cb, 2000, TS_THREAD_POOL_NET);
+
+  return TS_SUCCESS;
+}
+
+int
+CB_close(TSCont cont, TSEvent event, void *edata)
+{
+  TSVConn ssl_vc = reinterpret_cast<TSVConn>(edata);
+
+  int count = reinterpret_cast<intptr_t>(TSContDataGet(cont));
+
+  TSDebug(PN, "Close callback %d %p - event is %s", count, ssl_vc, event == TS_EVENT_VCONN_CLOSE ? "good" : "bad");
+
+  // All done, reactivate things
+  TSVConnReenable(ssl_vc);
+  return TS_SUCCESS;
+}
+
+int
+CB_out_close(TSCont cont, TSEvent event, void *edata)
+{
+  TSVConn ssl_vc = reinterpret_cast<TSVConn>(edata);
+
+  int count = reinterpret_cast<intptr_t>(TSContDataGet(cont));
+
+  TSDebug(PN, "Outbound close callback %d %p - event is %s", count, ssl_vc,
+          event == TS_EVENT_VCONN_OUTBOUND_CLOSE ? "good" : "bad");
+
+  // All done, reactivate things
+  TSVConnReenable(ssl_vc);
+  return TS_SUCCESS;
+}
+
+int
 CB_SNI(TSCont cont, TSEvent event, void *edata)
 {
   TSVConn ssl_vc = reinterpret_cast<TSVConn>(edata);
@@ -125,7 +190,8 @@ CB_Cert(TSCont cont, TSEvent event, void *edata)
 
 void
 parse_callbacks(int argc, const char *argv[], int &preaccept_count, int &sni_count, int &cert_count, int &cert_count_immediate,
-                int &preaccept_count_delay)
+                int &preaccept_count_delay, int &close_count, int &out_start_count, int &out_start_delay_count,
+                int &out_close_count)
 {
   int i = 0;
   const char *ptr;
@@ -147,7 +213,11 @@ parse_callbacks(int argc, const char *argv[], int &preaccept_count, int &sni_cou
       case 'c':
         ptr = index(argv[i], '=');
         if (ptr) {
-          cert_count = atoi(ptr + 1);
+          if (strncmp(argv[i] + 1, "close", strlen("close")) == 0) {
+            close_count = atoi(ptr + i);
+          } else {
+            cert_count = atoi(ptr + 1);
+          }
         }
         break;
       case 'd':
@@ -162,6 +232,17 @@ parse_callbacks(int argc, const char *argv[], int &preaccept_count, int &sni_cou
           cert_count_immediate = atoi(ptr + 1);
         }
         break;
+      case 'o':
+        ptr = index(argv[i], '=');
+        if (ptr) {
+          if (strncmp(argv[i] + 1, "out_start_delay", strlen("out_start_delay")) == 0) {
+            out_start_delay_count = atoi(ptr + 1);
+          } else if (strncmp(argv[i] + 1, "out_start", strlen("out_start")) == 0) {
+            out_start_count = atoi(ptr + 1);
+          } else if (strncmp(argv[i] + 1, "out_close", strlen("out_close")) == 0) {
+            out_close_count = atoi(ptr + 1);
+          }
+        }
       }
     }
   }
@@ -169,7 +250,7 @@ parse_callbacks(int argc, const char *argv[], int &preaccept_count, int &sni_cou
 
 void
 setup_callbacks(TSHttpTxn txn, int preaccept_count, int sni_count, int cert_count, int cert_count_immediate,
-                int preaccept_count_delay)
+                int preaccept_count_delay, int close_count, int out_start_count, int out_start_delay_count, int out_close_count)
 {
   TSCont cb = nullptr; // pre-accept callback continuation
   int i;
@@ -222,6 +303,43 @@ setup_callbacks(TSHttpTxn txn, int preaccept_count, int sni_count, int cert_coun
     }
   }
 
+  for (i = 0; i < close_count; i++) {
+    cb = TSContCreate(&CB_close, TSMutexCreate());
+    TSContDataSet(cb, (void *)(intptr_t)i);
+    if (txn) {
+      TSHttpTxnHookAdd(txn, TS_VCONN_CLOSE_HOOK, cb);
+    } else {
+      TSHttpHookAdd(TS_VCONN_CLOSE_HOOK, cb);
+    }
+  }
+  for (i = 0; i < out_start_count; i++) {
+    cb = TSContCreate(&CB_out_start, TSMutexCreate());
+    TSContDataSet(cb, (void *)(intptr_t)i);
+    if (txn) {
+      TSHttpTxnHookAdd(txn, TS_VCONN_OUTBOUND_START_HOOK, cb);
+    } else {
+      TSHttpHookAdd(TS_VCONN_OUTBOUND_START_HOOK, cb);
+    }
+  }
+  for (i = 0; i < out_start_delay_count; i++) {
+    cb = TSContCreate(&CB_out_start_delay, TSMutexCreate());
+    TSContDataSet(cb, (void *)(intptr_t)i);
+    if (txn) {
+      TSHttpTxnHookAdd(txn, TS_VCONN_OUTBOUND_START_HOOK, cb);
+    } else {
+      TSHttpHookAdd(TS_VCONN_OUTBOUND_START_HOOK, cb);
+    }
+  }
+  for (i = 0; i < out_close_count; i++) {
+    cb = TSContCreate(&CB_out_close, TSMutexCreate());
+    TSContDataSet(cb, (void *)(intptr_t)i);
+    if (txn) {
+      TSHttpTxnHookAdd(txn, TS_VCONN_OUTBOUND_CLOSE_HOOK, cb);
+    } else {
+      TSHttpHookAdd(TS_VCONN_OUTBOUND_CLOSE_HOOK, cb);
+    }
+  }
+
   return;
 }
 
@@ -242,7 +360,13 @@ TSPluginInit(int argc, const char *argv[])
   int cert_count            = 0;
   int cert_count_immediate  = 0;
   int preaccept_count_delay = 0;
-  parse_callbacks(argc, argv, preaccept_count, sni_count, cert_count, cert_count_immediate, preaccept_count_delay);
-  setup_callbacks(nullptr, preaccept_count, sni_count, cert_count, cert_count_immediate, preaccept_count_delay);
+  int close_count           = 0;
+  int out_start_count       = 0;
+  int out_start_delay_count = 0;
+  int out_close_count       = 0;
+  parse_callbacks(argc, argv, preaccept_count, sni_count, cert_count, cert_count_immediate, preaccept_count_delay, close_count,
+                  out_start_count, out_start_delay_count, out_close_count);
+  setup_callbacks(nullptr, preaccept_count, sni_count, cert_count, cert_count_immediate, preaccept_count_delay, close_count,
+                  out_start_count, out_start_delay_count, out_close_count);
   return;
 }