You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by su...@apache.org on 2015/06/30 23:48:12 UTC

trafficserver git commit: [TS-3714]: Changes summary: a) Issue a SSL_read right after SSL handshake to ensure data already in the SSL buffers is not lost. b) Add vc to net thread's read_ready_list immediately after accept, to ensure data already in the s

Repository: trafficserver
Updated Branches:
  refs/heads/master 4594083c9 -> e9eeab6be


[TS-3714]: Changes summary:
a) Issue a SSL_read right after SSL handshake to ensure
data already in the SSL buffers is not lost.
b) Add vc to net thread's read_ready_list immediately after
accept, to ensure data already in the socket buffers is not lost.
c) Fix a bunch of error cases to correctly release resources.


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

Branch: refs/heads/master
Commit: e9eeab6beb1021d9eb37a696504f02a3948a4aa5
Parents: 4594083
Author: Sudheer Vinukonda <su...@yahoo-inc.com>
Authored: Tue Jun 30 21:47:26 2015 +0000
Committer: Sudheer Vinukonda <su...@yahoo-inc.com>
Committed: Tue Jun 30 21:47:26 2015 +0000

----------------------------------------------------------------------
 iocore/net/P_SSLNetVConnection.h    | 27 ++++++++++++++++++
 iocore/net/SSLNetVConnection.cc     | 48 +++++++++++++++++++++++++++++---
 iocore/net/SSLNextProtocolAccept.cc |  2 +-
 iocore/net/UnixNetVConnection.cc    | 23 +++++++++++++--
 proxy/ProtocolProbeSessionAccept.cc | 32 ++++++++++++++++-----
 proxy/http/HttpClientSession.cc     | 13 +++++++++
 proxy/http/HttpSM.cc                | 11 +++++---
 7 files changed, 138 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/e9eeab6b/iocore/net/P_SSLNetVConnection.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNetVConnection.h b/iocore/net/P_SSLNetVConnection.h
index d4b79e9..75a3a3d 100644
--- a/iocore/net/P_SSLNetVConnection.h
+++ b/iocore/net/P_SSLNetVConnection.h
@@ -219,6 +219,30 @@ public:
   // least some of the hooks
   bool calledHooks(TSHttpHookID /* eventId */) { return (this->sslHandshakeHookState != HANDSHAKE_HOOKS_PRE); }
 
+  MIOBuffer*
+  get_ssl_iobuf()
+  {
+    return iobuf;
+  }
+
+  void
+  set_ssl_iobuf(MIOBuffer* buf)
+  {
+    iobuf = buf;
+  }
+
+  IOBufferReader*
+  get_ssl_reader()
+  {
+    return reader;
+  }
+
+  bool
+  isEosRcvd()
+  {
+    return eosRcvd;
+  }
+
 private:
   SSLNetVConnection(const SSLNetVConnection &);
   SSLNetVConnection &operator=(const SSLNetVConnection &);
@@ -256,6 +280,9 @@ private:
   const SSLNextProtocolSet *npnSet;
   Continuation *npnEndpoint;
   SessionAccept *sessionAcceptPtr;
+  MIOBuffer *iobuf;
+  IOBufferReader *reader;
+  bool eosRcvd;
 };
 
 typedef int (SSLNetVConnection::*SSLNetVConnHandler)(int, void *);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/e9eeab6b/iocore/net/SSLNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
index 9905efc..41cd780 100644
--- a/iocore/net/SSLNetVConnection.cc
+++ b/iocore/net/SSLNetVConnection.cc
@@ -190,8 +190,8 @@ ssl_read_from_net(SSLNetVConnection *sslvc, EThread *lthread, int64_t &ret)
   MIOBufferAccessor &buf = s->vio.buffer;
   IOBufferBlock *b = buf.writer()->first_write_block();
   int event = SSL_READ_ERROR_NONE;
