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/06/23 20:51:45 UTC

[trafficserver] branch 8.1.x updated: Fix rare SSN/TXN Start/Close Hook misorderings (#6364) (#6936)

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

zwoop 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 0912e18  Fix rare SSN/TXN Start/Close Hook misorderings (#6364) (#6936)
0912e18 is described below

commit 0912e182a97a49b23af87c9ccdde159dbb5e86f8
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Wed Jun 24 05:50:23 2020 +0900

    Fix rare SSN/TXN Start/Close Hook misorderings (#6364) (#6936)
    
    Cherry-pick 37cc7b82 for openclose_h2 AuTest.
    
    - without changes in test/gold_tests/
    - without changes of removing `client_vc->do_io_close()` from `Http1ClientSession::do_io_close()` and `Http2ClientSession::do_io_close()`.
      Because it made a bunch of crashes.
    
    ----
    
    - Fix rare SSN/TXN Start/Close Hook misorderings
    
    - Fix h2spec test failure
    
    Co-authored-by: a-a-ron acanary@verizonmedia.com
    (cherry picked from commit 37cc7b82af19382cd56d4ad3e7ba393fc9f7bff0)
    
    Conflicts:
    	proxy/ProxyClientSession.cc
    	proxy/ProxySession.h
    	proxy/http/HttpSM.cc
    	proxy/http/Http1ClientSession.cc
    	proxy/http2/Http2Stream.cc
    	proxy/http2/Http2ClientSession.cc
    	tests/gold_tests/continuations/openclose.test.py
    	tests/gold_tests/continuations/openclose_h2.test.py
    	tests/gold_tests/pluginTest/test_hooks/log.gold
    	tests/gold_tests/pluginTest/test_hooks/test_hooks.test.py
    	tests/tools/plugins/ssntxnorder_verify.cc
---
 proxy/ProxyClientSession.cc         |  7 ++++---
 proxy/ProxyClientSession.h          |  2 +-
 proxy/http/HttpSM.cc                | 34 +++++++++++++++++++++++++---------
 proxy/http/HttpSM.h                 |  4 ++--
 proxy/http2/Http2ClientSession.cc   |  2 +-
 proxy/http2/Http2ConnectionState.cc | 22 +++++++---------------
 proxy/http2/Http2ConnectionState.h  |  8 +++++++-
 proxy/http2/Http2Stream.cc          |  6 +++---
 8 files changed, 50 insertions(+), 35 deletions(-)

diff --git a/proxy/ProxyClientSession.cc b/proxy/ProxyClientSession.cc
index 1fe9daa..44d8dca 100644
--- a/proxy/ProxyClientSession.cc
+++ b/proxy/ProxyClientSession.cc
@@ -130,7 +130,7 @@ ProxyClientSession::state_api_callout(int event, void *data)
             if (!schedule_event) { // Don't bother to schedule is there is already one out.
               schedule_event = mutex->thread_holding->schedule_in(this, HRTIME_MSECONDS(10));
             }
-            return 0;
+            return -1;
           }
         }
 
@@ -160,7 +160,7 @@ ProxyClientSession::state_api_callout(int event, void *data)
   return 0;
 }
 
-void
+int
 ProxyClientSession::do_api_callout(TSHttpHookID id)
 {
   ink_assert(id == TS_HTTP_SSN_START_HOOK || id == TS_HTTP_SSN_CLOSE_HOOK);
@@ -171,10 +171,11 @@ ProxyClientSession::do_api_callout(TSHttpHookID id)
 
   if (this->hooks_on && this->has_hooks()) {
     SET_HANDLER(&ProxyClientSession::state_api_callout);
-    this->state_api_callout(EVENT_NONE, nullptr);
+    return this->state_api_callout(EVENT_NONE, nullptr);
   } else {
     this->handle_api_return(TS_EVENT_HTTP_CONTINUE);
   }
+  return 0;
 }
 
 void
