You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by sh...@apache.org on 2015/05/17 22:12:33 UTC

trafficserver git commit: TS-3578: Rearrange ProxyClientSession's to enable VC object to be available during SSN_CLOSE hook.

Repository: trafficserver
Updated Branches:
  refs/heads/master 90debbe2d -> 05fd0d780


TS-3578: Rearrange ProxyClientSession's to enable VC object to be available
during SSN_CLOSE hook.


Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/05fd0d78
Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/05fd0d78
Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/05fd0d78

Branch: refs/heads/master
Commit: 05fd0d780efd0f6450e2e102aedbe93797448d41
Parents: 90debbe
Author: shinrich <sh...@yahoo-inc.com>
Authored: Fri May 1 14:55:53 2015 -0500
Committer: shinrich <sh...@yahoo-inc.com>
Committed: Sun May 17 15:11:39 2015 -0500

----------------------------------------------------------------------
 CHANGES                           |  2 ++
 plugins/tcpinfo/tcpinfo.cc        |  8 ++++--
 proxy/InkAPI.cc                   |  4 ++-
 proxy/PluginVC.cc                 |  3 +-
 proxy/ProxyClientSession.cc       | 10 +++++--
 proxy/ProxyClientSession.h        |  7 +++--
 proxy/http/HttpClientSession.cc   |  6 ++--
 proxy/http/HttpClientSession.h    |  8 +++++-
 proxy/http2/Http2ClientSession.cc |  5 ++--
 proxy/http2/Http2ClientSession.h  | 13 ++++++++-
 proxy/spdy/Makefile.am            |  2 ++
 proxy/spdy/SpdyClientSession.cc   | 42 +++++++++++++++++++++-------
 proxy/spdy/SpdyClientSession.h    | 50 +++++++++++++++++++++++++++++++---
 proxy/spdy/SpdySessionAccept.cc   |  8 ++++--
 14 files changed, 135 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 8a05998..14bf0a5 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 6.0.0
 
+  *) [TS-3578] Rearrange ProxyClientSession so VC is available in close session hook
+
   *) [TS-3610] Loading reads certificate from file multiple times.
 
   *) [TS-3608] Client side SSL does not validate upstream hostname

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/plugins/tcpinfo/tcpinfo.cc
----------------------------------------------------------------------
diff --git a/plugins/tcpinfo/tcpinfo.cc b/plugins/tcpinfo/tcpinfo.cc
index eeafec5..91e5c33 100644
--- a/plugins/tcpinfo/tcpinfo.cc
+++ b/plugins/tcpinfo/tcpinfo.cc
@@ -107,8 +107,12 @@ log_tcp_info(Config *config, const char *event_name, TSHttpSsn ssnp)
 
   TSReleaseAssert(config->log != NULL);
 
