You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by jp...@apache.org on 2014/05/22 18:56:06 UTC

git commit: TS-2793: remove UnixNetVConnection::selected_next_protocol

Repository: trafficserver
Updated Branches:
  refs/heads/master 77dd7d282 -> 729d0273c


TS-2793: remove UnixNetVConnection::selected_next_protocol

There's no need to track the negotiated NPN protocol in the VC
layer, since we can know exactly what to do at the accept layer.
Pass down enough information from the point where we register SPDY
acceptors to figure out what version to initialize at client session
time.


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

Branch: refs/heads/master
Commit: 729d0273ca38d1f3b02b8f5fbd57cc97350a4cb5
Parents: 77dd7d2
Author: James Peach <jp...@apache.org>
Authored: Tue May 20 14:29:02 2014 -0700
Committer: James Peach <jp...@apache.org>
Committed: Thu May 22 09:54:03 2014 -0700

----------------------------------------------------------------------
 CHANGES                           |  2 ++
 iocore/net/P_SSLNextProtocolSet.h |  3 +--
 iocore/net/P_UnixNetVConnection.h |  1 -
 iocore/net/SSLNetVConnection.cc   | 10 +++++---
 iocore/net/SSLNextProtocolSet.cc  | 45 +++++++---------------------------
 iocore/net/UnixNetVConnection.cc  |  3 +--
 proxy/http/HttpProxyServerMain.cc | 30 +++++++++++++++--------
 proxy/spdy/SpdyClientSession.cc   | 22 ++++-------------
 proxy/spdy/SpdyClientSession.h    |  4 +--
 proxy/spdy/SpdySessionAccept.cc   | 18 +++++++++++---
 proxy/spdy/SpdySessionAccept.h    | 11 +++++++--
 11 files changed, 70 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 3fce1ff..36c49d5 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.0.0
 
+  *) [TS-2793] Remove UnixNetVConnection::selected_next_protocol.
+
   *) [TS-2831] Add SPDY stream count statistic.
 
   *) [TS-1665] Remove the old traffic_shell (R.I.P.).

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/iocore/net/P_SSLNextProtocolSet.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNextProtocolSet.h b/iocore/net/P_SSLNextProtocolSet.h
index c4cae0e..729038a 100644
--- a/iocore/net/P_SSLNextProtocolSet.h
+++ b/iocore/net/P_SSLNextProtocolSet.h
@@ -39,8 +39,7 @@ public:
   bool unregisterEndpoint(const char *, Continuation *);
   bool advertiseProtocols(const unsigned char ** out, unsigned * len) const;
 
-  Continuation * findEndpoint(const char *) const;
-  Continuation * findEndpoint(const unsigned char *, unsigned, const char **) const;
+  Continuation * findEndpoint(const unsigned char *, unsigned) const;
 
   struct NextProtocolEndpoint
   {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/iocore/net/P_UnixNetVConnection.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_UnixNetVConnection.h b/iocore/net/P_UnixNetVConnection.h
index 71aa3af..867b11d 100644
--- a/iocore/net/P_UnixNetVConnection.h
+++ b/iocore/net/P_UnixNetVConnection.h
@@ -233,7 +233,6 @@ public:
   ink_hrtime submit_time;
   OOB_callback *oob_ptr;
   bool from_accept_thread;
-  const char *selected_next_protocol;
 
   int startEvent(int event, Event *e);
   int acceptEvent(int event, Event *e);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/iocore/net/SSLNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
index a25f873..57c022c 100644
--- a/iocore/net/SSLNetVConnection.cc
+++ b/iocore/net/SSLNetVConnection.cc
@@ -595,11 +595,15 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
         // If there's no NPN set, we should not have done this negotiation.
         ink_assert(this->npnSet != NULL);
 
-        this->npnEndpoint = this->npnSet->findEndpoint(proto, len, &this->selected_next_protocol);
+        this->npnEndpoint = this->npnSet->findEndpoint(proto, len);
         this->npnSet = NULL;
 
-        ink_assert(this->npnEndpoint != NULL);
-        Debug("ssl", "client selected next protocol %.*s", len, proto);
+        if (this->npnEndpoint == NULL) {
+          Error("failed to find registered SSL endpoint for '%.*s'", (int)len, (const char *)proto);
+          return EVENT_ERROR;
+        }
+
+        Debug("ssl", "client selected next protocol '%.*s'", len, proto);
       } else {
         Debug("ssl", "client did not select a next protocol");
       }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/iocore/net/SSLNextProtocolSet.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNextProtocolSet.cc b/iocore/net/SSLNextProtocolSet.cc