-  int64_t bytes_read;
-  int64_t block_write_avail;
+  int64_t bytes_read = 0;
+  int64_t block_write_avail = 0;
   ssl_error_t sslErr = SSL_ERROR_NONE;
   int64_t nread = 0;
 
@@ -464,7 +464,7 @@ SSLNetVConnection::net_read_io(NetHandler *nh, EThread *lthread)
     if (ret == EVENT_ERROR) {
       this->read.triggered = 0;
       readSignalError(nh, err);
-    } else if (ret == SSL_HANDSHAKE_WANT_READ || ret == SSL_HANDSHAKE_WANT_ACCEPT) {
+    } else if (ret == SSL_HANDSHAKE_WANT_READ || ret == SSL_HANDSHAKE_WANT_ACCEPT || ret == EVENT_CONT) {
       if (SSLConfigParams::ssl_handshake_timeout_in > 0) {
         double handshake_time = ((Thread::get_hrtime() - sslHandshakeBeginTime)/1000000000);
         Debug ("ssl", "ssl handshake for vc %p, took %.3f seconds, configured handshake_timer: %d", this, handshake_time, SSLConfigParams::ssl_handshake_timeout_in);
@@ -488,6 +488,26 @@ SSLNetVConnection::net_read_io(NetHandler *nh, EThread *lthread)
       // the handshake is complete. Otherwise set up for continuing read
       // operations.
       if (ntodo <= 0) {
+        if (!getSSLClientConnection()) {
+          // we will not see another ET epoll event if the first byte is already
+          // in the ssl buffers, so, SSL_read if there's anything already..
+          Debug ("ssl", "ssl handshake completed on vc %p, check to see if first byte, is already in the ssl buffers", this);
+          this->iobuf = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+          this->reader = this->iobuf->alloc_reader();
+          s->vio.buffer.writer_for(this->iobuf);
+          if (this->iobuf) {
+            ret = ssl_read_from_net(this, lthread, r);
+            if (ret == SSL_READ_EOS) {
+              this->eosRcvd = true;
+            }
+            int pending = SSL_pending (this->ssl);
+            if (r > 0 || pending > 0) {
+              Debug ("ssl", "ssl read right after handshake, read %" PRId64 ", pending %d bytes, for vc %p", r, pending, this);
+            }
+          }
+          read.triggered = 0;
+          read_disable(nh, this);
+        }
         readSignalDone(VC_EVENT_READ_COMPLETE, nh);
       } else {
         read.triggered = 1;
@@ -591,6 +611,7 @@ SSLNetVConnection::net_read_io(NetHandler *nh, EThread *lthread)
     // close the connection if we have SSL_READ_EOS, this is the return value from ssl_read_from_net() if we get an
     // SSL_ERROR_ZERO_RETURN from SSL_get_error()
     // SSL_ERROR_ZERO_RETURN means that the origin server closed the SSL connection
+    eosRcvd = true;
     read.triggered = 0;
     readSignalDone(VC_EVENT_EOS, nh);
 
@@ -766,7 +787,8 @@ SSLNetVConnection::SSLNetVConnection()
   : ssl(NULL), sslHandshakeBeginTime(0), sslLastWriteTime(0), sslTotalBytesSent(0), hookOpRequested(TS_SSL_HOOK_OP_DEFAULT),
     sslHandShakeComplete(false), sslClientConnection(false), sslClientRenegotiationAbort(false), handShakeBuffer(NULL),
     handShakeHolder(NULL), handShakeReader(NULL), handShakeBioStored(0), sslPreAcceptHookState(SSL_HOOKS_INIT),
-    sslHandshakeHookState(HANDSHAKE_HOOKS_PRE), npnSet(NULL), npnEndpoint(NULL), sessionAcceptPtr(NULL)
+    sslHandshakeHookState(HANDSHAKE_HOOKS_PRE), npnSet(NULL), npnEndpoint(NULL), sessionAcceptPtr(NULL), iobuf(NULL),
+    reader(NULL), eosRcvd(false)
 {
 }
 
@@ -812,11 +834,21 @@ SSLNetVConnection::free(EThread *t)
   read.vio.mutex.clear();
   write.vio.mutex.clear();
   this->mutex.clear();
+  this->ep.stop();
+  this->con.close();
   flags = 0;
   SET_CONTINUATION_HANDLER(this, (SSLNetVConnHandler)&SSLNetVConnection::startEvent);
+  nh->read_ready_list.remove(this);
+  nh->write_ready_list.remove(this);
   nh = NULL;
   read.triggered = 0;
   write.triggered = 0;
+  read.enabled = 0;
+  write.enabled = 0;
+  read.vio._cont = NULL;
+  write.vio._cont = NULL;
+  read.vio.vc_server = NULL;
+  write.vio.vc_server = NULL;
   options.reset();
   closed = 0;
   ink_assert(con.fd == NO_FD);
@@ -824,8 +856,12 @@ SSLNetVConnection::free(EThread *t)
     SSL_free(ssl);
     ssl = NULL;
   }
+  if (iobuf) {
+    free_MIOBuffer(iobuf);
+  }
   sslHandShakeComplete = false;
   sslClientConnection = false;
+  sslHandshakeBeginTime = 0;
   sslLastWriteTime = 0;
   sslTotalBytesSent = 0;
   sslClientRenegotiationAbort = false;
@@ -838,6 +874,10 @@ SSLNetVConnection::free(EThread *t)
   npnSet = NULL;
   npnEndpoint = NULL;
   sessionAcceptPtr = NULL;
+  iobuf = NULL;
+  reader = NULL;
+  eosRcvd = false;
+  sslHandShakeComplete = false;
   free_handshake_buffers();
 
   if (from_accept_thread) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/e9eeab6b/iocore/net/SSLNextProtocolAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNextProtocolAccept.cc b/iocore/net/SSLNextProtocolAccept.cc
index 08b5920..c4748c3 100644
--- a/iocore/net/SSLNextProtocolAccept.cc
+++ b/iocore/net/SSLNextProtocolAccept.cc
@@ -87,7 +87,7 @@ struct SSLNextProtocolTrampoline : public Continuation {
     case VC_EVENT_INACTIVITY_TIMEOUT:
       netvc->do_io(VIO::CLOSE);
       delete this;
-      return EVENT_CONT;
+      return EVENT_ERROR;
     case VC_EVENT_READ_COMPLETE:
       break;
     default:

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/e9eeab6b/iocore/net/UnixNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
index 1749ba6..9533005 100644
--- a/iocore/net/UnixNetVConnection.cc
+++ b/iocore/net/UnixNetVConnection.cc
@@ -589,8 +589,11 @@ UnixNetVConnection::get_data(int id, void *data)
 VIO *
 UnixNetVConnection::do_io_read(Continuation *c, int64_t nbytes, MIOBuffer *buf)
 {
-  ink_assert(!closed);
   ink_assert(c || 0 == nbytes);
+  if (closed) {
+    Error("do_io_read invoked on closed vc %p, cont %p, nbytes %" PRId64 ", buf %p", this, c, nbytes, buf);
+    return NULL;
+  }
   read.vio.op = VIO::READ;
   read.vio.mutex = c ? c->mutex : this->mutex;
   read.vio._cont = c;
@@ -611,7 +614,10 @@ UnixNetVConnection::do_io_read(Continuation *c, int64_t nbytes, MIOBuffer *buf)
 VIO *
 UnixNetVConnection::do_io_write(Continuation *c, int64_t nbytes, IOBufferReader *reader, bool owner)
 {
-  ink_assert(!closed);
+  if (closed) {
+    Error("do_io_write invoked on closed vc %p, cont %p, nbytes %" PRId64 ", reader %p", this, c, nbytes, reader);
+    return NULL;
+  }
   write.vio.op = VIO::WRITE;
   write.vio.mutex = c ? c->mutex : this->mutex;
   write.vio._cont = c;
@@ -1035,6 +1041,13 @@ UnixNetVConnection::acceptEvent(int event, Event *e)
 
   nh->open_list.enqueue(this);
 
+#ifdef USE_EDGE_TRIGGER
+    // Set the vc as triggered and place it in the read ready queue in case there is already data on the socket.
+    Debug("iocore_net", "acceptEvent : Setting triggered and adding to the read ready queue");
+    read.triggered = 1;
+    nh->read_ready_list.enqueue(this);
+#endif
+
   if (inactivity_timeout_in) {
     UnixNetVConnection::set_inactivity_timeout(inactivity_timeout_in);
   }
@@ -1229,6 +1242,12 @@ UnixNetVConnection::free(EThread *t)
   nh = NULL;
   read.triggered = 0;
   write.triggered = 0;
+  read.enabled = 0;
+  write.enabled = 0;
+  read.vio._cont = NULL;
+  write.vio._cont = NULL;
+  read.vio.vc_server = NULL;
+  write.vio.vc_server = NULL;
   options.reset();
   closed = 0;
   ink_assert(!read.ready_link.prev && !read.ready_link.next);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/e9eeab6b/proxy/ProtocolProbeSessionAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/ProtocolProbeSessionAccept.cc b/proxy/ProtocolProbeSessionAccept.cc
index f512aa1..770119b 100644
--- a/proxy/ProtocolProbeSessionAccept.cc
+++ b/proxy/ProtocolProbeSessionAccept.cc
@@ -59,11 +59,11 @@ struct ProtocolProbeTrampoline : public Continuation, public ProtocolProbeSessio
   static const unsigned buffer_size_index = CLIENT_CONNECTION_FIRST_READ_BUFFER_SIZE_INDEX;
   IOBufferReader *reader;
 
-  explicit ProtocolProbeTrampoline(const ProtocolProbeSessionAccept *probe, ProxyMutex *mutex)
+  explicit ProtocolProbeTrampoline(const ProtocolProbeSessionAccept *probe, ProxyMutex *mutex, MIOBuffer *buffer, IOBufferReader *reader)
     : Continuation(mutex), probeParent(probe)
   {
-    this->iobuf = new_MIOBuffer(buffer_size_index);
-    reader = iobuf->alloc_reader(); // reader must be allocated only on a new MIOBuffer.
+    this->iobuf = buffer ? buffer : new_MIOBuffer(buffer_size_index);
+    this->reader = reader ? reader : iobuf->alloc_reader(); // reader must be allocated only on a new MIOBuffer.
     SET_HANDLER(&ProtocolProbeTrampoline::ioCompletionEvent);
   }
 
@@ -124,7 +124,11 @@ struct ProtocolProbeTrampoline : public Continuation, public ProtocolProbeSessio
     return EVENT_CONT;
 
   done:
-    free_MIOBuffer(this->iobuf);
+    SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(netvc);
+    if (!ssl_vc || (this->iobuf != ssl_vc->get_ssl_iobuf())) {
+      free_MIOBuffer(this->iobuf);
+    }
+    this->iobuf = NULL;
     delete this;
     return EVENT_CONT;
   }
@@ -141,12 +145,26 @@ ProtocolProbeSessionAccept::mainEvent(int event, void *data)
 
     VIO *vio;
     NetVConnection *netvc = static_cast<NetVConnection *>(data);
-    ProtocolProbeTrampoline *probe = new ProtocolProbeTrampoline(this, netvc->mutex);
+    SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(netvc);
+    MIOBuffer* buf = NULL;
+    IOBufferReader *reader = NULL;
+    if (ssl_vc) {
+      buf = ssl_vc->get_ssl_iobuf();
+      reader = ssl_vc->get_ssl_reader();
+    }
+    ProtocolProbeTrampoline * probe = new ProtocolProbeTrampoline(this, netvc->mutex, buf, reader);
 
     // XXX we need to apply accept inactivity timeout here ...
 
-    vio = netvc->do_io_read(probe, BUFFER_SIZE_FOR_INDEX(ProtocolProbeTrampoline::buffer_size_index), probe->iobuf);
-    vio->reenable();
+    if (!probe->reader->is_read_avail_more_than(0)) {
+      Debug ("http", "probe needs data, read..");
+      vio = netvc->do_io_read(probe, BUFFER_SIZE_FOR_INDEX(ProtocolProbeTrampoline::buffer_size_index), probe->iobuf);
+      vio->reenable();
+    } else {
+      Debug ("http", "probe already has data, call ioComplete directly..");
+      vio = netvc->do_io_read(NULL, 0, NULL);
+      probe->ioCompletionEvent(VC_EVENT_READ_COMPLETE, (void*)vio);
+    }
     return EVENT_CONT;
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/e9eeab6b/proxy/http/HttpClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
index b5e87f4..fe78143 100644
--- a/proxy/http/HttpClientSession.cc
+++ b/proxy/http/HttpClientSession.cc
@@ -200,6 +200,14 @@ HttpClientSession::new_connection(NetVConnection *new_vc, MIOBuffer *iobuf, IOBu
 
   DebugHttpSsn("[%" PRId64 "] session born, netvc %p", con_id, new_vc);
 
+  if (!iobuf) {
+    SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(new_vc);
+    if (ssl_vc) {
+      iobuf = ssl_vc->get_ssl_iobuf();
+      sm_reader = ssl_vc->get_ssl_reader();
+    }
+  }
+
   read_buffer = iobuf ? iobuf : new_MIOBuffer(HTTP_HEADER_BUFFER_SIZE_INDEX);
   sm_reader = reader ? reader : read_buffer->alloc_reader();
 
@@ -300,6 +308,11 @@ HttpClientSession::do_io_close(int alerrno)
     client_vc->set_active_timeout(HRTIME_SECONDS(current_reader->t_state.txn_conf->keep_alive_no_activity_timeout_out));
   } else {
     read_state = HCS_CLOSED;
+    // clean up ssl's first byte iobuf
+    SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(client_vc);
+    if (ssl_vc) {
+      ssl_vc->set_ssl_iobuf(NULL);
+    }
     if (upgrade_to_h2c) {
       Http2ClientSession *h2_session = http2ClientSessionAllocator.alloc();
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/e9eeab6b/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 1de4acc..f9962f0 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -545,8 +545,7 @@ HttpSM::setup_client_read_request_header()
   ua_entry->read_vio = ua_session->do_io_read(this, INT64_MAX, ua_buffer_reader->mbuf);
   // The header may already be in the buffer if this
   //  a request from a keep-alive connection
-  if (ua_buffer_reader->read_avail() > 0)
-    handleEvent(VC_EVENT_READ_READY, ua_entry->read_vio);
+  handleEvent(VC_EVENT_READ_READY, ua_entry->read_vio);
 }
 
 void
@@ -569,6 +568,12 @@ HttpSM::state_read_client_request_header(int event, void *data)
   int bytes_used = 0;
   ink_assert(ua_entry->eos == false);
 
+  // check to see if there was an EOS received on the SSL connection
+  SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(ua_session->get_netvc());
+  if (ssl_vc && ssl_vc->isEosRcvd()) {
+    DebugSM ("http", "EOS for ssl vc %p at read_first_btye state", ua_session->get_netvc());
+    event = VC_EVENT_EOS;
+  }
   switch (event) {
   case VC_EVENT_READ_READY:
   case VC_EVENT_READ_COMPLETE:
@@ -577,8 +582,6 @@ HttpSM::state_read_client_request_header(int event, void *data)
 
   case VC_EVENT_EOS:
     ua_entry->eos = true;
-    if (client_request_hdr_bytes != 0)
-      break;
   // Fall through
   case VC_EVENT_ERROR:
   case VC_EVENT_INACTIVITY_TIMEOUT: