You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by zw...@apache.org on 2020/07/13 22:50:08 UTC

[trafficserver] branch master updated: Improve client_vc tracking (#6889)

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

zwoop 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 90f1493  Improve client_vc tracking (#6889)
90f1493 is described below

commit 90f14938e8469fc5a9dfd966a43d28e3a4b6633f
Author: Susan Hinrichs <sh...@yahoo-inc.com>
AuthorDate: Mon Jul 13 17:49:58 2020 -0500

    Improve client_vc tracking (#6889)
    
    * Improve client_vc tracking
    
    * Refine Http/2 error code
    
    Co-authored-by: Susan Hinrichs <sh...@verizonmedia.com>
---
 .../hooks-and-transactions/http-sessions.en.rst    |  5 +-
 iocore/net/UnixNetVConnection.cc                   |  2 +
 proxy/http/Http1ClientSession.cc                   | 54 +++++++---------------
 proxy/http/Http1ClientSession.h                    |  1 -
 proxy/http/HttpSM.cc                               | 17 ++++---
 proxy/http2/Http2ClientSession.cc                  |  7 +--
 proxy/http2/Http2ConnectionState.cc                |  5 +-
 proxy/http2/Http2Stream.cc                         |  4 +-
 tests/gold_tests/pluginTest/test_hooks/log.gold    |  8 ++--
 .../pluginTest/test_hooks/test_hooks.test.py       |  1 +
 10 files changed, 43 insertions(+), 61 deletions(-)

diff --git a/doc/developer-guide/plugins/hooks-and-transactions/http-sessions.en.rst b/doc/developer-guide/plugins/hooks-and-transactions/http-sessions.en.rst
index 26da265..d994989 100644
--- a/doc/developer-guide/plugins/hooks-and-transactions/http-sessions.en.rst
+++ b/doc/developer-guide/plugins/hooks-and-transactions/http-sessions.en.rst
@@ -40,7 +40,10 @@ The HTTP session hooks are:
 
 -  ``TS_HTTP_SSN_CLOSE_HOOK`` Called when an HTTP session ends (a
    session ends when the client connection is closed). This hook must be
-   added as a global hook.
+   added as a global hook.  The relative order of invocation between the
+   ``TS_VCONN_CLOSE_HOOK`` and ``TS_HTTP_SSN_CLOSE_HOOK`` is undefined.  In
+   most cases the ``TS_VCONN_CLOSE_HOOK`` will execute first, but that is
+   not guaranteed.
 
 Use the session hooks to get a handle to a session (an ``TSHttpSsn``
 object). If you want your plugin to be called back for each transaction
diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
index b1d32b4..a0e860b 100644
--- a/iocore/net/UnixNetVConnection.cc
+++ b/iocore/net/UnixNetVConnection.cc
@@ -88,6 +88,7 @@ read_signal_and_update(int event, UnixNetVConnection *vc)
     case VC_EVENT_ACTIVE_TIMEOUT:
     case VC_EVENT_INACTIVITY_TIMEOUT:
       Debug("inactivity_cop", "event %d: null read.vio cont, closing vc %p", event, vc);
+      Warning("read: Closing orphaned vc %p", vc);
       vc->closed = 1;
       break;
     default:
@@ -119,6 +120,7 @@ write_signal_and_update(int event, UnixNetVConnection *vc)
     case VC_EVENT_ACTIVE_TIMEOUT:
     case VC_EVENT_INACTIVITY_TIMEOUT:
       Debug("inactivity_cop", "event %d: null write.vio cont, closing vc %p", event, vc);
+      Warning("write: Closing orphaned vc %p", vc);
       vc->closed = 1;
       break;
     default:
diff --git a/proxy/http/Http1ClientSession.cc b/proxy/http/Http1ClientSession.cc
index a3500d0..9ddd708 100644
--- a/proxy/http/Http1ClientSession.cc
+++ b/proxy/http/Http1ClientSession.cc
@@ -87,6 +87,8 @@ Http1ClientSession::release_transaction()
 {
   released_transactions++;
   if (transact_count == released_transactions) {
+    // Make sure we previously called release() or do_io_close() on the session
+    ink_release_assert(read_state != HCS_ACTIVE_READER && read_state != HCS_INIT);
     destroy();
   }
 }
@@ -111,9 +113,6 @@ Http1ClientSession::free()
     conn_decrease = false;
   }
 
-  // Clean up the write VIO in case of inactivity timeout
-  this->do_io_write(nullptr, 0, nullptr);
-
   // Free the transaction resources
   this->trans.super_type::destroy();
 
@@ -228,25 +227,17 @@ Http1ClientSession::do_io_close(int alerrno)
     slave_ka_vio = nullptr;
   }
   // Completed the last transaction.  Just shutdown already
-  if (transact_count == released_transactions) {
+  // Or the do_io_close is due to a network error
+  if (transact_count == released_transactions || alerrno == HTTP_ERRNO) {
     half_close = false;
   }
 
-  // Clean up the write VIO in case of inactivity timeout
-  this->do_io_write(nullptr, 0, nullptr);
-
   if (half_close && this->trans.get_sm()) {
     read_state = HCS_HALF_CLOSED;
     SET_HANDLER(&Http1ClientSession::state_wait_for_close);
     HttpSsnDebug("[%" PRId64 "] session half close", con_id);
 
     if (_vc) {
-      // We want the client to know that that we're finished
-      //  writing.  The write shutdown accomplishes this.  Unfortunately,
-      //  the IO Core semantics don't stop us from getting events
-      //  on the write side of the connection like timeouts so we
-      //  need to zero out the write of the continuation with
-      //  the do_io_write() call (INKqa05309)
       _vc->do_io_shutdown(IO_SHUTDOWN_WRITE);
 
       ka_vio = _vc->do_io_read(this, INT64_MAX, read_buffer);
@@ -264,12 +255,16 @@ Http1ClientSession::do_io_close(int alerrno)
     _reader->consume(_reader->read_avail());
   } else {
     read_state = HCS_CLOSED;
-    SET_HANDLER(&Http1ClientSession::state_wait_for_sm_shutdown);
-    ka_vio = _vc->do_io_read(this, INT64_MAX, read_buffer);
     HttpSsnDebug("[%" PRId64 "] session closed", con_id);
     HTTP_SUM_DYN_STAT(http_transactions_per_client_con, transact_count);
     HTTP_DECREMENT_DYN_STAT(http_current_client_connections_stat);
     conn_decrease = false;
+    // Can go ahead and close the netvc now, but keeping around the session object
+    // until all the transactions are closed
+    if (_vc) {
+      _vc->do_io_close();
+      _vc = nullptr;
+    }
   }
   if (transact_count == released_transactions) {
     this->destroy();
@@ -277,16 +272,6 @@ Http1ClientSession::do_io_close(int alerrno)
 }
 
 int
-Http1ClientSession::state_wait_for_sm_shutdown(int event, void *data)
-{
-  STATE_ENTER(&Http1ClientSession::state_wait_for_sm_shutdown, event, data);
-  ink_assert(read_state == HCS_CLOSED);
-
-  // Just eat IO events until the state machine has finished
-  return 0;
-}
-
-int
 Http1ClientSession::state_wait_for_close(int event, void *data)
 {
   STATE_ENTER(&Http1ClientSession::state_wait_for_close, event, data);
@@ -305,11 +290,7 @@ Http1ClientSession::state_wait_for_close(int event, void *data)
   case VC_EVENT_ACTIVE_TIMEOUT:
   case VC_EVENT_INACTIVITY_TIMEOUT:
     half_close = false;
-    this->do_io_close();
-    if (_vc != nullptr) {
-      _vc->do_io_close();
-      _vc = nullptr;
-    }
+    this->do_io_close(EHTTP_ERROR);
     break;
   case VC_EVENT_READ_READY:
     // Drain any data read
@@ -385,11 +366,7 @@ Http1ClientSession::state_keep_alive(int event, void *data)
     break;
 
   case VC_EVENT_EOS:
-    this->do_io_close();
-    if (_vc != nullptr) {
-      _vc->do_io_close();
-      _vc = nullptr;
-    }
+    this->do_io_close(EHTTP_ERROR);
     break;
 
   case VC_EVENT_READ_COMPLETE:
@@ -401,7 +378,7 @@ Http1ClientSession::state_keep_alive(int event, void *data)
   case VC_EVENT_ACTIVE_TIMEOUT:
   case VC_EVENT_INACTIVITY_TIMEOUT:
     // Keep-alive timed out
-    this->do_io_close();
+    this->do_io_close(EHTTP_ERROR);
     break;
   }
 
@@ -414,8 +391,8 @@ Http1ClientSession::release(ProxyTransaction *trans)
 {
   ink_assert(read_state == HCS_ACTIVE_READER || read_state == HCS_INIT);
 
-  // Clean up the write VIO in case of inactivity timeout
-  this->do_io_write(nullptr, 0, nullptr);
+  // Timeout events should be delivered to the session
+  this->do_io_write(this, 0, nullptr);
 
   // Check to see there is remaining data in the
   //  buffer.  If there is, spin up a new state
@@ -487,6 +464,7 @@ Http1ClientSession::attach_server_session(Http1ServerSession *ssession, bool tra
     //  the server net conneciton from calling back a dead sm
     SET_HANDLER(&Http1ClientSession::state_keep_alive);
     slave_ka_vio = ssession->do_io_read(this, INT64_MAX, ssession->read_buffer);
+    this->do_io_write(this, 0, nullptr); // Capture the inactivity timeouts
     ink_assert(slave_ka_vio != ka_vio);
 
     // Transfer control of the write side as well
diff --git a/proxy/http/Http1ClientSession.h b/proxy/http/Http1ClientSession.h
index 80c826f..aef989d 100644
--- a/proxy/http/Http1ClientSession.h
+++ b/proxy/http/Http1ClientSession.h
@@ -87,7 +87,6 @@ private:
   int state_keep_alive(int event, void *data);
   int state_slave_keep_alive(int event, void *data);
   int state_wait_for_close(int event, void *data);
-  int state_wait_for_sm_shutdown(int event, void *data);
 
   enum C_Read_State {
     HCS_INIT,
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index c4d68d7..e96e44b 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -586,7 +586,8 @@ HttpSM::attach_client_session(ProxyTransaction *client_vc, IOBufferReader *buffe
   //  this hook maybe asynchronous, we need to disable IO on
   //  client but set the continuation to be the state machine
   //  so if we get an timeout events the sm handles them
-  ua_entry->read_vio = client_vc->do_io_read(this, 0, buffer_reader->mbuf);
+  ua_entry->read_vio  = client_vc->do_io_read(this, 0, buffer_reader->mbuf);
+  ua_entry->write_vio = client_vc->do_io_write(this, 0, nullptr);
 
   /////////////////////////
   // set up timeouts     //
@@ -2775,7 +2776,6 @@ HttpSM::tunnel_handler_post(int event, void *data)
     if (ua_entry->write_buffer) {
       free_MIOBuffer(ua_entry->write_buffer);
       ua_entry->write_buffer = nullptr;
-      ua_entry->vc->do_io_write(nullptr, 0, nullptr);
     }
     if (!p->handler_state) {
       p->handler_state = HTTP_SM_POST_UA_FAIL;
@@ -3536,9 +3536,6 @@ HttpSM::tunnel_handler_post_ua(int event, HttpTunnelProducer *p)
       // if it is active timeout case, we need to give another chance to send 408 response;
       ua_txn->set_active_timeout(HRTIME_SECONDS(t_state.txn_conf->transaction_active_timeout_in));
 
-      p->vc->do_io_write(nullptr, 0, nullptr);
-      p->vc->do_io_shutdown(IO_SHUTDOWN_READ);
-
       return 0;
     }
   // fall through
@@ -3550,8 +3547,6 @@ HttpSM::tunnel_handler_post_ua(int event, HttpTunnelProducer *p)
     //   server and close the ua
     p->handler_state = HTTP_SM_POST_UA_FAIL;
     set_ua_abort(HttpTransact::ABORTED, event);
-    p->vc->do_io_write(nullptr, 0, nullptr);
-    p->vc->do_io_shutdown(IO_SHUTDOWN_READ);
     tunnel.chain_abort_all(p);
     server_session = nullptr;
     // the in_tunnel status on both the ua & and
@@ -3562,6 +3557,11 @@ HttpSM::tunnel_handler_post_ua(int event, HttpTunnelProducer *p)
     if (p->consumer_list.head && p->consumer_list.head->vc_type == HT_TRANSFORM) {
       hsm_release_assert(post_transform_info.entry->in_tunnel == true);
     } // server side may have completed before the user agent side, so it may no longer be in tunnel
+
+    // In the error case, start to take down the client session. There should
+    // be no reuse here
+    vc_table.remove_entry(this->ua_entry);
+    ua_txn->do_io_close();
     break;
 
   case VC_EVENT_READ_COMPLETE:
@@ -3578,8 +3578,7 @@ HttpSM::tunnel_handler_post_ua(int event, HttpTunnelProducer *p)
       }
     }
 
-    // Initiate another read to watch catch aborts and
-    //   timeouts
+    // Initiate another read to catch aborts and timeouts.
     ua_entry->vc_handler = &HttpSM::state_watch_for_client_abort;
     ua_entry->read_vio   = p->vc->do_io_read(this, INT64_MAX, ua_buffer_reader->mbuf);
     break;
diff --git a/proxy/http2/Http2ClientSession.cc b/proxy/http2/Http2ClientSession.cc
index 8045883..fb2650c 100644
--- a/proxy/http2/Http2ClientSession.cc
+++ b/proxy/http2/Http2ClientSession.cc
@@ -286,7 +286,7 @@ Http2ClientSession::do_io_close(int alerrno)
   this->clear_session_active();
 
   // Clean up the write VIO in case of inactivity timeout
-  this->do_io_write(nullptr, 0, nullptr);
+  this->do_io_write(this, 0, nullptr);
 }
 
 void
@@ -349,10 +349,6 @@ Http2ClientSession::main_event_handler(int event, void *edata)
     Http2SsnDebug("Closing event %d", event);
     this->set_dying_event(event);
     this->do_io_close();
-    if (_vc != nullptr) {
-      _vc->do_io_close();
-      _vc = nullptr;
-    }
     retval = 0;
     break;
 
@@ -594,7 +590,6 @@ Http2ClientSession::state_process_frame_read(int event, VIO *vio, bool inside_fr
         if (!this->connection_state.is_state_closed()) {
           this->connection_state.send_goaway_frame(this->connection_state.get_latest_stream_id_in(), err);
           this->set_half_close_local_flag(true);
-          this->do_io_close();
         }
       }
       return 0;
diff --git a/proxy/http2/Http2ConnectionState.cc b/proxy/http2/Http2ConnectionState.cc
index 6f1f4ab..bc39c12 100644
--- a/proxy/http2/Http2ConnectionState.cc
+++ b/proxy/http2/Http2ConnectionState.cc
@@ -242,9 +242,12 @@ rcv_headers_frame(Http2ConnectionState &cstate, const Http2Frame &frame)
 
   if (cstate.is_valid_streamid(stream_id)) {
     stream = cstate.find_stream(stream_id);
-    if (stream == nullptr || !stream->has_trailing_header()) {
+    if (stream == nullptr) {
       return Http2Error(Http2ErrorClass::HTTP2_ERROR_CLASS_CONNECTION, Http2ErrorCode::HTTP2_ERROR_STREAM_CLOSED,
                         "recv headers cannot find existing stream_id");
+    } else if (!stream->has_trailing_header()) {
+      return Http2Error(Http2ErrorClass::HTTP2_ERROR_CLASS_CONNECTION, Http2ErrorCode::HTTP2_ERROR_PROTOCOL_ERROR,
+                        "recv headers cannot find existing stream_id");
     }
   } else {
     // Create new stream
diff --git a/proxy/http2/Http2Stream.cc b/proxy/http2/Http2Stream.cc
index b6ea0d0..e58bd28 100644
--- a/proxy/http2/Http2Stream.cc
+++ b/proxy/http2/Http2Stream.cc
@@ -652,7 +652,9 @@ Http2Stream::signal_read_event(int event)
 void
 Http2Stream::signal_write_event(int event)
 {
-  if (this->write_vio.cont == nullptr || this->write_vio.cont->mutex == nullptr || this->write_vio.op == VIO::NONE) {
+  // Don't signal a write event if in fact nothing was written
+  if (this->write_vio.cont == nullptr || this->write_vio.cont->mutex == nullptr || this->write_vio.op == VIO::NONE ||
+      this->write_vio.nbytes == 0) {
     return;
   }
 
diff --git a/tests/gold_tests/pluginTest/test_hooks/log.gold b/tests/gold_tests/pluginTest/test_hooks/log.gold
index 8f3ab5b..7975eb5 100644
--- a/tests/gold_tests/pluginTest/test_hooks/log.gold
+++ b/tests/gold_tests/pluginTest/test_hooks/log.gold
@@ -24,10 +24,10 @@ Transaction: event=TS_EVENT_HTTP_READ_REQUEST_HDR
 Global: event=TS_EVENT_HTTP_TXN_CLOSE
 Session: event=TS_EVENT_HTTP_TXN_CLOSE
 Transaction: event=TS_EVENT_HTTP_TXN_CLOSE
+``
 Global: event=TS_EVENT_HTTP_SSN_CLOSE
 Session: event=TS_EVENT_HTTP_SSN_CLOSE
-Global: event=TS_EVENT_VCONN_CLOSE
-Global: ssl flag=1
+``
 Global: event=TS_EVENT_VCONN_START
 Global: ssl flag=1
 Global: event=TS_EVENT_SSL_SERVERNAME
@@ -43,7 +43,7 @@ Transaction: event=TS_EVENT_HTTP_READ_REQUEST_HDR
 Global: event=TS_EVENT_HTTP_TXN_CLOSE
 Session: event=TS_EVENT_HTTP_TXN_CLOSE
 Transaction: event=TS_EVENT_HTTP_TXN_CLOSE
+``
 Global: event=TS_EVENT_HTTP_SSN_CLOSE
 Session: event=TS_EVENT_HTTP_SSN_CLOSE
-Global: event=TS_EVENT_VCONN_CLOSE
-Global: ssl flag=1
+``
diff --git a/tests/gold_tests/pluginTest/test_hooks/test_hooks.test.py b/tests/gold_tests/pluginTest/test_hooks/test_hooks.test.py
index 40b9a47..934bc2b 100644
--- a/tests/gold_tests/pluginTest/test_hooks/test_hooks.test.py
+++ b/tests/gold_tests/pluginTest/test_hooks/test_hooks.test.py
@@ -99,3 +99,4 @@ tr.Processes.Default.Command = "echo check log"
 tr.Processes.Default.ReturnCode = 0
 f = tr.Disk.File("log.txt")
 f.Content = "log.gold"
+f.Content += Testers.ContainsExpression("Global: event=TS_EVENT_VCONN_CLOSE", "VCONN_CLOSE should trigger 2 times")