diff --git a/proxy/ProxyClientSession.h b/proxy/ProxyClientSession.h
index 76dc0c6..5e44262 100644
--- a/proxy/ProxyClientSession.h
+++ b/proxy/ProxyClientSession.h
@@ -159,7 +159,7 @@ public:
   }
 
   // Initiate an API hook invocation.
-  void do_api_callout(TSHttpHookID id);
+  int do_api_callout(TSHttpHookID id);
 
   // Override if your session protocol allows this.
   virtual bool
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 13b4f7d..56e5e0e 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -333,13 +333,14 @@ HttpSM::set_ua_half_close_flag()
   ua_txn->set_half_close_flag(true);
 }
 
-inline void
+inline int
 HttpSM::do_api_callout()
 {
   if (hooks_set) {
-    do_api_callout_internal();
+    return do_api_callout_internal();
   } else {
     handle_api_return();
+    return 0;
   }
 }
 
@@ -365,7 +366,16 @@ HttpSM::state_add_to_list(int event, void * /* data ATS_UNUSED */)
   }
 
   t_state.api_next_action = HttpTransact::SM_ACTION_API_SM_START;
-  do_api_callout();
+  if (do_api_callout() < 0) {
+    // Didn't get the hook continuation lock. Clear the read and wait for next event
+    if (ua_entry->read_vio) {
+      // Seems like ua_entry->read_vio->disable(); should work, but that was
+      // not sufficient to stop the state machine from processing IO events until the
+      // TXN_START hooks had completed
+      ua_entry->read_vio = ua_entry->vc->do_io_read(nullptr, 0, nullptr);
+    }
+    return EVENT_CONT;
+  }
   return EVENT_DONE;
 }
 
@@ -519,6 +529,7 @@ HttpSM::attach_client_session(ProxyClientTransaction *client_vc, IOBufferReader
   ++reentrancy_count;
   // Add our state sm to the sm list
   state_add_to_list(EVENT_NONE, nullptr);
+
   // This is another external entry point and it is possible for the state machine to get terminated
   // while down the call chain from @c state_add_to_list. So we need to use the reentrancy_count to
   // prevent cleanup there and do it here as we return to the external caller.
@@ -1434,7 +1445,7 @@ plugins required to work with sni_routing.
             // handler has been changed the value isn't important to the rest of the state machine
             // but not resetting means there is no way to reliably detect re-entrance to this state with an
             // outstanding callout.
-            return 0;
+            return -1;
           }
         } else {
           plugin_lock = false;
@@ -5038,12 +5049,12 @@ HttpSM::do_http_server_open(bool raw)
   return;
 }
 
-void
+int
 HttpSM::do_api_callout_internal()
 {
   if (t_state.backdoor_request) {
     handle_api_return();
-    return;
+    return 0;
   }
 
   switch (t_state.api_next_action) {
@@ -5084,7 +5095,7 @@ HttpSM::do_api_callout_internal()
   case HttpTransact::SM_ACTION_API_SM_SHUTDOWN:
     if (callout_state == HTTP_API_IN_CALLOUT || callout_state == HTTP_API_DEFERED_SERVER_ERROR) {
       callout_state = HTTP_API_DEFERED_CLOSE;
-      return;
+      return 0;
     } else {
       cur_hook_id = TS_HTTP_TXN_CLOSE_HOOK;
     }
@@ -5096,7 +5107,7 @@ HttpSM::do_api_callout_internal()
 
   cur_hook  = nullptr;
   cur_hooks = 0;
-  state_api_callout(0, nullptr);
+  return state_api_callout(0, nullptr);
 }
 
 VConnection *
@@ -6835,7 +6846,12 @@ HttpSM::kill_this()
     //  the terminate_flag
     terminate_sm            = false;
     t_state.api_next_action = HttpTransact::SM_ACTION_API_SM_SHUTDOWN;
-    do_api_callout();
+    if (do_api_callout() < 0) { // Failed to get a continuation lock
+      // Need to hang out until we can complete the TXN_CLOSE hook
+      terminate_sm = false;
+      reentrancy_count--;
+      return;
+    }
   }
   // The reentrancy_count is still valid up to this point since
   //   the api shutdown hook is asynchronous and double frees can
diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
index b9ce2d7..5990d96 100644
--- a/proxy/http/HttpSM.h
+++ b/proxy/http/HttpSM.h
@@ -466,8 +466,8 @@ protected:
   void do_cache_prepare_action(HttpCacheSM *c_sm, CacheHTTPInfo *object_read_info, bool retry, bool allow_multiple = false);
   void do_cache_delete_all_alts(Continuation *cont);
   void do_auth_callout();
-  void do_api_callout();
-  void do_api_callout_internal();
+  int do_api_callout();
+  int do_api_callout_internal();
   void do_redirect();
   void redirect_request(const char *redirect_url, const int redirect_len);
   void do_drain_request_body();
diff --git a/proxy/http2/Http2ClientSession.cc b/proxy/http2/Http2ClientSession.cc
index 6d7d3de..f106250 100644
--- a/proxy/http2/Http2ClientSession.cc
+++ b/proxy/http2/Http2ClientSession.cc
@@ -308,7 +308,7 @@ Http2ClientSession::do_io_close(int alerrno)
 
   {
     SCOPED_MUTEX_LOCK(lock, this->connection_state.mutex, this_ethread());
-    this->connection_state.release_stream(nullptr);
+    this->connection_state.release_stream();
   }
 
   this->clear_session_active();
diff --git a/proxy/http2/Http2ConnectionState.cc b/proxy/http2/Http2ConnectionState.cc
index 394b299..27cdd20 100644
--- a/proxy/http2/Http2ConnectionState.cc
+++ b/proxy/http2/Http2ConnectionState.cc
@@ -531,7 +531,7 @@ rcv_rst_stream_frame(Http2ConnectionState &cstate, const Http2Frame &frame)
     Http2StreamDebug(cstate.ua_session, stream_id, "RST_STREAM: Error Code: %u", rst_stream.error_code);
 
     stream->set_rx_error_code({ProxyErrorClass::TXN, static_cast<uint32_t>(rst_stream.error_code)});
-    cstate.delete_stream(stream);
+    stream->initiating_close();
   }
 
   return Http2Error(Http2ErrorClass::HTTP2_ERROR_CLASS_NONE);
@@ -1000,8 +1000,8 @@ Http2ConnectionState::main_event_handler(int event, void *edata)
     ink_assert(this->fini_received == false);
     this->fini_received = true;
     cleanup_streams();
+    release_stream();
     SET_HANDLER(&Http2ConnectionState::state_closed);
-    this->release_stream(nullptr);
   } break;
 
   case HTTP2_SESSION_EVENT_XMIT: {
@@ -1298,13 +1298,11 @@ Http2ConnectionState::cleanup_streams()
     if (this->tx_error_code.cls != ProxyErrorClass::NONE) {
       s->set_tx_error_code(this->tx_error_code);
     }
-    this->delete_stream(s);
+    s->initiating_close();
     ink_assert(s != next);
     s = next;
   }
 
-  ink_assert(stream_list.empty());
-
   if (!is_state_closed()) {
     SCOPED_MUTEX_LOCK(lock, this->ua_session->mutex, this_ethread());
 
@@ -1367,16 +1365,10 @@ Http2ConnectionState::delete_stream(Http2Stream *stream)
 }
 
 void
-Http2ConnectionState::release_stream(Http2Stream *stream)
+Http2ConnectionState::release_stream()
 {
   REMEMBER(NO_EVENT, this->recursion)
 
-  if (stream) {
-    // Decrement total_client_streams_count here, because it's a counter include streams in the process of shutting down.
-    // Other counters (client_streams_in_count/client_streams_out_count) are already decremented in delete_stream().
-    --total_client_streams_count;
-  }
-
   SCOPED_MUTEX_LOCK(lock, this->mutex, this_ethread());
   if (this->ua_session) {
     ink_assert(this->mutex == ua_session->mutex);
@@ -1475,7 +1467,7 @@ Http2ConnectionState::send_data_frames_depends_on_priority()
   }
   case Http2SendDataFrameResult::DONE: {
     dependency_tree->deactivate(node, len);
-    delete_stream(stream);
+    stream->initiating_close();
     break;
   }
   default:
@@ -1573,7 +1565,7 @@ Http2ConnectionState::send_data_frames(Http2Stream *stream)
   if (stream->get_state() == Http2StreamState::HTTP2_STREAM_STATE_HALF_CLOSED_LOCAL ||
       stream->get_state() == Http2StreamState::HTTP2_STREAM_STATE_CLOSED) {
     Http2StreamDebug(this->ua_session, stream->get_id(), "Shutdown half closed local stream");
-    this->delete_stream(stream);
+    stream->initiating_close();
     return;
   }
 
@@ -1588,7 +1580,7 @@ Http2ConnectionState::send_data_frames(Http2Stream *stream)
       // RST_STREAM and WINDOW_UPDATE.
       // See 'closed' state written at [RFC 7540] 5.1.
       Http2StreamDebug(this->ua_session, stream->get_id(), "Shutdown stream");
-      this->delete_stream(stream);
+      stream->initiating_close();
     }
   }
 
diff --git a/proxy/http2/Http2ConnectionState.h b/proxy/http2/Http2ConnectionState.h
index a30990e..104d2f4 100644
--- a/proxy/http2/Http2ConnectionState.h
+++ b/proxy/http2/Http2ConnectionState.h
@@ -176,7 +176,7 @@ public:
   Http2Stream *find_stream(Http2StreamId id) const;
   void restart_streams();
   bool delete_stream(Http2Stream *stream);
-  void release_stream(Http2Stream *stream);
+  void release_stream();
   void cleanup_streams();
   void restart_receiving(Http2Stream *stream);
   void update_initial_rwnd(Http2WindowSize new_size);
@@ -228,6 +228,12 @@ public:
     return client_streams_in_count;
   }
 
+  void
+  decrement_stream_count()
+  {
+    --total_client_streams_count;
+  }
+
   double
   get_stream_error_rate() const
   {
diff --git a/proxy/http2/Http2Stream.cc b/proxy/http2/Http2Stream.cc
index 6cbfe0a..d632a3a 100644
--- a/proxy/http2/Http2Stream.cc
+++ b/proxy/http2/Http2Stream.cc
@@ -410,7 +410,6 @@ Http2Stream::terminate_if_possible()
     REMEMBER(NO_EVENT, this->reentrancy_count);
     Http2ClientSession *h2_proxy_ssn = static_cast<Http2ClientSession *>(parent);
     SCOPED_MUTEX_LOCK(lock, h2_proxy_ssn->connection_state.mutex, this_ethread());
-    h2_proxy_ssn->connection_state.delete_stream(this);
     destroy();
   }
 }
@@ -780,8 +779,10 @@ Http2Stream::destroy()
     // In many cases, this has been called earlier, so this call is a no-op
     h2_proxy_ssn->connection_state.delete_stream(this);
 
+    h2_proxy_ssn->connection_state.decrement_stream_count();
+
     // Update session's stream counts, so it accurately goes into keep-alive state
-    h2_proxy_ssn->connection_state.release_stream(this);
+    h2_proxy_ssn->connection_state.release_stream();
 
     // Do not access `_proxy_ssn` in below. It might be freed by `release_stream`.
   }
@@ -931,7 +932,6 @@ void
 Http2Stream::release(IOBufferReader *r)
 {
   super::release(r);
-  current_reader = nullptr; // State machine is on its own way down.
   this->do_io_close();
 }