-  if (TSHttpSsnClientFdGet(ssnp, &fd) != TS_SUCCESS) {
-    TSDebug("tcpinfo", "error getting the client socket fd");
+  if (ssnp != NULL && (TSHttpSsnClientFdGet(ssnp, &fd) != TS_SUCCESS || fd <= 0)) {
+    TSDebug("tcpinfo", "error getting the client socket fd from ssn");
+    return;
+  }
+  if (ssnp == NULL) {
+    TSDebug("tcpinfo", "ssn is not specified");
     return;
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/InkAPI.cc
----------------------------------------------------------------------
diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
index 53bfb38..4a1c78c 100644
--- a/proxy/InkAPI.cc
+++ b/proxy/InkAPI.cc
@@ -33,6 +33,7 @@
 #include "MIME.h"
 #include "HTTP.h"
 #include "HttpClientSession.h"
+#include "Http2ClientSession.h"
 #include "HttpServerSession.h"
 #include "HttpSM.h"
 #include "HttpConfig.h"
@@ -6990,7 +6991,8 @@ TSHttpSsnClientFdGet(TSHttpSsn ssnp, int *fdp)
 {
   sdk_assert(sdk_sanity_check_null_ptr((void *)fdp) == TS_SUCCESS);
 
-  HttpClientSession *cs = (HttpClientSession *)ssnp;
+  VConnection *basecs = reinterpret_cast<VConnection *>(ssnp);
+  ProxyClientSession *cs = dynamic_cast<ProxyClientSession *>(basecs);
 
   if (cs == NULL)
     return TS_ERROR;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/PluginVC.cc
----------------------------------------------------------------------
diff --git a/proxy/PluginVC.cc b/proxy/PluginVC.cc
index 51641e5..5b72626 100644
--- a/proxy/PluginVC.cc
+++ b/proxy/PluginVC.cc
@@ -879,7 +879,8 @@ PluginVC::remove_from_keep_alive_lru()
 SOCKET
 PluginVC::get_socket()
 {
-  return 0;
+  // Return an invalid file descriptor
+  return ts::NO_FD;
 }
 
 void

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/ProxyClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/ProxyClientSession.cc b/proxy/ProxyClientSession.cc
index fab598e..b35f493 100644
--- a/proxy/ProxyClientSession.cc
+++ b/proxy/ProxyClientSession.cc
@@ -66,7 +66,7 @@ is_valid_hook(TSHttpHookID hookid)
 }
 
 void
-ProxyClientSession::cleanup()
+ProxyClientSession::destroy()
 {
   this->api_hooks.clear();
   this->mutex.clear();
@@ -167,9 +167,15 @@ ProxyClientSession::handle_api_return(int event)
       this->start();
     }
     break;
-  case TS_HTTP_SSN_CLOSE_HOOK:
+  case TS_HTTP_SSN_CLOSE_HOOK: {
+    NetVConnection *vc = this->get_netvc();
+    if (vc) {
+      vc->do_io_close();
+      this->release_netvc();
+    }
     this->destroy();
     break;
+  }
   default:
     Fatal("received invalid session hook %s (%d)", HttpDebugNames::get_api_hook_name(hookid), hookid);
     break;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/ProxyClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/ProxyClientSession.h b/proxy/ProxyClientSession.h
index a8079d0..4ed616c 100644
--- a/proxy/ProxyClientSession.h
+++ b/proxy/ProxyClientSession.h
@@ -38,7 +38,7 @@ class ProxyClientSession : public VConnection
 public:
   ProxyClientSession();
 
-  virtual void destroy() = 0;
+  virtual void destroy();
   virtual void start() = 0;
 
   virtual void new_connection(NetVConnection *new_vc, MIOBuffer *iobuf, IOBufferReader *reader, bool backdoor) = 0;
@@ -55,6 +55,9 @@ public:
     this->api_hooks.prepend(id, cont);
   }
 
+  virtual NetVConnection *get_netvc() const = 0;
+  virtual void release_netvc() = 0;
+
   APIHook *
   ssn_hook_get(TSHttpHookID id) const
   {
@@ -97,8 +100,6 @@ public:
   // Initiate an API hook invocation.
   void do_api_callout(TSHttpHookID id);
 
-  void cleanup();
-
   static int64_t next_connection_id();
 
 protected:

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/http/HttpClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
index c332e64..34afa84 100644
--- a/proxy/http/HttpClientSession.cc
+++ b/proxy/http/HttpClientSession.cc
@@ -93,7 +93,7 @@ HttpClientSession::destroy()
     conn_decrease = false;
   }
 
-  ProxyClientSession::cleanup();
+  super::destroy();
   THREAD_FREE(this, httpClientSessionAllocator, this_thread());
 }
 
@@ -299,11 +299,11 @@ HttpClientSession::do_io_close(int alerrno)
 
       h2_session->set_upgrade_context(&current_reader->t_state.hdr_info.client_request);
       h2_session->new_connection(client_vc, NULL, NULL, false /* backdoor */);
+      // Handed over control of the VC to the new H2 session, don't clean it up
+      this->release_netvc();
       // TODO Consider about handling HTTP/1 hooks and stats
     } else {
-      client_vc->do_io_close(alerrno);
       DebugHttpSsn("[%" PRId64 "] session closed", con_id);
-      client_vc = NULL;
     }
     HTTP_SUM_DYN_STAT(http_transactions_per_client_con, transact_count);
     HTTP_DECREMENT_DYN_STAT(http_current_client_connections_stat);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/http/HttpClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpClientSession.h b/proxy/http/HttpClientSession.h