index ac5b22b..3460b21 100644
--- a/iocore/net/SSLNextProtocolSet.cc
+++ b/iocore/net/SSLNextProtocolSet.cc
@@ -95,6 +95,8 @@ SSLNextProtocolSet::advertiseProtocols(const unsigned char ** out, unsigned * le
 bool
 SSLNextProtocolSet::registerEndpoint(const char * proto, Continuation * ep)
 {
+  size_t len = strlen(proto);
+
   // Once we start advertising, the set is closed. We need to hand an immutable
   // string down into OpenSSL, and there is no mechanism to tell us when it's
   // done with it so we have to keep it forever.
@@ -102,11 +104,12 @@ SSLNextProtocolSet::registerEndpoint(const char * proto, Continuation * ep)
     return false;
   }
 
-  if (strlen(proto) > 255) {
+  // Both ALPN and NPN only allow 255 bytes of protocol name.
+  if (len > 255) {
     return false;
   }
 
-  if (findEndpoint(proto) == NULL) {
+  if (!findEndpoint((const unsigned char *)proto, len)) {
     this->endpoints.push(NEW(new NextProtocolEndpoint(proto, ep)));
     return true;
   }
@@ -133,34 +136,17 @@ SSLNextProtocolSet::unregisterEndpoint(const char * proto, Continuation * ep)
 
 Continuation *
 SSLNextProtocolSet::findEndpoint(
-  const unsigned char * proto, unsigned len, const char **selected_protocol) const
+  const unsigned char * proto, unsigned len) const
 {
-  for (const NextProtocolEndpoint * ep = this->endpoints.head;
-        ep != NULL; ep = this->endpoints.next(ep)) {
+  for (const NextProtocolEndpoint * ep = this->endpoints.head; ep != NULL; ep = this->endpoints.next(ep)) {
     size_t sz = strlen(ep->protocol);
     if (sz == len && memcmp(ep->protocol, proto, len) == 0) {
-      if (selected_protocol) {
-        *selected_protocol = ep->protocol;
-      }
       return ep->endpoint;
     }
   }
   return NULL;
 }
 
-Continuation *
-SSLNextProtocolSet::findEndpoint(const char * proto) const
-{
-  for (const NextProtocolEndpoint * ep = this->endpoints.head;
-        ep != NULL; ep = this->endpoints.next(ep)) {
-    if (strcmp(proto, ep->protocol) == 0) {
-      return ep->endpoint;
-    }
-  }
-
-  return NULL;
-}
-
 SSLNextProtocolSet::SSLNextProtocolSet()
   : npn(0), npnsz(0)
 {
@@ -176,22 +162,9 @@ SSLNextProtocolSet::~SSLNextProtocolSet()
 }
 
 SSLNextProtocolSet::NextProtocolEndpoint::NextProtocolEndpoint(
-        const char * proto, Continuation * ep)
-  : protocol(proto), endpoint(ep)
+        const char * _proto, Continuation * _ep)
+  : protocol(_proto),  endpoint(_ep)
 {
-# if 0
-  if (proto == TS_NPN_PROTOCOL_HTTP_1_1 ||
-      proto == TS_NPN_PROTOCOL_HTTP_1_0) {
-    proto_stack = ((1u << TS_PROTO_TLS) | (1u << TS_PROTO_HTTP));
-  } else if (proto == TS_NPN_PROTOCOL_SPDY_3_1 ||
-             proto == TS_NPN_PROTOCOL_SPDY_3 ||
-             proto == TS_NPN_PROTOCOL_SPDY_2 ||
-             proto == TS_NPN_PROTOCOL_SPDY_1) {
-    proto_stack = ((1u << TS_PROTO_TLS) | (1u << TS_PROTO_SPDY));
-  } else {
-    proto_stack = (1u << TS_PROTO_TLS);
-  }
-# endif
 }
 
 SSLNextProtocolSet::NextProtocolEndpoint::~NextProtocolEndpoint()

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/iocore/net/UnixNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
index d7f6c68..d29ccc0 100644
--- a/iocore/net/UnixNetVConnection.cc
+++ b/iocore/net/UnixNetVConnection.cc
@@ -800,8 +800,7 @@ UnixNetVConnection::UnixNetVConnection()
 #endif
     active_timeout(NULL), nh(NULL),
     id(0), flags(0), recursion(0), submit_time(0), oob_ptr(0),
-    from_accept_thread(false),
-    selected_next_protocol(NULL)
+    from_accept_thread(false)
 {
   memset(&local_addr, 0, sizeof local_addr);
   memset(&server_addr, 0, sizeof server_addr);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/proxy/http/HttpProxyServerMain.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
index 28386cf..b770a8f 100644
--- a/proxy/http/HttpProxyServerMain.cc
+++ b/proxy/http/HttpProxyServerMain.cc
@@ -172,13 +172,17 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
 
   ProtocolProbeSessionAccept *probe = NEW(new ProtocolProbeSessionAccept());
   HttpSessionAccept *http = NEW(new HttpSessionAccept(accept_opt));
-  if (port.m_session_protocol_preference.intersects(HTTP_PROTOCOL_SET))
+
+  if (port.m_session_protocol_preference.intersects(HTTP_PROTOCOL_SET)) {
     probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_HTTP, http);
+  }
 
 #if TS_HAS_SPDY
-  SpdySessionAccept *spdy = NEW(new SpdySessionAccept(http));
-  if (port.m_session_protocol_preference.intersects(SPDY_PROTOCOL_SET))
-    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY, spdy);
+  if (port.m_session_protocol_preference.intersects(SPDY_PROTOCOL_SET)) {
+    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY, NEW(new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
+  }
+
+  probe->registerEndpoint(TS_PROTO_SPDY, spdy31);
 #endif
 
 
@@ -193,17 +197,23 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
     // http/1.0, http/1.1, spdy/3, spdy/3.1
 
     // HTTP
-    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_HTTP_1_0))
+    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_HTTP_1_0)) {
       ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, http);
-    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_HTTP_1_1))
+    }
+
+    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_HTTP_1_1)) {
       ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, http);
+    }
 
     // SPDY
 #if TS_HAS_SPDY
-    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3))
-      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, spdy);
-    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3_1))
-      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, spdy);
+    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3)) {
+      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, NEW(new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3));
+    }
+
+    if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3_1)) {
+      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, NEW(new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
+    }
 #endif
 
     ink_scoped_mutex lock(ssl_plugin_mutex);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/proxy/spdy/SpdyClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyClientSession.cc b/proxy/spdy/SpdyClientSession.cc
index 43cbcff..602b2ee 100644
--- a/proxy/spdy/SpdyClientSession.cc
+++ b/proxy/spdy/SpdyClientSession.cc
@@ -54,9 +54,9 @@ SpdyRequest::clear()
 }
 
 void
-SpdyClientSession::init(NetVConnection * netvc)
+SpdyClientSession::init(NetVConnection * netvc, spdylay_proto_version vers)
 {
-  int version, r;
+  int r;
 
   atomic_inc(g_sm_cnt);
 
@@ -64,19 +64,7 @@ SpdyClientSession::init(NetVConnection * netvc)
   this->vc = netvc;
   this->req_map.clear();
 
-  // XXX this has to die ... TS-2793
-  UnixNetVConnection * unixvc = reinterpret_cast<UnixNetVConnection *>(netvc);
-
-  if (unixvc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_3_1)
-    version = SPDYLAY_PROTO_SPDY3_1;
-  else if (unixvc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_3)
-    version = SPDYLAY_PROTO_SPDY3;
-  else if (unixvc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_2)
-    version = SPDYLAY_PROTO_SPDY2;
-  else
-    version = SPDYLAY_PROTO_SPDY3;
-
-  r = spdylay_session_server_new(&session, version,
+  r = spdylay_session_server_new(&session, vers,
                                  &SPDY_CFG.spdy.callbacks, this);
   ink_release_assert(r == 0);
   sm_id = atomic_inc(g_sm_id);
@@ -149,12 +137,12 @@ SpdyClientSession::clear()
 }
 
 void
-spdy_sm_create(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
+spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader)
 {
   SpdyClientSession  *sm;
 
   sm = spdyClientSessionAllocator.alloc();
-  sm->init(netvc);
+  sm->init(netvc, vers);
 
   sm->req_buffer = iobuf ? reinterpret_cast<TSIOBuffer>(iobuf) : TSIOBufferCreate();
   sm->req_reader = reader ? reinterpret_cast<TSIOBufferReader>(reader) : TSIOBufferReaderAlloc(sm->req_buffer);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/proxy/spdy/SpdyClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyClientSession.h b/proxy/spdy/SpdyClientSession.h
index ed0f2dd..5845b84 100644
--- a/proxy/spdy/SpdyClientSession.h
+++ b/proxy/spdy/SpdyClientSession.h
@@ -110,7 +110,7 @@ public:
     clear();
   }
 
-  void init(NetVConnection * netvc);
+  void init(NetVConnection * netvc, spdylay_proto_version vers);
   void clear();
 
   int64_t sm_id;
@@ -138,7 +138,7 @@ private:
   int state_session_readwrite(int event, void * edata);
 };
 
-void spdy_sm_create(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader);
+void spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader);
 
 extern ClassAllocator<SpdyRequest> spdyRequestAllocator;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/proxy/spdy/SpdySessionAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
index 452d7c7..b8c1d75 100644
--- a/proxy/spdy/SpdySessionAccept.cc
+++ b/proxy/spdy/SpdySessionAccept.cc
@@ -26,11 +26,21 @@
 
 #if TS_HAS_SPDY
 #include "SpdyClientSession.h"
+
+static const spdylay_proto_version versmap[] = {
+  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
+  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
+  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
+};
+
 #endif
 
-SpdySessionAccept::SpdySessionAccept(Continuation *ep)
-    : SessionAccept(new_ProxyMutex()), endpoint(ep)
+SpdySessionAccept::SpdySessionAccept(unsigned vers)
+    : SessionAccept(new_ProxyMutex()), version(vers)
 {
+#if TS_HAS_SPDY
+  ink_release_assert(vers < countof(versmap));
+#endif
   SET_HANDLER(&SpdySessionAccept::mainEvent);
 }
 
@@ -41,7 +51,7 @@ SpdySessionAccept::mainEvent(int event, void * edata)
     NetVConnection * netvc =static_cast<NetVConnection *>(edata);
 
 #if TS_HAS_SPDY
-    spdy_sm_create(netvc, NULL, NULL);
+    spdy_sm_create(netvc, versmap[this->version], NULL, NULL);
     SpdyStatIncrCount(Config::STAT_TOTAL_STREAMS, this);
 #else
     Error("accepted a SPDY session, but SPDY support is not available");
@@ -59,7 +69,7 @@ void
 SpdySessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
 {
 #if TS_HAS_SPDY
-  spdy_sm_create(netvc, iobuf, reader);
+  spdy_sm_create(netvc, versmap[this->version], iobuf, reader);
 #else
   (void)netvc;
   (void)iobuf;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/729d0273/proxy/spdy/SpdySessionAccept.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySessionAccept.h b/proxy/spdy/SpdySessionAccept.h
index dfe6484..9d740e2 100644
--- a/proxy/spdy/SpdySessionAccept.h
+++ b/proxy/spdy/SpdySessionAccept.h
@@ -32,7 +32,14 @@
 class SpdySessionAccept: public SessionAccept
 {
 public:
-  SpdySessionAccept(Continuation *ep);
+
+  enum {
+    SPDY_VERSION_2 = 0,
+    SPDY_VERSION_3,
+    SPDY_VERSION_3_1,
+  };
+
+  explicit SpdySessionAccept(unsigned vers);
   ~SpdySessionAccept() {}
 
   void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
@@ -42,7 +49,7 @@ private:
   SpdySessionAccept(const SpdySessionAccept &); // disabled
   SpdySessionAccept& operator =(const SpdySessionAccept&); // disabled
 
-  Continuation *endpoint;
+  unsigned version;
 };
 
 #endif /* SpdySessionAccept_H_ */