index dca3391..684815d 100644
--- a/proxy/http/HttpClientSession.h
+++ b/proxy/http/HttpClientSession.h
@@ -50,6 +50,7 @@ class SecurityContext;
 class HttpClientSession : public ProxyClientSession
 {
 public:
+  typedef ProxyClientSession super; ///< Parent type.
   HttpClientSession();
 
   // Implement ProxyClientSession interface.
@@ -89,11 +90,16 @@ public:
     return half_close;
   };
   virtual void release(IOBufferReader *r);
-  NetVConnection *
+  virtual NetVConnection *
   get_netvc() const
   {
     return client_vc;
   };
+  virtual void
+  release_netvc()
+  {
+    client_vc = NULL;
+  }
 
   virtual void attach_server_session(HttpServerSession *ssession, bool transaction_done = true);
   HttpServerSession *

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/http2/Http2ClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ClientSession.cc b/proxy/http2/Http2ClientSession.cc
index def776a..3431657 100644
--- a/proxy/http2/Http2ClientSession.cc
+++ b/proxy/http2/Http2ClientSession.cc
@@ -72,8 +72,9 @@ Http2ClientSession::destroy()
 
   this->connection_state.destroy();
 
+  super::destroy();
+
   free_MIOBuffer(this->read_buffer);
-  ProxyClientSession::cleanup();
   http2ClientSessionAllocator.free(this);
 }
 
@@ -190,8 +191,6 @@ Http2ClientSession::do_io_close(int alerrno)
 
   send_connection_event(&this->connection_state, HTTP2_SESSION_EVENT_FINI, this);
 
-  this->client_vc->do_io_close(alerrno);
-  this->client_vc = NULL;
 
   do_api_callout(TS_HTTP_SSN_CLOSE_HOOK);
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/http2/Http2ClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ClientSession.h b/proxy/http2/Http2ClientSession.h
index 2456b99..5e3ab23 100644
--- a/proxy/http2/Http2ClientSession.h
+++ b/proxy/http2/Http2ClientSession.h
@@ -146,13 +146,14 @@ private:
 class Http2ClientSession : public ProxyClientSession
 {
 public:
+  typedef ProxyClientSession super; ///< Parent type.
   Http2ClientSession();
 
   typedef int (Http2ClientSession::*SessionHandler)(int, void *);
 
   // Implement ProxyClientSession interface.
   void start();
-  void destroy();
+  virtual void destroy();
   void new_connection(NetVConnection *new_vc, MIOBuffer *iobuf, IOBufferReader *reader, bool backdoor);
 
   // Implement VConnection interface.
@@ -161,6 +162,16 @@ public:
   void do_io_close(int lerrno = -1);
   void do_io_shutdown(ShutdownHowTo_t howto);
   void reenable(VIO *vio);
+  virtual NetVConnection *
+  get_netvc() const
+  {
+    return client_vc;
+  };
+  virtual void
+  release_netvc()
+  {
+    client_vc = NULL;
+  }
 
   int64_t
   connection_id() const

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/spdy/Makefile.am
----------------------------------------------------------------------
diff --git a/proxy/spdy/Makefile.am b/proxy/spdy/Makefile.am
index a7c44a9..5eaf948 100644
--- a/proxy/spdy/Makefile.am
+++ b/proxy/spdy/Makefile.am
@@ -18,11 +18,13 @@
 
 AM_CPPFLAGS = \
   $(iocore_include_dirs) \
+  -I$(top_srcdir)/proxy/api/ts \
   -I$(top_srcdir)/lib \
   -I$(top_builddir)/lib \
   -I$(top_srcdir)/lib/records \
   -I$(top_srcdir)/lib/ts \
   -I$(top_srcdir)/proxy \
+  -I$(top_srcdir)/proxy/http \
   -I$(top_srcdir)/proxy/api \
   -I$(top_srcdir)/proxy/hdrs \
   -I$(top_srcdir)/proxy/shared \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/spdy/SpdyClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyClientSession.cc b/proxy/spdy/SpdyClientSession.cc
index 8e831b4..b039892 100644
--- a/proxy/spdy/SpdyClientSession.cc
+++ b/proxy/spdy/SpdyClientSession.cc
@@ -22,6 +22,7 @@
  */
 
 #include "SpdyClientSession.h"
+#include "SpdySessionAccept.h"
 #include "I_Net.h"
 
 static ClassAllocator<SpdyClientSession> spdyClientSessionAllocator("spdyClientSessionAllocator");
@@ -85,16 +86,15 @@ SpdyRequest::clear()
 }
 
 void
-SpdyClientSession::init(NetVConnection *netvc, spdy::SessionVersion vers)
+SpdyClientSession::init(NetVConnection *netvc)
 {
   int r;
 
   this->mutex = new_ProxyMutex();
   this->vc = netvc;
   this->req_map.clear();
-  this->version = vers;
 
-  r = spdylay_session_server_new(&session, versmap[vers], &spdy_callbacks, this);
+  r = spdylay_session_server_new(&session, versmap[this->version], &spdy_callbacks, this);
 
   // A bit ugly but we need a thread and I don't want to wait until the
   // session start event in case of a time out generating a decrement
@@ -174,12 +174,14 @@ SpdyClientSession::clear()
 }
 
 void
-spdy_cs_create(NetVConnection *netvc, spdy::SessionVersion vers, MIOBuffer *iobuf, IOBufferReader *reader)
+SpdyClientSession::new_connection(NetVConnection *new_vc, MIOBuffer *iobuf, IOBufferReader *reader, bool backdoor)
 {
-  SpdyClientSession *sm;
+  // SPDY for the backdoor connections? Let's not deal woth that yet.
+  ink_release_assert(backdoor == false);
 
-  sm = spdyClientSessionAllocator.alloc();
-  sm->init(netvc, vers);
+  SpdyClientSession *sm = this;
+
+  sm->init(new_vc);
 
   sm->req_buffer = iobuf ? reinterpret_cast<TSIOBuffer>(iobuf) : TSIOBufferCreate();
   sm->req_reader = reader ? reinterpret_cast<TSIOBufferReader>(reader) : TSIOBufferReaderAlloc(sm->req_buffer);
@@ -251,8 +253,7 @@ SpdyClientSession::state_session_readwrite(int event, void *edata)
   Debug("spdy-event", "++++SpdyClientSession[%" PRIu64 "], EVENT:%d, ret:%d", this->sm_id, event, ret);
 out:
   if (ret) {
-    this->clear();
-    spdyClientSessionAllocator.free(this);
+    this->do_io_close();
   } else if (!from_fetch) {
     this->vc->set_inactivity_timeout(HRTIME_SECONDS(spdy_no_activity_timeout_in));
   }
@@ -260,6 +261,20 @@ out:
   return EVENT_CONT;
 }
 
+void
+SpdyClientSession::destroy()
+{
+  this->clear();
+  spdyClientSessionAllocator.free(this);
+}
+
+
+SpdyClientSession *
+SpdyClientSession::alloc()
+{
+  return spdyClientSessionAllocator.alloc();
+}
+
 int64_t
 SpdyClientSession::getPluginId() const
 {
@@ -272,7 +287,6 @@ SpdyClientSession::getPluginTag() const
   return npnmap[this->version];
 }
 
-
 static int
 spdy_process_read(TSEvent /* event ATS_UNUSED */, SpdyClientSession *sm)
 {
@@ -455,3 +469,11 @@ spdy_process_fetch_body(TSEvent event, SpdyClientSession *sm, TSFetchSM fetch_sm
   TSVIOReenable(sm->write_vio);
   return ret;
 }
+
+void
+SpdyClientSession::do_io_close(int alertno)
+{
+  // The object will be cleaned up from within ProxyClientSession::handle_api_return
+  // This way, the object will still be alive for any SSN_CLOSE hooks
+  do_api_callout(TS_HTTP_SSN_CLOSE_HOOK);
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/spdy/SpdyClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyClientSession.h b/proxy/spdy/SpdyClientSession.h
index 0b444ba..8c3750f 100644
--- a/proxy/spdy/SpdyClientSession.h
+++ b/proxy/spdy/SpdyClientSession.h
@@ -29,6 +29,7 @@
 #include "SpdyCallbacks.h"
 #include <openssl/md5.h>
 #include "Plugin.h"
+#include "ProxyClientSession.h"
 
 class SpdyClientSession;
 typedef int (*SpdyClientSessionHandler)(TSCont contp, TSEvent event, void *data);
@@ -87,15 +88,57 @@ public:
 
 extern ClassAllocator<SpdyRequest> spdyRequestAllocator;
 
-class SpdyClientSession : public Continuation, public PluginIdentity
+// class SpdyClientSession : public Continuation, public PluginIdentity
+class SpdyClientSession : public ProxyClientSession
 {
 public:
-  SpdyClientSession() : Continuation(NULL) {}
+  typedef ProxyClientSession super; ///< Parent type.
+  SpdyClientSession() {}
 
   ~SpdyClientSession() { clear(); }
 
-  void init(NetVConnection *netvc, spdy::SessionVersion vers);
+  void init(NetVConnection *netvc);
   void clear();
+  void destroy();
+
+  static SpdyClientSession *alloc();
+
+  VIO *
+  do_io_read(Continuation *, int64_t, MIOBuffer *)
+  {
+    // Due to spdylay, SPDY does not exercise do_io_read
+    ink_release_assert(false);
+    return NULL;
+  }
+  VIO *
+  do_io_write(Continuation *, int64_t, IOBufferReader *, bool)
+  {
+    // Due to spdylay, SPDY does not exercise do_io_write
+    ink_release_assert(false);
+    return NULL;
+  }
+  void
+  start()
+  {
+    ink_release_assert(false);
+  }
+  void do_io_close(int lerrno = -1);
+  void
+  do_io_shutdown(ShutdownHowTo_t howto)
+  {
+    ink_release_assert(false);
+  }
+  NetVConnection *
+  get_netvc() const
+  {
+    return vc;
+  }
+  void
+  release_netvc()
+  {
+    vc = NULL;
+  }
+  void new_connection(NetVConnection *new_vc, MIOBuffer *iobuf, IOBufferReader *reader, bool backdoor);
 
   int64_t sm_id;
   spdy::SessionVersion version;
@@ -147,6 +190,5 @@ private:
   int state_session_readwrite(int event, void *edata);
 };
 
-void spdy_cs_create(NetVConnection *netvc, spdy::SessionVersion vers, MIOBuffer *iobuf, IOBufferReader *reader);
 
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/05fd0d78/proxy/spdy/SpdySessionAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
index 3eb068f..98ff9ab 100644
--- a/proxy/spdy/SpdySessionAccept.cc
+++ b/proxy/spdy/SpdySessionAccept.cc
@@ -43,7 +43,9 @@ SpdySessionAccept::mainEvent(int event, void *edata)
     NetVConnection *netvc = static_cast<NetVConnection *>(edata);
 
 #if TS_HAS_SPDY
-    spdy_cs_create(netvc, this->version, NULL, NULL);
+    SpdyClientSession *sm = SpdyClientSession::alloc();
+    sm->version = this->version;
+    sm->new_connection(netvc, NULL, NULL, false);
 #else
     Error("accepted a SPDY session, but SPDY support is not available");
     netvc->do_io_close();
@@ -60,7 +62,9 @@ void
 SpdySessionAccept::accept(NetVConnection *netvc, MIOBuffer *iobuf, IOBufferReader *reader)
 {
 #if TS_HAS_SPDY
-  spdy_cs_create(netvc, this->version, iobuf, reader);
+  SpdyClientSession *sm = SpdyClientSession::alloc();
+  sm->version = this->version;
+  sm->new_connection(netvc, iobuf, reader, false);
 #else
   (void)netvc;
   (void)iobuf;