You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by yu...@apache.org on 2014/03/20 18:32:59 UTC

[2/3] git commit: TS-2431: Preparation of SPDY protocol

TS-2431: Preparation of SPDY protocol

*) Create basic data structures for SPDY protocal, such as:
   SpdyAcceptCont.

*) SPDY will share the same port number with HTTP protocol, ATS can
   recognize them by detecting the first byte of client request.

*) HttpAccept looks like a subclass of NetAccept. To avoid confusing,
   rename HttpAccept to HttpAcceptCont.

Signed-off-by: Yunkai Zhang <qi...@taobao.com>


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

Branch: refs/heads/master
Commit: f90f3a488323ccb8b98c5571ef6cf035f1be51a9
Parents: f1a005e
Author: Yunkai Zhang <qi...@taobao.com>
Authored: Wed Dec 11 21:21:53 2013 +0800
Committer: Yunkai Zhang <qi...@taobao.com>
Committed: Fri Mar 21 01:29:21 2014 +0800

----------------------------------------------------------------------
 configure.ac                         |   1 +
 iocore/net/AcceptCont.cc             |  41 +++++++
 iocore/net/I_AcceptCont.h            |  48 ++++++++
 iocore/net/I_Net.h                   |   1 +
 iocore/net/I_NetProcessor.h          |   1 +
 iocore/net/Makefile.am               |   8 ++
 iocore/net/P_Net.h                   |   1 +
 iocore/net/P_NetAccept.h             |   2 +
 iocore/net/P_ProtocolAcceptCont.h    |  50 ++++++++
 iocore/net/P_ProtocolNetAccept.h     |  62 ++++++++++
 iocore/net/P_SSLNetAccept.h          |   1 +
 iocore/net/P_SSLNextProtocolAccept.h |   2 +-
 iocore/net/P_SSLNextProtocolSet.h    |   2 +-
 iocore/net/P_UnixNetVConnection.h    |   9 ++
 iocore/net/ProtocolAcceptCont.cc     |  73 +++++++++++
 iocore/net/ProtocolNetAccept.cc      |  68 ++++++++++
 iocore/net/SSLNetAccept.cc           |  16 ++-
 iocore/net/SSLNetVConnection.cc      |   3 +-
 iocore/net/SSLNextProtocolAccept.cc  |   2 +-
 iocore/net/SSLNextProtocolSet.cc     |   5 +-
 iocore/net/UnixNetAccept.cc          |  69 +++++++----
 iocore/net/UnixNetProcessor.cc       |  20 ++-
 iocore/net/UnixNetVConnection.cc     |  95 +++++++++++++-
 lib/ts/apidefs.h.in                  |   3 +
 proxy/InkAPI.cc                      |   6 +-
 proxy/Makefile.am                    |  12 +-
 proxy/SocksProxy.cc                  |   6 +-
 proxy/api/ts/InkAPIPrivateIOCore.h   |   4 +
 proxy/http/HttpAccept.cc             |  98 ---------------
 proxy/http/HttpAccept.h              | 198 ------------------------------
 proxy/http/HttpAcceptCont.cc         |  98 +++++++++++++++
 proxy/http/HttpAcceptCont.h          | 198 ++++++++++++++++++++++++++++++
 proxy/http/HttpProxyServerMain.cc    |  42 ++++---
 proxy/http/Makefile.am               |   7 +-
 proxy/spdy/Makefile.am               |  38 ++++++
 proxy/spdy/P_SpdyAcceptCont.h        |  46 +++++++
 proxy/spdy/SpdyAcceptCont.cc         |  37 ++++++
 37 files changed, 1004 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/configure.ac
----------------------------------------------------------------------
diff --git a/configure.ac b/configure.ac
index f87674d..796158e 100644
--- a/configure.ac
+++ b/configure.ac
@@ -2010,6 +2010,7 @@ AC_CONFIG_FILES([
   proxy/http/Makefile
   proxy/http/remap/Makefile
   proxy/logging/Makefile
+  proxy/spdy/Makefile
   rc/Makefile
   rc/trafficserver
   rc/trafficserver.conf

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/AcceptCont.cc
----------------------------------------------------------------------
diff --git a/iocore/net/AcceptCont.cc b/iocore/net/AcceptCont.cc
new file mode 100644
index 0000000..e112230
--- /dev/null
+++ b/iocore/net/AcceptCont.cc
@@ -0,0 +1,41 @@
+/** @file
+
+  AcceptCont
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "I_AcceptCont.h"
+#include "P_Net.h"
+
+AcceptCont::AcceptCont(ProxyMutex *amutex)
+    : Continuation(amutex)
+{
+  SET_HANDLER(&AcceptCont::mainEvent);
+}
+
+AcceptCont::~AcceptCont()
+{
+}
+
+void *
+AcceptCont::createNetAccept()
+{
+    return (NEW(new NetAccept));
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_AcceptCont.h
----------------------------------------------------------------------
diff --git a/iocore/net/I_AcceptCont.h b/iocore/net/I_AcceptCont.h
new file mode 100644
index 0000000..2d59e1e
--- /dev/null
+++ b/iocore/net/I_AcceptCont.h
@@ -0,0 +1,48 @@
+/** @file
+
+  AcceptCont
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#ifndef I_AcceptCont_H_
+#define I_AcceptCont_H_
+#include "I_Net.h"
+#include "I_VConnection.h"
+
+class AcceptCont: public Continuation
+{
+public:
+  AcceptCont(ProxyMutex *amutex);
+  ~AcceptCont();
+
+  //
+  // Virtual function allows creation of an SSLNetAccept
+  // or NetAccept transparent to NetProcessor.
+  //
+  // This function should return a pointer
+  // of NetAccept or its subclass.
+  //
+  virtual void *createNetAccept();
+
+private:
+  virtual int mainEvent(int event, void * netvc) = 0;
+};
+
+#endif /* I_AcceptCont_H_ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_Net.h
----------------------------------------------------------------------
diff --git a/iocore/net/I_Net.h b/iocore/net/I_Net.h
index 78f3d62..a98b6a3 100644
--- a/iocore/net/I_Net.h
+++ b/iocore/net/I_Net.h
@@ -91,6 +91,7 @@ extern int net_config_poll_timeout;
 
 #include "I_NetVConnection.h"
 #include "I_NetProcessor.h"
+#include "I_AcceptCont.h"
 
 void ink_net_init(ModuleVersion version);
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_NetProcessor.h
----------------------------------------------------------------------
diff --git a/iocore/net/I_NetProcessor.h b/iocore/net/I_NetProcessor.h
index 3deae4c..68ce117 100644
--- a/iocore/net/I_NetProcessor.h
+++ b/iocore/net/I_NetProcessor.h
@@ -100,6 +100,7 @@ public:
     AcceptOptions() { this->reset(); }
     /// Reset all values to defaults.
     self& reset();
+    bool create_default_NetAccept;
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/Makefile.am
----------------------------------------------------------------------
diff --git a/iocore/net/Makefile.am b/iocore/net/Makefile.am
index 4575e9e..efbfa86 100644
--- a/iocore/net/Makefile.am
+++ b/iocore/net/Makefile.am
@@ -52,6 +52,8 @@ libinknet_a_SOURCES = \
   I_UDPNet.h \
   I_UDPPacket.h \
   Inline.cc \
+  I_AcceptCont.h \
+  AcceptCont.cc \
   Net.cc \
   NetVConnection.cc \
   P_CompletionUtil.h \
@@ -61,6 +63,9 @@ libinknet_a_SOURCES = \
   P_Net.h \
   P_NetAccept.h \
   P_NetVConnection.h \
+  P_ProtocolNetAccept.h \
+  P_ProtocolAcceptCont.h \
+  P_Socks.h \
   P_SSLCertLookup.h \
   P_SSLConfig.h \
   P_SSLNetAccept.h \
@@ -81,6 +86,9 @@ libinknet_a_SOURCES = \
   P_UnixNetVConnection.h \
   P_UnixPollDescriptor.h \
   P_UnixUDPConnection.h \
+  ProtocolNetAccept.cc \
+  ProtocolAcceptCont.cc \
+  Socks.cc \
   SSLCertLookup.cc \
   SSLConfig.cc \
   SSLNetAccept.cc \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_Net.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_Net.h b/iocore/net/P_Net.h
index 13ab617..db9acc6 100644
--- a/iocore/net/P_Net.h
+++ b/iocore/net/P_Net.h
@@ -104,6 +104,7 @@ do { \
 #include "P_NetVCTest.h"
 #include "P_LibBulkIO.h"
 
+#include "P_ProtocolNetAccept.h"
 #include "P_SSLNetVConnection.h"
 #include "P_SSLNetProcessor.h"
 #include "P_SSLNetAccept.h"

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_NetAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_NetAccept.h b/iocore/net/P_NetAccept.h
index 3a82e4b..762a601 100644
--- a/iocore/net/P_NetAccept.h
+++ b/iocore/net/P_NetAccept.h
@@ -106,11 +106,13 @@ struct NetAccept:public Continuation
   virtual UnixNetVConnection *allocateThread(EThread *t);
   virtual void freeThread(UnixNetVConnection *vc, EThread *t);
   virtual UnixNetVConnection *allocateGlobal();
+  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection &con);
   virtual EventType getEtype();
 
   void init_accept_loop(const char *);
   virtual void init_accept(EThread * t = NULL);
   virtual void init_accept_per_thread();
+  virtual NetAccept *clone();
   // 0 == success
   int do_listen(bool non_blocking, bool transparent = false);
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolAcceptCont.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_ProtocolAcceptCont.h b/iocore/net/P_ProtocolAcceptCont.h
new file mode 100644
index 0000000..7631e5c
--- /dev/null
+++ b/iocore/net/P_ProtocolAcceptCont.h
@@ -0,0 +1,50 @@
+/** @file
+
+  ProtocolAcceptCont
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#ifndef P_ProtocolAcceptCont_H_
+#define P_ProtocolAcceptCont_H_
+
+#include "I_AcceptCont.h"
+
+class ProtocolAcceptCont: public AcceptCont
+{
+public:
+  ProtocolAcceptCont(): AcceptCont(NULL)
+  {
+    memset(endpoint, 0, TS_PROTO_MAX * sizeof(AcceptCont *));
+    SET_HANDLER(&ProtocolAcceptCont::mainEvent);
+  }
+  ~ProtocolAcceptCont() {}
+
+  void *createNetAccept();
+  void registerEndpoint(TSProtoType type, Continuation *ep);
+
+private:
+  int mainEvent(int event, void * netvc);
+  ProtocolAcceptCont(const ProtocolAcceptCont &); // disabled
+  ProtocolAcceptCont& operator =(const ProtocolAcceptCont&); // disabled
+
+  Continuation *endpoint[TS_PROTO_MAX];
+};
+
+#endif /* P_ProtocolAcceptCont_H_ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolNetAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_ProtocolNetAccept.h b/iocore/net/P_ProtocolNetAccept.h
new file mode 100644
index 0000000..ac49571
--- /dev/null
+++ b/iocore/net/P_ProtocolNetAccept.h
@@ -0,0 +1,62 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+/****************************************************************************
+
+  P_ProtocolNetAccept.h
+
+
+   NetAccept is a generalized facility which allows
+   Connections of different classes to be accepted either
+   from a blockable thread or by adaptive polling.
+
+   It is used by the NetProcessor and the ClusterProcessor
+   and should be considered PRIVATE to processor implementations.
+
+
+
+ ****************************************************************************/
+#if !defined (_ProtocolNetAccept_h_)
+#define _ProtocolNetAccept_h_
+
+#include "libts.h"
+#include "P_NetAccept.h"
+#include "P_Connection.h"
+
+class Event;
+class UnixNetVConnection;
+
+//
+// NetAccept
+// Handles accepting connections.
+//
+struct ProtocolNetAccept: public NetAccept
+{
+  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection &con);
+  virtual NetAccept *clone();
+
+  ProtocolNetAccept() {}
+  virtual ~ProtocolNetAccept() {}
+
+};
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNetAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNetAccept.h b/iocore/net/P_SSLNetAccept.h
index e9a8f7b..6d963a3 100644
--- a/iocore/net/P_SSLNetAccept.h
+++ b/iocore/net/P_SSLNetAccept.h
@@ -57,6 +57,7 @@ struct SSLNetAccept: public NetAccept
   virtual UnixNetVConnection *allocateGlobal();
   virtual EventType getEtype();
   virtual void init_accept_per_thread();
+  virtual NetAccept *clone();
 
   SSLNetAccept()
     { };

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNextProtocolAccept.h b/iocore/net/P_SSLNextProtocolAccept.h
index c3ee575..342485a 100644
--- a/iocore/net/P_SSLNextProtocolAccept.h
+++ b/iocore/net/P_SSLNextProtocolAccept.h
@@ -31,7 +31,7 @@
 #include "P_SSLNextProtocolSet.h"
 #include "I_IOBuffer.h"
 
-class SSLNextProtocolAccept: public Continuation
+class SSLNextProtocolAccept: public AcceptCont
 {
 public:
   SSLNextProtocolAccept(Continuation *);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolSet.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNextProtocolSet.h b/iocore/net/P_SSLNextProtocolSet.h
index e25f50d..d3c37b6 100644
--- a/iocore/net/P_SSLNextProtocolSet.h
+++ b/iocore/net/P_SSLNextProtocolSet.h
@@ -40,7 +40,7 @@ public:
   bool advertiseProtocols(const unsigned char ** out, unsigned * len) const;
 
   Continuation * findEndpoint(const char *) const;
-  Continuation * findEndpoint(const unsigned char *, unsigned, TSClientProtoStack *) const;
+  Continuation * findEndpoint(const unsigned char *, unsigned, TSClientProtoStack *, const char **) const;
 
   struct NextProtocolEndpoint
   {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_UnixNetVConnection.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_UnixNetVConnection.h b/iocore/net/P_UnixNetVConnection.h
index abdf4ba..0e4eae8 100644
--- a/iocore/net/P_UnixNetVConnection.h
+++ b/iocore/net/P_UnixNetVConnection.h
@@ -41,6 +41,13 @@ class UnixNetVConnection;
 class NetHandler;
 struct PollDescriptor;
 
+enum SpdyProbeState
+{
+  SPDY_PROBE_STATE_NONE,
+  SPDY_PROBE_STATE_BEGIN,
+  SPDY_PROBE_STATE_END
+};
+
 TS_INLINE void
 NetVCOptions::reset()
 {
@@ -233,6 +240,8 @@ public:
   ink_hrtime submit_time;
   OOB_callback *oob_ptr;
   bool from_accept_thread;
+  SpdyProbeState probe_state;
+  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/f90f3a48/iocore/net/ProtocolAcceptCont.cc
----------------------------------------------------------------------
diff --git a/iocore/net/ProtocolAcceptCont.cc b/iocore/net/ProtocolAcceptCont.cc
new file mode 100644
index 0000000..2e73b77
--- /dev/null
+++ b/iocore/net/ProtocolAcceptCont.cc
@@ -0,0 +1,73 @@
+/** @file
+
+  ProtocolAcceptCont
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "P_ProtocolAcceptCont.h"
+#include "P_SSLNextProtocolAccept.h"
+#include "P_Net.h"
+#include "I_Machine.h"
+#include "Error.h"
+
+void *
+ProtocolAcceptCont::createNetAccept()
+{
+  return ((NetAccept *) NEW(new ProtocolNetAccept));
+}
+
+void
+ProtocolAcceptCont::registerEndpoint(TSProtoType type, Continuation *ep)
+{
+  endpoint[type] = ep;
+}
+
+int
+ProtocolAcceptCont::mainEvent(int event, void *netvc)
+{
+  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
+  ink_release_assert((event == NET_EVENT_ACCEPT) ? (netvc!= 0) : (1));
+
+  if (event == NET_EVENT_ACCEPT) {
+    TSProtoType proto_type;
+    UnixNetVConnection *vc = (UnixNetVConnection *)netvc;
+
+    if (vc->proto_stack & (1u << TS_PROTO_TLS)) {
+      proto_type = TS_PROTO_TLS;
+    } else if (vc->proto_stack & (1u << TS_PROTO_HTTP)) {
+      proto_type = TS_PROTO_HTTP;
+    } else if (vc->proto_stack & (1u << TS_PROTO_SPDY)) {
+      proto_type = TS_PROTO_SPDY;
+    } else {
+      Warning("Invalid protocol stack:%x", vc->proto_stack);
+      return EVENT_CONT;
+    }
+
+    if (endpoint[proto_type])
+      endpoint[proto_type]->handleEvent(NET_EVENT_ACCEPT, netvc);
+    else
+      Warning("Unregistered protocol type:%x", proto_type);
+
+    return EVENT_CONT;
+  }
+
+  MachineFatal("Protocol Accept received fatal error: errno = %d", -((int)(intptr_t)netvc));
+  return EVENT_CONT;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/ProtocolNetAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/ProtocolNetAccept.cc b/iocore/net/ProtocolNetAccept.cc
new file mode 100644
index 0000000..c4257e2
--- /dev/null
+++ b/iocore/net/ProtocolNetAccept.cc
@@ -0,0 +1,68 @@
+/** @file
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "ink_config.h"
+#include "P_Net.h"
+#include "P_ProtocolNetAccept.h"
+
+UnixNetVConnection *
+ProtocolNetAccept::createSuitableVC(EThread *t, Connection &con)
+{
+  UnixNetVConnection *vc;
+
+  if (etype == SSLNetProcessor::ET_SSL && etype) {
+    // SSL protocol
+    if (t)
+      vc = (UnixNetVConnection *)THREAD_ALLOC(sslNetVCAllocator, t);
+    else
+      vc = (UnixNetVConnection *)sslNetVCAllocator.alloc();
+    vc->proto_stack = (1u << TS_PROTO_TLS);
+  } else {
+    if (t)
+      vc = THREAD_ALLOC(netVCAllocator, t);
+    else
+      vc = netVCAllocator.alloc();
+
+#if TS_HAS_SPDY
+    vc->probe_state = SPDY_PROBE_STATE_BEGIN;
+#else
+    vc->probe_state = SPDY_PROBE_STATE_NONE;
+#endif
+
+    //
+    // Protocol stack may be changed by
+    // following call of SpdyProbe()
+    //
+    vc->proto_stack = (1u << TS_PROTO_HTTP);
+  }
+
+  vc->con = con;
+  return vc;
+}
+
+NetAccept *
+ProtocolNetAccept::clone()
+{
+  NetAccept *na;
+  na = NEW(new ProtocolNetAccept);
+  *na = *this;
+  return na;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetAccept.cc b/iocore/net/SSLNetAccept.cc
index a321f89..6aa5ed9 100644
--- a/iocore/net/SSLNetAccept.cc
+++ b/iocore/net/SSLNetAccept.cc
@@ -70,10 +70,9 @@ SSLNetAccept::init_accept_per_thread()
   NetAccept *a = this;
   n = eventProcessor.n_threads_for_type[SSLNetProcessor::ET_SSL];
   for (i = 0; i < n; i++) {
-    if (i < n - 1) {
-      a = NEW(new SSLNetAccept);
-      *a = *this;
-    } else
+    if (i < n - 1)
+      a = clone();
+    else
       a = this;
     EThread *t = eventProcessor.eventthread[SSLNetProcessor::ET_SSL][i];
 
@@ -84,3 +83,12 @@ SSLNetAccept::init_accept_per_thread()
     t->schedule_every(a, period, etype);
   }
 }
+
+NetAccept *
+SSLNetAccept::clone()
+{
+  NetAccept *na;
+  na = NEW(new SSLNetAccept);
+  *na = *this;
+  return na;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
index b55dcf7..3925de9 100644
--- a/iocore/net/SSLNetVConnection.cc
+++ b/iocore/net/SSLNetVConnection.cc
@@ -578,7 +578,8 @@ 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->proto_stack);
+        this->npnEndpoint = this->npnSet->findEndpoint(proto, len, &this->proto_stack,
+                                                       &this->selected_next_protocol);
         this->npnSet = NULL;
 
         ink_assert(this->npnEndpoint != NULL);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNextProtocolAccept.cc b/iocore/net/SSLNextProtocolAccept.cc
index bfc2f0b..746995b 100644
--- a/iocore/net/SSLNextProtocolAccept.cc
+++ b/iocore/net/SSLNextProtocolAccept.cc
@@ -147,7 +147,7 @@ SSLNextProtocolAccept::unregisterEndpoint(
 }
 
 SSLNextProtocolAccept::SSLNextProtocolAccept(Continuation * ep)
-    : Continuation(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
+    : AcceptCont(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
 {
   SET_HANDLER(&SSLNextProtocolAccept::mainEvent);
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolSet.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNextProtocolSet.cc b/iocore/net/SSLNextProtocolSet.cc
index 148a6d1..ed0a5e3 100644
--- a/iocore/net/SSLNextProtocolSet.cc
+++ b/iocore/net/SSLNextProtocolSet.cc
@@ -133,7 +133,8 @@ SSLNextProtocolSet::unregisterEndpoint(const char * proto, Continuation * ep)
 
 Continuation *
 SSLNextProtocolSet::findEndpoint(const unsigned char * proto, unsigned len,
-                                 TSClientProtoStack *proto_stack) const
+                                 TSClientProtoStack *proto_stack,
+                                 const char **selected_protocol) const
 {
   for (const NextProtocolEndpoint * ep = this->endpoints.head;
         ep != NULL; ep = this->endpoints.next(ep)) {
@@ -141,6 +142,8 @@ SSLNextProtocolSet::findEndpoint(const unsigned char * proto, unsigned len,
     if (sz == len && memcmp(ep->protocol, proto, len) == 0) {
       if (proto_stack)
         *proto_stack = ep->proto_stack;
+      if (selected_protocol)
+        *selected_protocol = ep->protocol;
       return ep->endpoint;
     }
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetAccept.cc b/iocore/net/UnixNetAccept.cc
index c038a2e..af4d8f9 100644
--- a/iocore/net/UnixNetAccept.cc
+++ b/iocore/net/UnixNetAccept.cc
@@ -223,10 +223,9 @@ NetAccept::init_accept_per_thread()
   NetAccept *a;
   n = eventProcessor.n_threads_for_type[ET_NET];
   for (i = 0; i < n; i++) {
-    if (i < n - 1) {
-      a = NEW(new NetAccept);
-      *a = *this;
-    } else
+    if (i < n - 1)
+      a = clone();
+    else
       a = this;
     EThread *t = eventProcessor.eventthread[ET_NET][i];
     PollDescriptor *pd = get_PollDescriptor(t);
@@ -237,6 +236,14 @@ NetAccept::init_accept_per_thread()
   }
 }
 
+NetAccept *
+NetAccept::clone()
+{
+  NetAccept *na;
+  na = NEW(new NetAccept);
+  *na = *this;
+  return na;
+}
 
 int
 NetAccept::do_listen(bool non_blocking, bool transparent)
@@ -264,6 +271,20 @@ NetAccept::do_listen(bool non_blocking, bool transparent)
   return res;
 }
 
+UnixNetVConnection *
+NetAccept::createSuitableVC(EThread *t, Connection &con)
+{
+  UnixNetVConnection *vc;
+
+  if (t)
+    vc = allocateThread(t);
+  else
+    vc = allocateGlobal();
+
+  vc->con = con;
+
+  return vc;
+}
 
 int
 NetAccept::do_blocking_accept(EThread * t)
@@ -271,18 +292,11 @@ NetAccept::do_blocking_accept(EThread * t)
   int res = 0;
   int loop = accept_till_done;
   UnixNetVConnection *vc = NULL;
+  Connection con;
 
   //do-while for accepting all the connections
   //added by YTS Team, yamsat
   do {
-    vc = (UnixNetVConnection *)alloc_cache;
-    if (likely(!vc)) {
-      //vc = allocateThread(t);
-      vc = allocateGlobal(); // Bypass proxy / thread allocator
-      vc->from_accept_thread = true;
-      vc->id = net_next_connection_number();
-      alloc_cache = vc;
-    }
     ink_hrtime now = ink_get_hrtime();
 
     // Throttle accepts
@@ -297,7 +311,7 @@ NetAccept::do_blocking_accept(EThread * t)
       now = ink_get_hrtime();
     }
 
-    if ((res = server.accept(&vc->con)) < 0) {
+    if ((res = server.accept(&con)) < 0) {
     Lerror:
       int seriousness = accept_error_seriousness(res);
       if (seriousness >= 0) {   // not so bad
@@ -314,16 +328,23 @@ NetAccept::do_blocking_accept(EThread * t)
       }
       return -1;
     }
-    check_emergency_throttle(vc->con);
+
+    // Use 'NULL' to Bypass thread allocator
+    vc = createSuitableVC(NULL, con);
+    if (!vc)
+      return -1;
+    vc->from_accept_thread = true;
+    vc->id = net_next_connection_number();
     alloc_cache = NULL;
 
+    check_emergency_throttle(con);
+
     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
     vc->submit_time = now;
     ats_ip_copy(&vc->server_addr, &vc->con.addr);
     vc->set_is_transparent(server.f_inbound_transparent);
     vc->mutex = new_ProxyMutex();
     vc->action_ = *action_;
-    vc->proto_stack = (1u << TS_PROTO_HTTP);
     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) & UnixNetVConnection::acceptEvent);
     //eventProcessor.schedule_imm(vc, getEtype());
     eventProcessor.schedule_imm_signal(vc, getEtype());
@@ -384,6 +405,7 @@ NetAccept::acceptFastEvent(int event, void *ep)
   (void) event;
   (void) e;
   int bufsz, res;
+  Connection con;
 
   PollDescriptor *pd = get_PollDescriptor(e->ethread);
   UnixNetVConnection *vc = NULL;
@@ -394,10 +416,10 @@ NetAccept::acceptFastEvent(int event, void *ep)
       ifd = -1;
       return EVENT_CONT;
     }
-    vc = allocateThread(e->ethread);
 
-    socklen_t sz = sizeof(vc->con.addr);
-    int fd = socketManager.accept(server.fd, &vc->con.addr.sa, &sz);
+    socklen_t sz = sizeof(con.addr);
+    int fd = socketManager.accept(server.fd, &con.addr.sa, &sz);
+    con.fd = fd;
 
     if (likely(fd >= 0)) {
       Debug("iocore_net", "accepted a new socket: %d", fd);
@@ -443,6 +465,11 @@ NetAccept::acceptFastEvent(int event, void *ep)
       do {
         res = safe_nonblocking(fd);
       } while (res < 0 && (errno == EAGAIN || errno == EINTR));
+
+      vc = createSuitableVC(e->ethread, con);
+      if (!vc)
+        goto Ldone;
+
     } else {
       res = fd;
     }
@@ -453,20 +480,15 @@ NetAccept::acceptFastEvent(int event, void *ep)
           || res == -EPIPE
 #endif
         ) {
-        ink_assert(vc->con.fd == NO_FD);
-        ink_assert(!vc->link.next && !vc->link.prev);
-        freeThread(vc, e->ethread);
         goto Ldone;
       } else if (accept_error_seriousness(res) >= 0) {
         check_transient_accept_error(res);
-        freeThread(vc, e->ethread);
         goto Ldone;
       }
       if (!action_->cancelled)
         action_->continuation->handleEvent(EVENT_ERROR, (void *)(intptr_t)res);
       goto Lerror;
     }
-    vc->con.fd = fd;
 
     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
     vc->id = net_next_connection_number();
@@ -478,7 +500,6 @@ NetAccept::acceptFastEvent(int event, void *ep)
     vc->thread = e->ethread;
 
     vc->nh = get_NetHandler(e->ethread);
-    vc->proto_stack = (1u << TS_PROTO_HTTP);
 
     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) & UnixNetVConnection::mainEvent);
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetProcessor.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetProcessor.cc b/iocore/net/UnixNetProcessor.cc
index eaae439..ce59bbe 100644
--- a/iocore/net/UnixNetProcessor.cc
+++ b/iocore/net/UnixNetProcessor.cc
@@ -46,6 +46,7 @@ NetProcessor::AcceptOptions::reset()
   packet_mark = 0;
   packet_tos = 0;
   f_inbound_transparent = false;
+  create_default_NetAccept = true;
   return *this;
 }
 
@@ -84,16 +85,22 @@ NetProcessor::main_accept(Continuation *cont, SOCKET fd, AcceptOptions const& op
 Action *
 UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions const& opt)
 {
-  EventType et = opt.etype; // setEtype requires non-const ref.
-  NetAccept *na = createNetAccept();
+  EventType upgraded_etype = opt.etype; // setEtype requires non-const ref.
+  AcceptCont *acceptCont = static_cast<AcceptCont *>(cont);
   EThread *thread = this_ethread();
   ProxyMutex *mutex = thread->mutex;
   int accept_threads = opt.accept_threads; // might be changed.
   IpEndpoint accept_ip; // local binding address.
   char thr_name[MAX_THREAD_NAME_LENGTH];
 
+  NetAccept *na;
+  if (opt.create_default_NetAccept)
+    na = createNetAccept();
+  else
+    na = (NetAccept *)acceptCont->createNetAccept();
+
   // Potentially upgrade to SSL.
-  upgradeEtype(et);
+  upgradeEtype(upgraded_etype);
 
   // Fill in accept thread from configuration if necessary.
   if (opt.accept_threads < 0) {
@@ -137,7 +144,7 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
   na->sockopt_flags = opt.sockopt_flags;
   na->packet_mark = opt.packet_mark;
   na->packet_tos = opt.packet_tos;
-  na->etype = opt.etype;
+  na->etype = upgraded_etype;
   na->backdoor = opt.backdoor;
   if (na->callback_on_open)
     na->mutex = cont->mutex;
@@ -147,7 +154,10 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
         NetAccept *a;
 
         for (int i=1; i < accept_threads; ++i) {
-          a = createNetAccept();
+          if (opt.create_default_NetAccept)
+            a = createNetAccept();
+          else
+            a = (NetAccept *)acceptCont->createNetAccept();
           *a = *na;
           snprintf(thr_name, MAX_THREAD_NAME_LENGTH, "[ACCEPT %d:%d]", i-1, ats_ip_port_host_order(&accept_ip));
           a->init_accept_loop(thr_name);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
index bf1dc0a..8a919c5 100644
--- a/iocore/net/UnixNetVConnection.cc
+++ b/iocore/net/UnixNetVConnection.cc
@@ -38,6 +38,84 @@ typedef struct iovec IOVec;
 #define NET_MAX_IOV UIO_MAXIOV
 #endif
 
+struct SpdyProbeCont:public Continuation
+{
+  MIOBuffer buf;
+  unsigned char data;
+  SpdyProbeCont(): data(0)
+  {
+    SET_HANDLER(&SpdyProbeCont::mainEvent);
+  }
+
+  int mainEvent(int event, void *e);
+};
+
+static ClassAllocator<SpdyProbeCont> spdyProberContAllocator("spdyProberContAllocator");
+
+SpdyProbeCont *
+new_SpdyProbeCont(UnixNetVConnection *vc)
+{
+  SpdyProbeCont *c = spdyProberContAllocator.alloc();
+  c->buf.clear();
+  c->buf.set(&c->data, sizeof c->data);
+  c->buf._writer->fill(-(sizeof c->data));
+  c->mutex = vc->mutex;
+  return c;
+}
+void
+free_SpdyProbeCont(SpdyProbeCont *c)
+{
+  c->mutex.clear();
+  c->buf.clear();
+  spdyProberContAllocator.free(c);
+}
+
+inline int
+SpdyProbeCont::mainEvent(int event, void *e) {
+  UnixNetVConnection *vc = (UnixNetVConnection *) ((VIO *) e)->vc_server;
+  vc->probe_state = SPDY_PROBE_STATE_END;
+
+  switch (event) {
+  case VC_EVENT_EOS:
+  case VC_EVENT_ERROR:
+  case VC_EVENT_INACTIVITY_TIMEOUT:
+  case VC_EVENT_ACTIVE_TIMEOUT:
+    vc->do_io_close();
+    free_SpdyProbeCont(this);
+    return EVENT_DONE;
+  case VC_EVENT_READ_COMPLETE:
+    if ((data & 0x80) != 0) {
+      //
+      // SPDY Request
+      //
+      free_SpdyProbeCont(this);
+      vc->proto_stack = (1u << TS_PROTO_SPDY);
+      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
+      return EVENT_DONE;
+    } else {
+      //
+      // HTTP Request
+      //
+      free_SpdyProbeCont(this);
+      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
+      return EVENT_DONE;
+    }
+  default:
+    ink_release_assert(!"unexpected event");
+  }
+  return EVENT_CONT;
+}
+
+int SpdyProbeStart(UnixNetVConnection *vc)
+{
+  SpdyProbeCont *spdyProbeCont= new_SpdyProbeCont(vc);
+  //
+  // TODO: make it configurable
+  //
+  vc->set_inactivity_timeout(HRTIME_SECONDS(30));
+  vc->do_io_read(spdyProbeCont, 1, &spdyProbeCont->buf);
+  return EVENT_CONT;
+}
 // Global
 ClassAllocator<UnixNetVConnection> netVCAllocator("netVCAllocator");
 
@@ -259,8 +337,12 @@ read_from_net(NetHandler *nh, UnixNetVConnection *vc, EThread *thread)
         }
         b = b->next;
       }
+      ink_assert(vc->probe_state != SPDY_PROBE_STATE_BEGIN || niov == 1);
       if (niov == 1) {
-        r = socketManager.read(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len);
+        if (vc->probe_state == SPDY_PROBE_STATE_BEGIN) {
+          r = recv(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len, MSG_PEEK);
+        } else
+          r = socketManager.read(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len);
       } else {
         r = socketManager.readv(vc->con.fd, &tiovec[0], niov);
       }
@@ -801,7 +883,8 @@ UnixNetVConnection::UnixNetVConnection()
 #endif
     active_timeout(NULL), nh(NULL),
     id(0), flags(0), recursion(0), submit_time(0), oob_ptr(0),
-    from_accept_thread(false)
+    from_accept_thread(false), probe_state(SPDY_PROBE_STATE_NONE),
+    selected_next_protocol(NULL)
 {
   memset(&local_addr, 0, sizeof local_addr);
   memset(&server_addr, 0, sizeof server_addr);
@@ -988,7 +1071,13 @@ UnixNetVConnection::acceptEvent(int event, Event *e)
     UnixNetVConnection::set_inactivity_timeout(inactivity_timeout_in);
   if (active_timeout_in)
     UnixNetVConnection::set_active_timeout(active_timeout_in);
-  action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
+  if (probe_state == SPDY_PROBE_STATE_NONE)
+    action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
+  else {
+    ink_assert(probe_state == SPDY_PROBE_STATE_BEGIN);
+    SpdyProbeStart(this);
+  }
+
   return EVENT_DONE;
 }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/lib/ts/apidefs.h.in
----------------------------------------------------------------------
diff --git a/lib/ts/apidefs.h.in b/lib/ts/apidefs.h.in
index 86a650f..0356921 100644
--- a/lib/ts/apidefs.h.in
+++ b/lib/ts/apidefs.h.in
@@ -117,6 +117,9 @@ extern "C"
     TS_PROTO_SPDY = 13,
     TS_PROTO_RTMP = 14,
     TS_PROTO_WBSK = 15, /* WebSocket */
+
+    /* MAX value of TSProtoType  */
+    TS_PROTO_MAX
   } TSProtoType;
 
   typedef uint32_t TSClientProtoStack;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/InkAPI.cc
----------------------------------------------------------------------
diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
index 00a78e1..682f889 100644
--- a/proxy/InkAPI.cc
+++ b/proxy/InkAPI.cc
@@ -51,7 +51,7 @@
 #include "PluginVC.h"
 #include "api/ts/experimental.h"
 #include "ICP.h"
-#include "HttpAccept.h"
+#include "HttpAcceptCont.h"
 #include "PluginVC.h"
 #include "FetchSM.h"
 #include "HttpDebugNames.h"
@@ -6082,8 +6082,8 @@ TSHttpAltInfoQualitySet(TSHttpAltInfo infop, float quality)
   info->m_qvalue = quality;
 }
 
-extern HttpAccept *plugin_http_accept;
-extern HttpAccept *plugin_http_transparent_accept;
+extern HttpAcceptCont *plugin_http_accept;
+extern HttpAcceptCont *plugin_http_transparent_accept;
 
 TSVConn
 TSHttpConnect(sockaddr const* addr)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/Makefile.am
----------------------------------------------------------------------
diff --git a/proxy/Makefile.am b/proxy/Makefile.am
index 8ef6104..6dfc816 100644
--- a/proxy/Makefile.am
+++ b/proxy/Makefile.am
@@ -17,7 +17,7 @@
 #  limitations under the License.
 
 # Note that hdrs is targeted from ../Makefile.am
-SUBDIRS = congest http logging config
+SUBDIRS = congest http spdy logging config
 noinst_LIBRARIES =
 bin_PROGRAMS = \
   traffic_server \
@@ -52,11 +52,10 @@ noinst_HEADERS = \
   ConfigParse.h \
   Show.h
 
-
 if STATIC_LIBTS
-   which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
+    which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
 else
-   which_libts = $(top_builddir)/lib/ts/libtsutil.la
+    which_libts = $(top_builddir)/lib/ts/libtsutil.la
 endif
 
 EXTRA_DIST = InkAPITestTool.cc example_alarm_bin.sh example_prep.sh
@@ -121,6 +120,7 @@ endif
 traffic_server_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
 traffic_server_LDADD = \
   http/libhttp.a \
+  spdy/libspdy.a \
   http/remap/libhttp_remap.a \
   congest/libCongestionControl.a \
   logging/liblogging.a \
@@ -157,6 +157,7 @@ traffic_server_LDADD = \
   @LIBZ@ \
   @LIBLZMA@ \
   @LIBPROFILER@ \
+  @SPDYLAY_LIBS@ \
   -lm
 
 if BUILD_LUA_SUPPORT
@@ -223,6 +224,7 @@ traffic_sac_SOURCES = \
 traffic_sac_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
 traffic_sac_LDADD = \
   http/libhttp.a \
+  spdy/libspdy.a \
   shared/libdiagsconfig.a \
   http/remap/libhttp_remap.a \
   congest/libCongestionControl.a \
@@ -247,7 +249,7 @@ traffic_sac_LDADD = \
   $(top_builddir)/lib/records/librecprocess.a \
   $(top_builddir)/lib/ts/libtsutil.la \
   @LIBRESOLV@ @LIBPCRE@ @OPENSSL_LIBS@ @LIBTCL@ @hwloc_LIBS@ \
-  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@ -lm
+  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@ @SPDYLAY_LIBS@ -lm
 
 if BUILD_TESTS
   traffic_sac_SOURCES += RegressionSM.cc

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/SocksProxy.cc
----------------------------------------------------------------------
diff --git a/proxy/SocksProxy.cc b/proxy/SocksProxy.cc
index e447f12..d855689 100644
--- a/proxy/SocksProxy.cc
+++ b/proxy/SocksProxy.cc
@@ -31,7 +31,7 @@
 #include "StatSystem.h"
 #include "P_Net.h"
 #include "I_OneWayTunnel.h"
-#include "HttpAccept.h"
+#include "HttpAcceptCont.h"
 
 enum
 {
@@ -145,14 +145,14 @@ SocksProxy::mainEvent(int event, void *data)
 
     switch (state) {
     case HTTP_REQ:{
-      HttpAccept::Options ha_opt;
+      HttpAcceptCont::Options ha_opt;
       //This is a WRITE_COMPLETE. vio->nbytes == vio->ndone is true
 
       SOCKSPROXY_INC_STAT(socksproxy_http_connections_stat);
       Debug("SocksProxy", "Handing over the HTTP request\n");
 
       ha_opt.transport_type = clientVC->attributes;
-      HttpAccept http_accept(ha_opt);
+      HttpAcceptCont http_accept(ha_opt);
       http_accept.mainEvent(NET_EVENT_ACCEPT, clientVC);
       state = ALL_DONE;
       break;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/api/ts/InkAPIPrivateIOCore.h
----------------------------------------------------------------------
diff --git a/proxy/api/ts/InkAPIPrivateIOCore.h b/proxy/api/ts/InkAPIPrivateIOCore.h
index 26e3fac..97a9bcd 100644
--- a/proxy/api/ts/InkAPIPrivateIOCore.h
+++ b/proxy/api/ts/InkAPIPrivateIOCore.h
@@ -51,6 +51,10 @@ public:
 
   void handle_event_count(int event);
   int handle_event(int event, void *edata);
+  virtual void *createNetAccept()
+  {
+    return (NEW(new NetAccept));
+  }
 
 public:
   void *mdata;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpAccept.cc b/proxy/http/HttpAccept.cc
deleted file mode 100644
index 9fb0066..0000000
--- a/proxy/http/HttpAccept.cc
+++ /dev/null
@@ -1,98 +0,0 @@
-/** @file
-
-  A brief file description
-
-  @section license License
-
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
- */
-
-#include "HttpAccept.h"
-#include "IPAllow.h"
-#include "HttpClientSession.h"
-#include "I_Machine.h"
-#include "Error.h"
-
-int
-HttpAccept::mainEvent(int event, void *data)
-{
-  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
-  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
-
-  if (event == NET_EVENT_ACCEPT) {
-    ////////////////////////////////////////////////////
-    // if client address forbidden, close immediately //
-    ////////////////////////////////////////////////////
-    NetVConnection *netvc = static_cast<NetVConnection *>(data);
-    sockaddr const* client_ip = netvc->get_remote_addr();
-    uint32_t acl_method_mask = 0;
-    ip_port_text_buffer ipb;
-    IpAllow::scoped_config ipallow;
-
-    // The backdoor port is now only bound to "localhost", so no
-    // reason to check for if it's incoming from "localhost" or not.
-    if (backdoor) {
-      acl_method_mask = IpAllow::AllMethodMask();
-    } else if (ipallow && ((acl_method_mask = ipallow->match(client_ip)) == 0)) {
-      Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
-      netvc->do_io_close();
-
-      return VC_EVENT_CONT;
-    }
-
-    netvc->attributes = transport_type;
-
-    if (is_debug_tag_set("http_seq"))
-      Debug("http_seq", "[HttpAccept:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
-
-    HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
-
-   // copy over session related data.
-    new_session->f_outbound_transparent = f_outbound_transparent;
-    new_session->f_transparent_passthrough = f_transparent_passthrough;
-    new_session->outbound_ip4 = outbound_ip4;
-    new_session->outbound_ip6 = outbound_ip6;
-    new_session->outbound_port = outbound_port;
-    new_session->host_res_style = ats_host_res_from(client_ip->sa_family, host_res_preference);
-    new_session->acl_method_mask = acl_method_mask;
-
-    new_session->new_connection(netvc, backdoor);
-
-    return EVENT_CONT;
-  }
-
-  /////////////////
-  // EVENT_ERROR //
-  /////////////////
-  if (((long) data) == -ECONNABORTED) {
-    /////////////////////////////////////////////////
-    // Under Solaris, when accept() fails and sets //
-    // errno to EPROTO, it means the client has    //
-    // sent a TCP reset before the connection has  //
-    // been accepted by the server...  Note that   //
-    // in 2.5.1 with the Internet Server Supplement//
-    // and also in 2.6 the errno for this case has //
-    // changed from EPROTO to ECONNABORTED.        //
-    /////////////////////////////////////////////////
-
-    // FIX: add time to user_agent_hangup
-    HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
-  }
-
-  MachineFatal("HTTP accept received fatal error: errno = %d", -((int)(intptr_t)data));
-  return EVENT_CONT;
-}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpAccept.h b/proxy/http/HttpAccept.h
deleted file mode 100644
index 30d68e6..0000000
--- a/proxy/http/HttpAccept.h
+++ /dev/null
@@ -1,198 +0,0 @@
-/** @file
-
-  A brief file description
-
-  @section license License
-
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
- */
-
-#if !defined (_HttpAccept_h_)
-#define _HttpAccept_h_
-
-#include "libts.h"
-#include "P_EventSystem.h"
-#include "HttpConfig.h"
-#include "HTTP.h"
-
-namespace detail {
-  /** Options for @c HttpAccept.
-
-      @internal This is done as a separate class for two reasons.
-
-      The first is that in current usage many instances are created
-      with the same options so (for the client) this is easier and
-      more efficient than passing options directly to the @c
-      HttpAccept or calling setters.
-
-      The second is that @c HttpAccept is not provided with any thread
-      safety because it is intended as an immutable object. Putting
-      the setters here and not there makes that clearer.
-
-      We don't do this directly as nested class because we want to
-      inherit the data members rather than duplicate the declarations
-      and initializations.
-   */
-  class HttpAcceptOptions {
-  private:
-    typedef HttpAcceptOptions self; ///< Self reference type.
-  public:
-    HttpAcceptOptions();
-
-    // Connection type (HttpProxyPort::TransportType)
-    int transport_type;
-    /// Set the transport type.
-    self& setTransportType(int);
-    /// Local address to bind for outbound connections.
-    IpAddr outbound_ip4;
-    /// Local address to bind for outbound connections.
-    IpAddr outbound_ip6;
-    /// Set the outbound IP address to @a ip.
-    self& setOutboundIp(IpAddr& ip);
-    /// Set the outbound IP address to @a ip.
-    self& setOutboundIp(IpEndpoint* ip);
-    /// Local port for outbound connection.
-    uint16_t outbound_port;
-    /// Set outbound port.
-    self& setOutboundPort(uint16_t);
-    /// Outbound transparent.
-    bool f_outbound_transparent;
-    /// Set outbound transparency.
-    self& setOutboundTransparent(bool);
-    /// Transparent pass-through.
-    bool f_transparent_passthrough;
-    /// Set transparent passthrough.
-    self& setTransparentPassthrough(bool);
-    /// Accepting backdoor connections.
-    bool backdoor;
-    /// Set backdoor accept.
-    self& setBackdoor(bool);
-    /// Host address resolution preference order.
-    HostResPreferenceOrder host_res_preference;
-    /// Set the host query preference.
-    self& setHostResPreference(HostResPreferenceOrder const);
-  };
-
-  inline HttpAcceptOptions::HttpAcceptOptions()
-    : transport_type(0)
-    , outbound_port(0)
-    , f_outbound_transparent(false)
-    , f_transparent_passthrough(false)
-    , backdoor(false)
-  {
-    memcpy(host_res_preference, host_res_default_preference_order, sizeof(host_res_preference));
-  }
-
-  inline HttpAcceptOptions&
-  HttpAcceptOptions::setTransportType(int type) {
-    transport_type =  type;
-    return *this;
-  }
-
-  inline HttpAcceptOptions&
-  HttpAcceptOptions::setOutboundIp(IpAddr& ip) {
-    if (ip.isIp4()) outbound_ip4 = ip;
-    else if (ip.isIp6()) outbound_ip6 = ip;
-    return *this;
-  }
-
-  inline HttpAcceptOptions&
-  HttpAcceptOptions::setOutboundIp(IpEndpoint* ip) {
-    if (ip->isIp4()) outbound_ip4 = *ip;
-    else if (ip->isIp6()) outbound_ip6 = *ip;
-    return *this;
-  }
-
-  inline HttpAcceptOptions&
-  HttpAcceptOptions::setOutboundPort(uint16_t port) {
-    outbound_port = port;
-    return *this;
-  }
-
-  inline HttpAcceptOptions&
-  HttpAcceptOptions::setOutboundTransparent(bool flag) {
-    f_outbound_transparent = flag;
-    return *this;
-  }
-
-  inline HttpAcceptOptions&
-  HttpAcceptOptions::setTransparentPassthrough(bool flag) {
-    f_transparent_passthrough = flag;
-    return *this;
-  }
-
- inline HttpAcceptOptions&
-  HttpAcceptOptions::setBackdoor(bool flag) {
-    backdoor = flag;
-    return *this;
-  }
-
-  inline HttpAcceptOptions&
-  HttpAcceptOptions::setHostResPreference(HostResPreferenceOrder const order) {
-    memcpy(host_res_preference, order, sizeof(host_res_preference));
-    return *this;
-  }
-}
-
-/**
-   The continuation mutex is NULL to allow parellel accepts in NT. No
-   state is recorded by the handler and values are required to be set
-   during construction via the @c Options struct and never changed. So
-   a NULL mutex is safe.
-
-   Most of the state is simply passed on to the @c ClientSession after
-   an accept. It is done here because this is the least bad pathway
-   from the top level configuration to the HTTP session.
-*/
-
-class HttpAccept: public Continuation, private detail::HttpAcceptOptions
-{
-private:
-  typedef HttpAccept self; ///< Self reference type.
-public:
-  /** Construction options.
-      Provide an easier to remember typedef for clients.
-  */
-  typedef detail::HttpAcceptOptions Options;
-
-  /** Default constructor.
-    
-      @internal We don't use a static default options object because of
-      initialization order issues. It is important to pick up data that is read
-      from the config file and a static is initialized long before that point.
-  */
-  HttpAccept(Options const& opt = Options())
-    : Continuation(NULL)
-    , detail::HttpAcceptOptions(opt) // copy these.
-  {
-    SET_HANDLER(&HttpAccept::mainEvent);
-    return;
-  }
-
-  ~HttpAccept()
-  {
-    return;
-  }
-
-  int mainEvent(int event, void *netvc);
-
-private:
-    HttpAccept(const HttpAccept &);
-    HttpAccept & operator =(const HttpAccept &);
-};
-
-#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpAcceptCont.cc b/proxy/http/HttpAcceptCont.cc
new file mode 100644
index 0000000..d3744e7
--- /dev/null
+++ b/proxy/http/HttpAcceptCont.cc
@@ -0,0 +1,98 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "HttpAcceptCont.h"
+#include "IPAllow.h"
+#include "HttpClientSession.h"
+#include "I_Machine.h"
+#include "Error.h"
+
+int
+HttpAcceptCont::mainEvent(int event, void *data)
+{
+  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
+  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
+
+  if (event == NET_EVENT_ACCEPT) {
+    ////////////////////////////////////////////////////
+    // if client address forbidden, close immediately //
+    ////////////////////////////////////////////////////
+    NetVConnection *netvc = static_cast<NetVConnection *>(data);
+    sockaddr const* client_ip = netvc->get_remote_addr();
+    uint32_t acl_method_mask = 0;
+    ip_port_text_buffer ipb;
+    IpAllow::scoped_config ipallow;
+
+    // The backdoor port is now only bound to "localhost", so no
+    // reason to check for if it's incoming from "localhost" or not.
+    if (backdoor) {
+      acl_method_mask = IpAllow::AllMethodMask();
+    } else if (ipallow && ((acl_method_mask = ipallow->match(client_ip)) == 0)) {
+      Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
+      netvc->do_io_close();
+
+      return VC_EVENT_CONT;
+    }
+
+    netvc->attributes = transport_type;
+
+    if (is_debug_tag_set("http_seq"))
+      Debug("http_seq", "[HttpAcceptCont:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
+
+    HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
+
+   // copy over session related data.
+    new_session->f_outbound_transparent = f_outbound_transparent;
+    new_session->f_transparent_passthrough = f_transparent_passthrough;
+    new_session->outbound_ip4 = outbound_ip4;
+    new_session->outbound_ip6 = outbound_ip6;
+    new_session->outbound_port = outbound_port;
+    new_session->host_res_style = ats_host_res_from(client_ip->sa_family, host_res_preference);
+    new_session->acl_method_mask = acl_method_mask;
+
+    new_session->new_connection(netvc, backdoor);
+
+    return EVENT_CONT;
+  }
+
+  /////////////////
+  // EVENT_ERROR //
+  /////////////////
+  if (((long) data) == -ECONNABORTED) {
+    /////////////////////////////////////////////////
+    // Under Solaris, when accept() fails and sets //
+    // errno to EPROTO, it means the client has    //
+    // sent a TCP reset before the connection has  //
+    // been accepted by the server...  Note that   //
+    // in 2.5.1 with the Internet Server Supplement//
+    // and also in 2.6 the errno for this case has //
+    // changed from EPROTO to ECONNABORTED.        //
+    /////////////////////////////////////////////////
+
+    // FIX: add time to user_agent_hangup
+    HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
+  }
+
+  MachineFatal("HTTP accept received fatal error: errno = %d", -((int)(intptr_t)data));
+  return EVENT_CONT;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpAcceptCont.h b/proxy/http/HttpAcceptCont.h
new file mode 100644
index 0000000..5b8c437
--- /dev/null
+++ b/proxy/http/HttpAcceptCont.h
@@ -0,0 +1,198 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#if !defined (_HttpAcceptCont_h_)
+#define _HttpAcceptCont_h_
+
+#include "libts.h"
+#include "P_EventSystem.h"
+#include "HttpConfig.h"
+#include "HTTP.h"
+#include "I_Net.h"
+
+namespace detail {
+  /** Options for @c HttpAcceptCont.
+
+      @internal This is done as a separate class for two reasons.
+
+      The first is that in current usage many instances are created
+      with the same options so (for the client) this is easier and
+      more efficient than passing options directly to the @c
+      HttpAcceptCont or calling setters.
+
+      The second is that @c HttpAcceptCont is not provided with any thread
+      safety because it is intended as an immutable object. Putting
+      the setters here and not there makes that clearer.
+
+      We don't do this directly as nested class because we want to
+      inherit the data members rather than duplicate the declarations
+      and initializations.
+   */
+  class HttpAcceptContOptions {
+  private:
+    typedef HttpAcceptContOptions self; ///< Self reference type.
+  public:
+    HttpAcceptContOptions();
+
+    // Connection type (HttpProxyPort::TransportType)
+    int transport_type;
+    /// Set the transport type.
+    self& setTransportType(int);
+    /// Local address to bind for outbound connections.
+    IpAddr outbound_ip4;
+    /// Local address to bind for outbound connections.
+    IpAddr outbound_ip6;
+    /// Set the outbound IP address to @a ip.
+    self& setOutboundIp(IpAddr& ip);
+    /// Set the outbound IP address to @a ip.
+    self& setOutboundIp(IpEndpoint* ip);
+    /// Local port for outbound connection.
+    uint16_t outbound_port;
+    /// Set outbound port.
+    self& setOutboundPort(uint16_t);
+    /// Outbound transparent.
+    bool f_outbound_transparent;
+    /// Set outbound transparency.
+    self& setOutboundTransparent(bool);
+    /// Transparent pass-through.
+    bool f_transparent_passthrough;
+    /// Set transparent passthrough.
+    self& setTransparentPassthrough(bool);
+    /// Accepting backdoor connections.
+    bool backdoor;
+    /// Set backdoor accept.
+    self& setBackdoor(bool);
+    /// Host address resolution preference order.
+    HostResPreferenceOrder host_res_preference;
+    /// Set the host query preference.
+    self& setHostResPreference(HostResPreferenceOrder const);
+  };
+
+  inline HttpAcceptContOptions::HttpAcceptContOptions()
+    : transport_type(0)
+    , outbound_port(0)
+    , f_outbound_transparent(false)
+    , f_transparent_passthrough(false)
+    , backdoor(false)
+  {
+    memcpy(host_res_preference, host_res_default_preference_order, sizeof(host_res_preference));
+  }
+
+  inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setTransportType(int type) {
+    transport_type =  type;
+    return *this;
+  }
+
+  inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setOutboundIp(IpAddr& ip) {
+    if (ip.isIp4()) outbound_ip4 = ip;
+    else if (ip.isIp6()) outbound_ip6 = ip;
+    return *this;
+  }
+
+  inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setOutboundIp(IpEndpoint* ip) {
+    if (ip->isIp4()) outbound_ip4 = *ip;
+    else if (ip->isIp6()) outbound_ip6 = *ip;
+    return *this;
+  }
+
+  inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setOutboundPort(uint16_t port) {
+    outbound_port = port;
+    return *this;
+  }
+
+  inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setOutboundTransparent(bool flag) {
+    f_outbound_transparent = flag;
+    return *this;
+  }
+
+  inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setTransparentPassthrough(bool flag) {
+    f_transparent_passthrough = flag;
+    return *this;
+  }
+
+ inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setBackdoor(bool flag) {
+    backdoor = flag;
+    return *this;
+  }
+
+  inline HttpAcceptContOptions&
+  HttpAcceptContOptions::setHostResPreference(HostResPreferenceOrder const order) {
+    memcpy(host_res_preference, order, sizeof(host_res_preference));
+    return *this;
+  }
+}
+
+/**
+   The continuation mutex is NULL to allow parellel accepts in NT. No
+   state is recorded by the handler and values are required to be set
+   during construction via the @c Options struct and never changed. So
+   a NULL mutex is safe.
+
+   Most of the state is simply passed on to the @c ClientSession after
+   an accept. It is done here because this is the least bad pathway
+   from the top level configuration to the HTTP session.
+*/
+
+class HttpAcceptCont: public AcceptCont, private detail::HttpAcceptContOptions
+{
+private:
+  typedef HttpAcceptCont self; ///< Self reference type.
+public:
+  /** Construction options.
+      Provide an easier to remember typedef for clients.
+  */
+  typedef detail::HttpAcceptContOptions Options;
+
+  /** Default constructor.
+      @internal We don't use a static default options object because of
+      initialization order issues. It is important to pick up data that is read
+      from the config file and a static is initialized long before that point.
+  */
+  HttpAcceptCont(Options const& opt = Options())
+    : AcceptCont(NULL)
+    , detail::HttpAcceptContOptions(opt) // copy these.
+  {
+    SET_HANDLER(&HttpAcceptCont::mainEvent);
+    return;
+  }
+
+  ~HttpAcceptCont()
+  {
+    return;
+  }
+
+  int mainEvent(int event, void *netvc);
+
+private:
+    HttpAcceptCont(const HttpAcceptCont &);
+    HttpAcceptCont & operator =(const HttpAcceptCont &);
+};
+
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpProxyServerMain.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
index 662856a..8988253 100644
--- a/proxy/http/HttpProxyServerMain.cc
+++ b/proxy/http/HttpProxyServerMain.cc
@@ -26,7 +26,7 @@
 #include "Main.h"
 #include "Error.h"
 #include "HttpConfig.h"
-#include "HttpAccept.h"
+#include "HttpAcceptCont.h"
 #include "ReverseProxy.h"
 #include "HttpSessionManager.h"
 #include "HttpUpdateSM.h"
@@ -35,9 +35,11 @@
 #include "HttpTunnel.h"
 #include "Tokenizer.h"
 #include "P_SSLNextProtocolAccept.h"
+#include "P_ProtocolAcceptCont.h"
+#include "P_SpdyAcceptCont.h"
 
-HttpAccept *plugin_http_accept = NULL;
-HttpAccept *plugin_http_transparent_accept = 0;
+HttpAcceptCont *plugin_http_accept = NULL;
+HttpAcceptCont *plugin_http_transparent_accept = 0;
 
 static SLL<SSLNextProtocolAccept> ssl_plugin_acceptors;
 static ink_mutex ssl_plugin_mutex = PTHREAD_MUTEX_INITIALIZER;
@@ -135,9 +137,10 @@ static void
 MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned nthreads)
 {
   NetProcessor::AcceptOptions& net_opt = acceptor._net_opt;
-  HttpAccept::Options         accept_opt;
+  HttpAcceptCont::Options         accept_opt;
 
   net_opt = make_net_accept_options(port, nthreads);
+  net_opt.create_default_NetAccept = false;
   REC_ReadConfigInteger(net_opt.recv_bufsize, "proxy.config.net.sock_recv_buffer_size_in");
   REC_ReadConfigInteger(net_opt.send_bufsize, "proxy.config.net.sock_send_buffer_size_in");
   REC_ReadConfigInteger(net_opt.packet_mark, "proxy.config.net.sock_packet_mark_in");
@@ -160,21 +163,24 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
     accept_opt.outbound_ip6 = HttpConfig::m_master.outbound_ip6;
   }
 
-  if (port.isSSL()) {
-    HttpAccept * accept = NEW(new HttpAccept(accept_opt));
-    SSLNextProtocolAccept * ssl = NEW(new SSLNextProtocolAccept(accept));
+  HttpAcceptCont *http = NEW(new HttpAcceptCont(accept_opt));
+  SpdyAcceptCont *spdy = NEW(new SpdyAcceptCont(http));
+  SSLNextProtocolAccept *ssl = NEW(new SSLNextProtocolAccept(http));
+  ProtocolAcceptCont *proto = NEW(new ProtocolAcceptCont());
+
+  proto->registerEndpoint(TS_PROTO_TLS, ssl);
+  proto->registerEndpoint(TS_PROTO_HTTP, http);
+  proto->registerEndpoint(TS_PROTO_SPDY, spdy);
 
+  if (port.isSSL()) {
     // ALPN selects the first server-offered protocol, so make sure that we offer HTTP/1.1 first.
-    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, accept);
-    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, accept);
+    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, http);
+    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, http);
 
     ink_scoped_mutex lock(ssl_plugin_mutex);
     ssl_plugin_acceptors.push(ssl);
-
-    acceptor._accept = ssl;
-  } else {
-    acceptor._accept = NEW(new HttpAccept(accept_opt));
   }
+  acceptor._accept = proto;
 }
 
 /** Set up all the accepts and sockets.
@@ -197,14 +203,14 @@ init_HttpProxyServer(int n_accept_threads)
   //   port but without going through the operating system
   //
   if (plugin_http_accept == NULL) {
-    plugin_http_accept = NEW(new HttpAccept);
+    plugin_http_accept = NEW(new HttpAcceptCont);
     plugin_http_accept->mutex = new_ProxyMutex();
   }
   // Same as plugin_http_accept except outbound transparent.
   if (! plugin_http_transparent_accept) {
-    HttpAccept::Options ha_opt;
+    HttpAcceptCont::Options ha_opt;
     ha_opt.setOutboundTransparent(true);
-    plugin_http_transparent_accept = NEW(new HttpAccept(ha_opt));
+    plugin_http_transparent_accept = NEW(new HttpAcceptCont(ha_opt));
     plugin_http_transparent_accept->mutex = new_ProxyMutex();
   }
   ink_mutex_init(&ssl_plugin_mutex, "SSL Acceptor List");
@@ -262,7 +268,7 @@ void
 start_HttpProxyServerBackDoor(int port, int accept_threads)
 {
   NetProcessor::AcceptOptions opt;
-  HttpAccept::Options ha_opt;
+  HttpAcceptCont::Options ha_opt;
 
   opt.local_port = port;
   opt.accept_threads = accept_threads;
@@ -271,5 +277,5 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
   opt.backdoor = true;
   
   // The backdoor only binds the loopback interface
-  netProcessor.main_accept(NEW(new HttpAccept(ha_opt)), NO_FD, opt);
+  netProcessor.main_accept(NEW(new HttpAcceptCont(ha_opt)), NO_FD, opt);
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/Makefile.am
----------------------------------------------------------------------
diff --git a/proxy/http/Makefile.am b/proxy/http/Makefile.am
index c413ed7..a71d287 100644
--- a/proxy/http/Makefile.am
+++ b/proxy/http/Makefile.am
@@ -31,14 +31,15 @@ AM_CPPFLAGS = \
   -I$(top_srcdir)/proxy/hdrs \
   -I$(top_srcdir)/proxy/shared \
   -I$(top_srcdir)/proxy/http/remap \
-  -I$(top_srcdir)/proxy/logging
+  -I$(top_srcdir)/proxy/logging \
+  -I$(top_srcdir)/proxy/spdy
 
 noinst_HEADERS = HttpProxyServerMain.h
 noinst_LIBRARIES = libhttp.a
 
 libhttp_a_SOURCES = \
-  HttpAccept.cc \
-  HttpAccept.h \
+  HttpAcceptCont.cc \
+  HttpAcceptCont.h \
   HttpBodyFactory.cc \
   HttpBodyFactory.h \
   HttpCacheSM.cc \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/Makefile.am
----------------------------------------------------------------------
diff --git a/proxy/spdy/Makefile.am b/proxy/spdy/Makefile.am
new file mode 100644
index 0000000..63911e1
--- /dev/null
+++ b/proxy/spdy/Makefile.am
@@ -0,0 +1,38 @@
+# Makefile.am for the traffic/iocore/net hierarchy
+#
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+AM_CPPFLAGS = \
+  $(iocore_include_dirs) \
+  -I$(top_srcdir)/lib \
+  -I$(top_srcdir)/lib/records \
+  -I$(top_srcdir)/lib/ts \
+  -I$(top_srcdir)/proxy \
+  -I$(top_srcdir)/proxy/api \
+  -I$(top_srcdir)/proxy/hdrs \
+  -I$(top_srcdir)/proxy/shared \
+  -I$(top_srcdir)/proxy/spdy \
+  -I$(top_srcdir)/mgmt \
+  -I$(top_srcdir)/mgmt/preparse \
+  -I$(top_srcdir)/mgmt/utils \
+  @SPDYLAY_CFLAGS@
+
+noinst_LIBRARIES = libspdy.a
+
+libspdy_a_SOURCES = \
+  P_SpdyAcceptCont.h \
+  SpdyAcceptCont.cc

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/P_SpdyAcceptCont.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/P_SpdyAcceptCont.h b/proxy/spdy/P_SpdyAcceptCont.h
new file mode 100644
index 0000000..88aea02
--- /dev/null
+++ b/proxy/spdy/P_SpdyAcceptCont.h
@@ -0,0 +1,46 @@
+/** @file
+
+  SpdyAcceptCont
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#ifndef P_SpdyAcceptCont_H_
+#define P_SpdyAcceptCont_H_
+
+#include "P_Net.h"
+#include "P_EventSystem.h"
+#include "P_UnixNet.h"
+#include "I_IOBuffer.h"
+
+class SpdyAcceptCont: public AcceptCont
+{
+public:
+  SpdyAcceptCont(Continuation *ep);
+  ~SpdyAcceptCont() {}
+
+private:
+  int mainEvent(int event, void *netvc);
+  SpdyAcceptCont(const SpdyAcceptCont &); // disabled
+  SpdyAcceptCont& operator =(const SpdyAcceptCont&); // disabled
+
+  Continuation *endpoint;
+};
+
+#endif /* P_SpdyAcceptCont_H_ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/SpdyAcceptCont.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyAcceptCont.cc b/proxy/spdy/SpdyAcceptCont.cc
new file mode 100644
index 0000000..b9efc80
--- /dev/null
+++ b/proxy/spdy/SpdyAcceptCont.cc
@@ -0,0 +1,37 @@
+/** @file
+
+  SpdyNetAccept
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "P_SpdyAcceptCont.h"
+
+SpdyAcceptCont::SpdyAcceptCont(Continuation *ep)
+    : AcceptCont(new_ProxyMutex()), endpoint(ep)
+{
+  SET_HANDLER(&SpdyAcceptCont::mainEvent);
+}
+
+int
+SpdyAcceptCont::mainEvent(int event, void *netvc)
+{
+  printf("spdy accepted\n");
+  return 0;
+}


Re: [2/3] git commit: TS-2431: Preparation of SPDY protocol

Posted by Yunkai Zhang <yu...@gmail.com>.
On Fri, Mar 28, 2014 at 7:50 AM, James Peach <jp...@apache.org> wrote:

> I just started reviewing this, a few high level comments ...
>
> I don't like the renaming of *Accept to *AcceptCont. This results in names
> like HttpAcceptCont::Options, which is pretty ugly.
>

I was really confused by the relationship between *NetAccpet* and
*HttpAccept*.

I think one of them should pick a better name, do you have any idea?


>
> I think we need to separate SPDY probe from the UnixNetAccept better. A
> good goal to have is for iocore to know nothing at all about SPDY. The way
> I imagine this is for the generic protocol acceptor to take a table of
> probe functions or continuations that can detect the next protocol.
> Enabling detection should be a per-port option.
>

Good idea! We can improve it later.


>
> I'll work on this and other SPDY issues over the coming weeks, hopefully
> we can keep improving it :)
>
> On Mar 20, 2014, at 10:32 AM, yunkai@apache.org wrote:
>
> > TS-2431: Preparation of SPDY protocol
> >
> > *) Create basic data structures for SPDY protocal, such as:
> >   SpdyAcceptCont.
> >
> > *) SPDY will share the same port number with HTTP protocol, ATS can
> >   recognize them by detecting the first byte of client request.
> >
> > *) HttpAccept looks like a subclass of NetAccept. To avoid confusing,
> >   rename HttpAccept to HttpAcceptCont.
> >
> > Signed-off-by: Yunkai Zhang <qi...@taobao.com>
> >
> >
> > Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> > Commit:
> http://git-wip-us.apache.org/repos/asf/trafficserver/commit/f90f3a48
> > Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/f90f3a48
> > Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/f90f3a48
> >
> > Branch: refs/heads/master
> > Commit: f90f3a488323ccb8b98c5571ef6cf035f1be51a9
> > Parents: f1a005e
> > Author: Yunkai Zhang <qi...@taobao.com>
> > Authored: Wed Dec 11 21:21:53 2013 +0800
> > Committer: Yunkai Zhang <qi...@taobao.com>
> > Committed: Fri Mar 21 01:29:21 2014 +0800
> >
> > ----------------------------------------------------------------------
> > configure.ac                         |   1 +
> > iocore/net/AcceptCont.cc             |  41 +++++++
> > iocore/net/I_AcceptCont.h            |  48 ++++++++
> > iocore/net/I_Net.h                   |   1 +
> > iocore/net/I_NetProcessor.h          |   1 +
> > iocore/net/Makefile.am               |   8 ++
> > iocore/net/P_Net.h                   |   1 +
> > iocore/net/P_NetAccept.h             |   2 +
> > iocore/net/P_ProtocolAcceptCont.h    |  50 ++++++++
> > iocore/net/P_ProtocolNetAccept.h     |  62 ++++++++++
> > iocore/net/P_SSLNetAccept.h          |   1 +
> > iocore/net/P_SSLNextProtocolAccept.h |   2 +-
> > iocore/net/P_SSLNextProtocolSet.h    |   2 +-
> > iocore/net/P_UnixNetVConnection.h    |   9 ++
> > iocore/net/ProtocolAcceptCont.cc     |  73 +++++++++++
> > iocore/net/ProtocolNetAccept.cc      |  68 ++++++++++
> > iocore/net/SSLNetAccept.cc           |  16 ++-
> > iocore/net/SSLNetVConnection.cc      |   3 +-
> > iocore/net/SSLNextProtocolAccept.cc  |   2 +-
> > iocore/net/SSLNextProtocolSet.cc     |   5 +-
> > iocore/net/UnixNetAccept.cc          |  69 +++++++----
> > iocore/net/UnixNetProcessor.cc       |  20 ++-
> > iocore/net/UnixNetVConnection.cc     |  95 +++++++++++++-
> > lib/ts/apidefs.h.in                  |   3 +
> > proxy/InkAPI.cc                      |   6 +-
> > proxy/Makefile.am                    |  12 +-
> > proxy/SocksProxy.cc                  |   6 +-
> > proxy/api/ts/InkAPIPrivateIOCore.h   |   4 +
> > proxy/http/HttpAccept.cc             |  98 ---------------
> > proxy/http/HttpAccept.h              | 198 ------------------------------
> > proxy/http/HttpAcceptCont.cc         |  98 +++++++++++++++
> > proxy/http/HttpAcceptCont.h          | 198 ++++++++++++++++++++++++++++++
> > proxy/http/HttpProxyServerMain.cc    |  42 ++++---
> > proxy/http/Makefile.am               |   7 +-
> > proxy/spdy/Makefile.am               |  38 ++++++
> > proxy/spdy/P_SpdyAcceptCont.h        |  46 +++++++
> > proxy/spdy/SpdyAcceptCont.cc         |  37 ++++++
> > 37 files changed, 1004 insertions(+), 369 deletions(-)
> > ----------------------------------------------------------------------
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/configure.ac
> > ----------------------------------------------------------------------
> > diff --git a/configure.ac b/configure.ac
> > index f87674d..796158e 100644
> > --- a/configure.ac
> > +++ b/configure.ac
> > @@ -2010,6 +2010,7 @@ AC_CONFIG_FILES([
> >   proxy/http/Makefile
> >   proxy/http/remap/Makefile
> >   proxy/logging/Makefile
> > +  proxy/spdy/Makefile
> >   rc/Makefile
> >   rc/trafficserver
> >   rc/trafficserver.conf
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/AcceptCont.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/AcceptCont.cc b/iocore/net/AcceptCont.cc
> > new file mode 100644
> > index 0000000..e112230
> > --- /dev/null
> > +++ b/iocore/net/AcceptCont.cc
> > @@ -0,0 +1,41 @@
> > +/** @file
> > +
> > +  AcceptCont
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#include "I_AcceptCont.h"
> > +#include "P_Net.h"
> > +
> > +AcceptCont::AcceptCont(ProxyMutex *amutex)
> > +    : Continuation(amutex)
> > +{
> > +  SET_HANDLER(&AcceptCont::mainEvent);
> > +}
> > +
> > +AcceptCont::~AcceptCont()
> > +{
> > +}
> > +
> > +void *
> > +AcceptCont::createNetAccept()
> > +{
> > +    return (NEW(new NetAccept));
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_AcceptCont.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/I_AcceptCont.h b/iocore/net/I_AcceptCont.h
> > new file mode 100644
> > index 0000000..2d59e1e
> > --- /dev/null
> > +++ b/iocore/net/I_AcceptCont.h
> > @@ -0,0 +1,48 @@
> > +/** @file
> > +
> > +  AcceptCont
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#ifndef I_AcceptCont_H_
> > +#define I_AcceptCont_H_
> > +#include "I_Net.h"
> > +#include "I_VConnection.h"
> > +
> > +class AcceptCont: public Continuation
> > +{
> > +public:
> > +  AcceptCont(ProxyMutex *amutex);
> > +  ~AcceptCont();
> > +
> > +  //
> > +  // Virtual function allows creation of an SSLNetAccept
> > +  // or NetAccept transparent to NetProcessor.
> > +  //
> > +  // This function should return a pointer
> > +  // of NetAccept or its subclass.
> > +  //
> > +  virtual void *createNetAccept();
> > +
> > +private:
> > +  virtual int mainEvent(int event, void * netvc) = 0;
> > +};
> > +
> > +#endif /* I_AcceptCont_H_ */
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_Net.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/I_Net.h b/iocore/net/I_Net.h
> > index 78f3d62..a98b6a3 100644
> > --- a/iocore/net/I_Net.h
> > +++ b/iocore/net/I_Net.h
> > @@ -91,6 +91,7 @@ extern int net_config_poll_timeout;
> >
> > #include "I_NetVConnection.h"
> > #include "I_NetProcessor.h"
> > +#include "I_AcceptCont.h"
> >
> > void ink_net_init(ModuleVersion version);
> > #endif
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_NetProcessor.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/I_NetProcessor.h b/iocore/net/I_NetProcessor.h
> > index 3deae4c..68ce117 100644
> > --- a/iocore/net/I_NetProcessor.h
> > +++ b/iocore/net/I_NetProcessor.h
> > @@ -100,6 +100,7 @@ public:
> >     AcceptOptions() { this->reset(); }
> >     /// Reset all values to defaults.
> >     self& reset();
> > +    bool create_default_NetAccept;
> >   };
> >
> >   /**
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/Makefile.am
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/Makefile.am b/iocore/net/Makefile.am
> > index 4575e9e..efbfa86 100644
> > --- a/iocore/net/Makefile.am
> > +++ b/iocore/net/Makefile.am
> > @@ -52,6 +52,8 @@ libinknet_a_SOURCES = \
> >   I_UDPNet.h \
> >   I_UDPPacket.h \
> >   Inline.cc \
> > +  I_AcceptCont.h \
> > +  AcceptCont.cc \
> >   Net.cc \
> >   NetVConnection.cc \
> >   P_CompletionUtil.h \
> > @@ -61,6 +63,9 @@ libinknet_a_SOURCES = \
> >   P_Net.h \
> >   P_NetAccept.h \
> >   P_NetVConnection.h \
> > +  P_ProtocolNetAccept.h \
> > +  P_ProtocolAcceptCont.h \
> > +  P_Socks.h \
> >   P_SSLCertLookup.h \
> >   P_SSLConfig.h \
> >   P_SSLNetAccept.h \
> > @@ -81,6 +86,9 @@ libinknet_a_SOURCES = \
> >   P_UnixNetVConnection.h \
> >   P_UnixPollDescriptor.h \
> >   P_UnixUDPConnection.h \
> > +  ProtocolNetAccept.cc \
> > +  ProtocolAcceptCont.cc \
> > +  Socks.cc \
> >   SSLCertLookup.cc \
> >   SSLConfig.cc \
> >   SSLNetAccept.cc \
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_Net.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_Net.h b/iocore/net/P_Net.h
> > index 13ab617..db9acc6 100644
> > --- a/iocore/net/P_Net.h
> > +++ b/iocore/net/P_Net.h
> > @@ -104,6 +104,7 @@ do { \
> > #include "P_NetVCTest.h"
> > #include "P_LibBulkIO.h"
> >
> > +#include "P_ProtocolNetAccept.h"
> > #include "P_SSLNetVConnection.h"
> > #include "P_SSLNetProcessor.h"
> > #include "P_SSLNetAccept.h"
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_NetAccept.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_NetAccept.h b/iocore/net/P_NetAccept.h
> > index 3a82e4b..762a601 100644
> > --- a/iocore/net/P_NetAccept.h
> > +++ b/iocore/net/P_NetAccept.h
> > @@ -106,11 +106,13 @@ struct NetAccept:public Continuation
> >   virtual UnixNetVConnection *allocateThread(EThread *t);
> >   virtual void freeThread(UnixNetVConnection *vc, EThread *t);
> >   virtual UnixNetVConnection *allocateGlobal();
> > +  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection
> &con);
> >   virtual EventType getEtype();
> >
> >   void init_accept_loop(const char *);
> >   virtual void init_accept(EThread * t = NULL);
> >   virtual void init_accept_per_thread();
> > +  virtual NetAccept *clone();
> >   // 0 == success
> >   int do_listen(bool non_blocking, bool transparent = false);
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolAcceptCont.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_ProtocolAcceptCont.h
> b/iocore/net/P_ProtocolAcceptCont.h
> > new file mode 100644
> > index 0000000..7631e5c
> > --- /dev/null
> > +++ b/iocore/net/P_ProtocolAcceptCont.h
> > @@ -0,0 +1,50 @@
> > +/** @file
> > +
> > +  ProtocolAcceptCont
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#ifndef P_ProtocolAcceptCont_H_
> > +#define P_ProtocolAcceptCont_H_
> > +
> > +#include "I_AcceptCont.h"
> > +
> > +class ProtocolAcceptCont: public AcceptCont
> > +{
> > +public:
> > +  ProtocolAcceptCont(): AcceptCont(NULL)
> > +  {
> > +    memset(endpoint, 0, TS_PROTO_MAX * sizeof(AcceptCont *));
> > +    SET_HANDLER(&ProtocolAcceptCont::mainEvent);
> > +  }
> > +  ~ProtocolAcceptCont() {}
> > +
> > +  void *createNetAccept();
> > +  void registerEndpoint(TSProtoType type, Continuation *ep);
> > +
> > +private:
> > +  int mainEvent(int event, void * netvc);
> > +  ProtocolAcceptCont(const ProtocolAcceptCont &); // disabled
> > +  ProtocolAcceptCont& operator =(const ProtocolAcceptCont&); // disabled
> > +
> > +  Continuation *endpoint[TS_PROTO_MAX];
> > +};
> > +
> > +#endif /* P_ProtocolAcceptCont_H_ */
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolNetAccept.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_ProtocolNetAccept.h
> b/iocore/net/P_ProtocolNetAccept.h
> > new file mode 100644
> > index 0000000..ac49571
> > --- /dev/null
> > +++ b/iocore/net/P_ProtocolNetAccept.h
> > @@ -0,0 +1,62 @@
> > +/** @file
> > +
> > +  A brief file description
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> >
> +/****************************************************************************
> > +
> > +  P_ProtocolNetAccept.h
> > +
> > +
> > +   NetAccept is a generalized facility which allows
> > +   Connections of different classes to be accepted either
> > +   from a blockable thread or by adaptive polling.
> > +
> > +   It is used by the NetProcessor and the ClusterProcessor
> > +   and should be considered PRIVATE to processor implementations.
> > +
> > +
> > +
> > +
> ****************************************************************************/
> > +#if !defined (_ProtocolNetAccept_h_)
> > +#define _ProtocolNetAccept_h_
> > +
> > +#include "libts.h"
> > +#include "P_NetAccept.h"
> > +#include "P_Connection.h"
> > +
> > +class Event;
> > +class UnixNetVConnection;
> > +
> > +//
> > +// NetAccept
> > +// Handles accepting connections.
> > +//
> > +struct ProtocolNetAccept: public NetAccept
> > +{
> > +  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection
> &con);
> > +  virtual NetAccept *clone();
> > +
> > +  ProtocolNetAccept() {}
> > +  virtual ~ProtocolNetAccept() {}
> > +
> > +};
> > +#endif
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNetAccept.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_SSLNetAccept.h b/iocore/net/P_SSLNetAccept.h
> > index e9a8f7b..6d963a3 100644
> > --- a/iocore/net/P_SSLNetAccept.h
> > +++ b/iocore/net/P_SSLNetAccept.h
> > @@ -57,6 +57,7 @@ struct SSLNetAccept: public NetAccept
> >   virtual UnixNetVConnection *allocateGlobal();
> >   virtual EventType getEtype();
> >   virtual void init_accept_per_thread();
> > +  virtual NetAccept *clone();
> >
> >   SSLNetAccept()
> >     { };
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolAccept.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_SSLNextProtocolAccept.h
> b/iocore/net/P_SSLNextProtocolAccept.h
> > index c3ee575..342485a 100644
> > --- a/iocore/net/P_SSLNextProtocolAccept.h
> > +++ b/iocore/net/P_SSLNextProtocolAccept.h
> > @@ -31,7 +31,7 @@
> > #include "P_SSLNextProtocolSet.h"
> > #include "I_IOBuffer.h"
> >
> > -class SSLNextProtocolAccept: public Continuation
> > +class SSLNextProtocolAccept: public AcceptCont
> > {
> > public:
> >   SSLNextProtocolAccept(Continuation *);
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolSet.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_SSLNextProtocolSet.h
> b/iocore/net/P_SSLNextProtocolSet.h
> > index e25f50d..d3c37b6 100644
> > --- a/iocore/net/P_SSLNextProtocolSet.h
> > +++ b/iocore/net/P_SSLNextProtocolSet.h
> > @@ -40,7 +40,7 @@ public:
> >   bool advertiseProtocols(const unsigned char ** out, unsigned * len)
> const;
> >
> >   Continuation * findEndpoint(const char *) const;
> > -  Continuation * findEndpoint(const unsigned char *, unsigned,
> TSClientProtoStack *) const;
> > +  Continuation * findEndpoint(const unsigned char *, unsigned,
> TSClientProtoStack *, const char **) const;
> >
> >   struct NextProtocolEndpoint
> >   {
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_UnixNetVConnection.h
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/P_UnixNetVConnection.h
> b/iocore/net/P_UnixNetVConnection.h
> > index abdf4ba..0e4eae8 100644
> > --- a/iocore/net/P_UnixNetVConnection.h
> > +++ b/iocore/net/P_UnixNetVConnection.h
> > @@ -41,6 +41,13 @@ class UnixNetVConnection;
> > class NetHandler;
> > struct PollDescriptor;
> >
> > +enum SpdyProbeState
> > +{
> > +  SPDY_PROBE_STATE_NONE,
> > +  SPDY_PROBE_STATE_BEGIN,
> > +  SPDY_PROBE_STATE_END
> > +};
> > +
> > TS_INLINE void
> > NetVCOptions::reset()
> > {
> > @@ -233,6 +240,8 @@ public:
> >   ink_hrtime submit_time;
> >   OOB_callback *oob_ptr;
> >   bool from_accept_thread;
> > +  SpdyProbeState probe_state;
> > +  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/f90f3a48/iocore/net/ProtocolAcceptCont.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/ProtocolAcceptCont.cc
> b/iocore/net/ProtocolAcceptCont.cc
> > new file mode 100644
> > index 0000000..2e73b77
> > --- /dev/null
> > +++ b/iocore/net/ProtocolAcceptCont.cc
> > @@ -0,0 +1,73 @@
> > +/** @file
> > +
> > +  ProtocolAcceptCont
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#include "P_ProtocolAcceptCont.h"
> > +#include "P_SSLNextProtocolAccept.h"
> > +#include "P_Net.h"
> > +#include "I_Machine.h"
> > +#include "Error.h"
> > +
> > +void *
> > +ProtocolAcceptCont::createNetAccept()
> > +{
> > +  return ((NetAccept *) NEW(new ProtocolNetAccept));
> > +}
> > +
> > +void
> > +ProtocolAcceptCont::registerEndpoint(TSProtoType type, Continuation *ep)
> > +{
> > +  endpoint[type] = ep;
> > +}
> > +
> > +int
> > +ProtocolAcceptCont::mainEvent(int event, void *netvc)
> > +{
> > +  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> > +  ink_release_assert((event == NET_EVENT_ACCEPT) ? (netvc!= 0) : (1));
> > +
> > +  if (event == NET_EVENT_ACCEPT) {
> > +    TSProtoType proto_type;
> > +    UnixNetVConnection *vc = (UnixNetVConnection *)netvc;
> > +
> > +    if (vc->proto_stack & (1u << TS_PROTO_TLS)) {
> > +      proto_type = TS_PROTO_TLS;
> > +    } else if (vc->proto_stack & (1u << TS_PROTO_HTTP)) {
> > +      proto_type = TS_PROTO_HTTP;
> > +    } else if (vc->proto_stack & (1u << TS_PROTO_SPDY)) {
> > +      proto_type = TS_PROTO_SPDY;
> > +    } else {
> > +      Warning("Invalid protocol stack:%x", vc->proto_stack);
> > +      return EVENT_CONT;
> > +    }
> > +
> > +    if (endpoint[proto_type])
> > +      endpoint[proto_type]->handleEvent(NET_EVENT_ACCEPT, netvc);
> > +    else
> > +      Warning("Unregistered protocol type:%x", proto_type);
> > +
> > +    return EVENT_CONT;
> > +  }
> > +
> > +  MachineFatal("Protocol Accept received fatal error: errno = %d",
> -((int)(intptr_t)netvc));
> > +  return EVENT_CONT;
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/ProtocolNetAccept.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/ProtocolNetAccept.cc
> b/iocore/net/ProtocolNetAccept.cc
> > new file mode 100644
> > index 0000000..c4257e2
> > --- /dev/null
> > +++ b/iocore/net/ProtocolNetAccept.cc
> > @@ -0,0 +1,68 @@
> > +/** @file
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#include "ink_config.h"
> > +#include "P_Net.h"
> > +#include "P_ProtocolNetAccept.h"
> > +
> > +UnixNetVConnection *
> > +ProtocolNetAccept::createSuitableVC(EThread *t, Connection &con)
> > +{
> > +  UnixNetVConnection *vc;
> > +
> > +  if (etype == SSLNetProcessor::ET_SSL && etype) {
> > +    // SSL protocol
> > +    if (t)
> > +      vc = (UnixNetVConnection *)THREAD_ALLOC(sslNetVCAllocator, t);
> > +    else
> > +      vc = (UnixNetVConnection *)sslNetVCAllocator.alloc();
> > +    vc->proto_stack = (1u << TS_PROTO_TLS);
> > +  } else {
> > +    if (t)
> > +      vc = THREAD_ALLOC(netVCAllocator, t);
> > +    else
> > +      vc = netVCAllocator.alloc();
> > +
> > +#if TS_HAS_SPDY
> > +    vc->probe_state = SPDY_PROBE_STATE_BEGIN;
> > +#else
> > +    vc->probe_state = SPDY_PROBE_STATE_NONE;
> > +#endif
> > +
> > +    //
> > +    // Protocol stack may be changed by
> > +    // following call of SpdyProbe()
> > +    //
> > +    vc->proto_stack = (1u << TS_PROTO_HTTP);
> > +  }
> > +
> > +  vc->con = con;
> > +  return vc;
> > +}
> > +
> > +NetAccept *
> > +ProtocolNetAccept::clone()
> > +{
> > +  NetAccept *na;
> > +  na = NEW(new ProtocolNetAccept);
> > +  *na = *this;
> > +  return na;
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetAccept.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/SSLNetAccept.cc b/iocore/net/SSLNetAccept.cc
> > index a321f89..6aa5ed9 100644
> > --- a/iocore/net/SSLNetAccept.cc
> > +++ b/iocore/net/SSLNetAccept.cc
> > @@ -70,10 +70,9 @@ SSLNetAccept::init_accept_per_thread()
> >   NetAccept *a = this;
> >   n = eventProcessor.n_threads_for_type[SSLNetProcessor::ET_SSL];
> >   for (i = 0; i < n; i++) {
> > -    if (i < n - 1) {
> > -      a = NEW(new SSLNetAccept);
> > -      *a = *this;
> > -    } else
> > +    if (i < n - 1)
> > +      a = clone();
> > +    else
> >       a = this;
> >     EThread *t = eventProcessor.eventthread[SSLNetProcessor::ET_SSL][i];
> >
> > @@ -84,3 +83,12 @@ SSLNetAccept::init_accept_per_thread()
> >     t->schedule_every(a, period, etype);
> >   }
> > }
> > +
> > +NetAccept *
> > +SSLNetAccept::clone()
> > +{
> > +  NetAccept *na;
> > +  na = NEW(new SSLNetAccept);
> > +  *na = *this;
> > +  return na;
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetVConnection.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/SSLNetVConnection.cc
> b/iocore/net/SSLNetVConnection.cc
> > index b55dcf7..3925de9 100644
> > --- a/iocore/net/SSLNetVConnection.cc
> > +++ b/iocore/net/SSLNetVConnection.cc
> > @@ -578,7 +578,8 @@ 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->proto_stack);
> > +        this->npnEndpoint = this->npnSet->findEndpoint(proto, len,
> &this->proto_stack,
> > +
> &this->selected_next_protocol);
> >         this->npnSet = NULL;
> >
> >         ink_assert(this->npnEndpoint != NULL);
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolAccept.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/SSLNextProtocolAccept.cc
> b/iocore/net/SSLNextProtocolAccept.cc
> > index bfc2f0b..746995b 100644
> > --- a/iocore/net/SSLNextProtocolAccept.cc
> > +++ b/iocore/net/SSLNextProtocolAccept.cc
> > @@ -147,7 +147,7 @@ SSLNextProtocolAccept::unregisterEndpoint(
> > }
> >
> > SSLNextProtocolAccept::SSLNextProtocolAccept(Continuation * ep)
> > -    : Continuation(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
> > +    : AcceptCont(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
> > {
> >   SET_HANDLER(&SSLNextProtocolAccept::mainEvent);
> > }
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolSet.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/SSLNextProtocolSet.cc
> b/iocore/net/SSLNextProtocolSet.cc
> > index 148a6d1..ed0a5e3 100644
> > --- a/iocore/net/SSLNextProtocolSet.cc
> > +++ b/iocore/net/SSLNextProtocolSet.cc
> > @@ -133,7 +133,8 @@ SSLNextProtocolSet::unregisterEndpoint(const char *
> proto, Continuation * ep)
> >
> > Continuation *
> > SSLNextProtocolSet::findEndpoint(const unsigned char * proto, unsigned
> len,
> > -                                 TSClientProtoStack *proto_stack) const
> > +                                 TSClientProtoStack *proto_stack,
> > +                                 const char **selected_protocol) const
> > {
> >   for (const NextProtocolEndpoint * ep = this->endpoints.head;
> >         ep != NULL; ep = this->endpoints.next(ep)) {
> > @@ -141,6 +142,8 @@ SSLNextProtocolSet::findEndpoint(const unsigned char
> * proto, unsigned len,
> >     if (sz == len && memcmp(ep->protocol, proto, len) == 0) {
> >       if (proto_stack)
> >         *proto_stack = ep->proto_stack;
> > +      if (selected_protocol)
> > +        *selected_protocol = ep->protocol;
> >       return ep->endpoint;
> >     }
> >   }
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetAccept.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/UnixNetAccept.cc b/iocore/net/UnixNetAccept.cc
> > index c038a2e..af4d8f9 100644
> > --- a/iocore/net/UnixNetAccept.cc
> > +++ b/iocore/net/UnixNetAccept.cc
> > @@ -223,10 +223,9 @@ NetAccept::init_accept_per_thread()
> >   NetAccept *a;
> >   n = eventProcessor.n_threads_for_type[ET_NET];
> >   for (i = 0; i < n; i++) {
> > -    if (i < n - 1) {
> > -      a = NEW(new NetAccept);
> > -      *a = *this;
> > -    } else
> > +    if (i < n - 1)
> > +      a = clone();
> > +    else
> >       a = this;
> >     EThread *t = eventProcessor.eventthread[ET_NET][i];
> >     PollDescriptor *pd = get_PollDescriptor(t);
> > @@ -237,6 +236,14 @@ NetAccept::init_accept_per_thread()
> >   }
> > }
> >
> > +NetAccept *
> > +NetAccept::clone()
> > +{
> > +  NetAccept *na;
> > +  na = NEW(new NetAccept);
> > +  *na = *this;
> > +  return na;
> > +}
> >
> > int
> > NetAccept::do_listen(bool non_blocking, bool transparent)
> > @@ -264,6 +271,20 @@ NetAccept::do_listen(bool non_blocking, bool
> transparent)
> >   return res;
> > }
> >
> > +UnixNetVConnection *
> > +NetAccept::createSuitableVC(EThread *t, Connection &con)
> > +{
> > +  UnixNetVConnection *vc;
> > +
> > +  if (t)
> > +    vc = allocateThread(t);
> > +  else
> > +    vc = allocateGlobal();
> > +
> > +  vc->con = con;
> > +
> > +  return vc;
> > +}
> >
> > int
> > NetAccept::do_blocking_accept(EThread * t)
> > @@ -271,18 +292,11 @@ NetAccept::do_blocking_accept(EThread * t)
> >   int res = 0;
> >   int loop = accept_till_done;
> >   UnixNetVConnection *vc = NULL;
> > +  Connection con;
> >
> >   //do-while for accepting all the connections
> >   //added by YTS Team, yamsat
> >   do {
> > -    vc = (UnixNetVConnection *)alloc_cache;
> > -    if (likely(!vc)) {
> > -      //vc = allocateThread(t);
> > -      vc = allocateGlobal(); // Bypass proxy / thread allocator
> > -      vc->from_accept_thread = true;
> > -      vc->id = net_next_connection_number();
> > -      alloc_cache = vc;
> > -    }
> >     ink_hrtime now = ink_get_hrtime();
> >
> >     // Throttle accepts
> > @@ -297,7 +311,7 @@ NetAccept::do_blocking_accept(EThread * t)
> >       now = ink_get_hrtime();
> >     }
> >
> > -    if ((res = server.accept(&vc->con)) < 0) {
> > +    if ((res = server.accept(&con)) < 0) {
> >     Lerror:
> >       int seriousness = accept_error_seriousness(res);
> >       if (seriousness >= 0) {   // not so bad
> > @@ -314,16 +328,23 @@ NetAccept::do_blocking_accept(EThread * t)
> >       }
> >       return -1;
> >     }
> > -    check_emergency_throttle(vc->con);
> > +
> > +    // Use 'NULL' to Bypass thread allocator
> > +    vc = createSuitableVC(NULL, con);
> > +    if (!vc)
> > +      return -1;
> > +    vc->from_accept_thread = true;
> > +    vc->id = net_next_connection_number();
> >     alloc_cache = NULL;
> >
> > +    check_emergency_throttle(con);
> > +
> >     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
> >     vc->submit_time = now;
> >     ats_ip_copy(&vc->server_addr, &vc->con.addr);
> >     vc->set_is_transparent(server.f_inbound_transparent);
> >     vc->mutex = new_ProxyMutex();
> >     vc->action_ = *action_;
> > -    vc->proto_stack = (1u << TS_PROTO_HTTP);
> >     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) &
> UnixNetVConnection::acceptEvent);
> >     //eventProcessor.schedule_imm(vc, getEtype());
> >     eventProcessor.schedule_imm_signal(vc, getEtype());
> > @@ -384,6 +405,7 @@ NetAccept::acceptFastEvent(int event, void *ep)
> >   (void) event;
> >   (void) e;
> >   int bufsz, res;
> > +  Connection con;
> >
> >   PollDescriptor *pd = get_PollDescriptor(e->ethread);
> >   UnixNetVConnection *vc = NULL;
> > @@ -394,10 +416,10 @@ NetAccept::acceptFastEvent(int event, void *ep)
> >       ifd = -1;
> >       return EVENT_CONT;
> >     }
> > -    vc = allocateThread(e->ethread);
> >
> > -    socklen_t sz = sizeof(vc->con.addr);
> > -    int fd = socketManager.accept(server.fd, &vc->con.addr.sa, &sz);
> > +    socklen_t sz = sizeof(con.addr);
> > +    int fd = socketManager.accept(server.fd, &con.addr.sa, &sz);
> > +    con.fd = fd;
> >
> >     if (likely(fd >= 0)) {
> >       Debug("iocore_net", "accepted a new socket: %d", fd);
> > @@ -443,6 +465,11 @@ NetAccept::acceptFastEvent(int event, void *ep)
> >       do {
> >         res = safe_nonblocking(fd);
> >       } while (res < 0 && (errno == EAGAIN || errno == EINTR));
> > +
> > +      vc = createSuitableVC(e->ethread, con);
> > +      if (!vc)
> > +        goto Ldone;
> > +
> >     } else {
> >       res = fd;
> >     }
> > @@ -453,20 +480,15 @@ NetAccept::acceptFastEvent(int event, void *ep)
> >           || res == -EPIPE
> > #endif
> >         ) {
> > -        ink_assert(vc->con.fd == NO_FD);
> > -        ink_assert(!vc->link.next && !vc->link.prev);
> > -        freeThread(vc, e->ethread);
> >         goto Ldone;
> >       } else if (accept_error_seriousness(res) >= 0) {
> >         check_transient_accept_error(res);
> > -        freeThread(vc, e->ethread);
> >         goto Ldone;
> >       }
> >       if (!action_->cancelled)
> >         action_->continuation->handleEvent(EVENT_ERROR, (void
> *)(intptr_t)res);
> >       goto Lerror;
> >     }
> > -    vc->con.fd = fd;
> >
> >     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
> >     vc->id = net_next_connection_number();
> > @@ -478,7 +500,6 @@ NetAccept::acceptFastEvent(int event, void *ep)
> >     vc->thread = e->ethread;
> >
> >     vc->nh = get_NetHandler(e->ethread);
> > -    vc->proto_stack = (1u << TS_PROTO_HTTP);
> >
> >     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) &
> UnixNetVConnection::mainEvent);
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetProcessor.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/UnixNetProcessor.cc
> b/iocore/net/UnixNetProcessor.cc
> > index eaae439..ce59bbe 100644
> > --- a/iocore/net/UnixNetProcessor.cc
> > +++ b/iocore/net/UnixNetProcessor.cc
> > @@ -46,6 +46,7 @@ NetProcessor::AcceptOptions::reset()
> >   packet_mark = 0;
> >   packet_tos = 0;
> >   f_inbound_transparent = false;
> > +  create_default_NetAccept = true;
> >   return *this;
> > }
> >
> > @@ -84,16 +85,22 @@ NetProcessor::main_accept(Continuation *cont, SOCKET
> fd, AcceptOptions const& op
> > Action *
> > UnixNetProcessor::accept_internal(Continuation *cont, int fd,
> AcceptOptions const& opt)
> > {
> > -  EventType et = opt.etype; // setEtype requires non-const ref.
> > -  NetAccept *na = createNetAccept();
> > +  EventType upgraded_etype = opt.etype; // setEtype requires non-const
> ref.
> > +  AcceptCont *acceptCont = static_cast<AcceptCont *>(cont);
> >   EThread *thread = this_ethread();
> >   ProxyMutex *mutex = thread->mutex;
> >   int accept_threads = opt.accept_threads; // might be changed.
> >   IpEndpoint accept_ip; // local binding address.
> >   char thr_name[MAX_THREAD_NAME_LENGTH];
> >
> > +  NetAccept *na;
> > +  if (opt.create_default_NetAccept)
> > +    na = createNetAccept();
> > +  else
> > +    na = (NetAccept *)acceptCont->createNetAccept();
> > +
> >   // Potentially upgrade to SSL.
> > -  upgradeEtype(et);
> > +  upgradeEtype(upgraded_etype);
> >
> >   // Fill in accept thread from configuration if necessary.
> >   if (opt.accept_threads < 0) {
> > @@ -137,7 +144,7 @@ UnixNetProcessor::accept_internal(Continuation
> *cont, int fd, AcceptOptions cons
> >   na->sockopt_flags = opt.sockopt_flags;
> >   na->packet_mark = opt.packet_mark;
> >   na->packet_tos = opt.packet_tos;
> > -  na->etype = opt.etype;
> > +  na->etype = upgraded_etype;
> >   na->backdoor = opt.backdoor;
> >   if (na->callback_on_open)
> >     na->mutex = cont->mutex;
> > @@ -147,7 +154,10 @@ UnixNetProcessor::accept_internal(Continuation
> *cont, int fd, AcceptOptions cons
> >         NetAccept *a;
> >
> >         for (int i=1; i < accept_threads; ++i) {
> > -          a = createNetAccept();
> > +          if (opt.create_default_NetAccept)
> > +            a = createNetAccept();
> > +          else
> > +            a = (NetAccept *)acceptCont->createNetAccept();
> >           *a = *na;
> >           snprintf(thr_name, MAX_THREAD_NAME_LENGTH, "[ACCEPT %d:%d]",
> i-1, ats_ip_port_host_order(&accept_ip));
> >           a->init_accept_loop(thr_name);
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetVConnection.cc
> > ----------------------------------------------------------------------
> > diff --git a/iocore/net/UnixNetVConnection.cc
> b/iocore/net/UnixNetVConnection.cc
> > index bf1dc0a..8a919c5 100644
> > --- a/iocore/net/UnixNetVConnection.cc
> > +++ b/iocore/net/UnixNetVConnection.cc
> > @@ -38,6 +38,84 @@ typedef struct iovec IOVec;
> > #define NET_MAX_IOV UIO_MAXIOV
> > #endif
> >
> > +struct SpdyProbeCont:public Continuation
> > +{
> > +  MIOBuffer buf;
> > +  unsigned char data;
> > +  SpdyProbeCont(): data(0)
> > +  {
> > +    SET_HANDLER(&SpdyProbeCont::mainEvent);
> > +  }
> > +
> > +  int mainEvent(int event, void *e);
> > +};
> > +
> > +static ClassAllocator<SpdyProbeCont>
> spdyProberContAllocator("spdyProberContAllocator");
> > +
> > +SpdyProbeCont *
> > +new_SpdyProbeCont(UnixNetVConnection *vc)
> > +{
> > +  SpdyProbeCont *c = spdyProberContAllocator.alloc();
> > +  c->buf.clear();
> > +  c->buf.set(&c->data, sizeof c->data);
> > +  c->buf._writer->fill(-(sizeof c->data));
> > +  c->mutex = vc->mutex;
> > +  return c;
> > +}
> > +void
> > +free_SpdyProbeCont(SpdyProbeCont *c)
> > +{
> > +  c->mutex.clear();
> > +  c->buf.clear();
> > +  spdyProberContAllocator.free(c);
> > +}
> > +
> > +inline int
> > +SpdyProbeCont::mainEvent(int event, void *e) {
> > +  UnixNetVConnection *vc = (UnixNetVConnection *) ((VIO *)
> e)->vc_server;
> > +  vc->probe_state = SPDY_PROBE_STATE_END;
> > +
> > +  switch (event) {
> > +  case VC_EVENT_EOS:
> > +  case VC_EVENT_ERROR:
> > +  case VC_EVENT_INACTIVITY_TIMEOUT:
> > +  case VC_EVENT_ACTIVE_TIMEOUT:
> > +    vc->do_io_close();
> > +    free_SpdyProbeCont(this);
> > +    return EVENT_DONE;
> > +  case VC_EVENT_READ_COMPLETE:
> > +    if ((data & 0x80) != 0) {
> > +      //
> > +      // SPDY Request
> > +      //
> > +      free_SpdyProbeCont(this);
> > +      vc->proto_stack = (1u << TS_PROTO_SPDY);
> > +      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
> > +      return EVENT_DONE;
> > +    } else {
> > +      //
> > +      // HTTP Request
> > +      //
> > +      free_SpdyProbeCont(this);
> > +      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
> > +      return EVENT_DONE;
> > +    }
> > +  default:
> > +    ink_release_assert(!"unexpected event");
> > +  }
> > +  return EVENT_CONT;
> > +}
> > +
> > +int SpdyProbeStart(UnixNetVConnection *vc)
> > +{
> > +  SpdyProbeCont *spdyProbeCont= new_SpdyProbeCont(vc);
> > +  //
> > +  // TODO: make it configurable
> > +  //
> > +  vc->set_inactivity_timeout(HRTIME_SECONDS(30));
> > +  vc->do_io_read(spdyProbeCont, 1, &spdyProbeCont->buf);
> > +  return EVENT_CONT;
> > +}
> > // Global
> > ClassAllocator<UnixNetVConnection> netVCAllocator("netVCAllocator");
> >
> > @@ -259,8 +337,12 @@ read_from_net(NetHandler *nh, UnixNetVConnection
> *vc, EThread *thread)
> >         }
> >         b = b->next;
> >       }
> > +      ink_assert(vc->probe_state != SPDY_PROBE_STATE_BEGIN || niov ==
> 1);
> >       if (niov == 1) {
> > -        r = socketManager.read(vc->con.fd, tiovec[0].iov_base,
> tiovec[0].iov_len);
> > +        if (vc->probe_state == SPDY_PROBE_STATE_BEGIN) {
> > +          r = recv(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len,
> MSG_PEEK);
> > +        } else
> > +          r = socketManager.read(vc->con.fd, tiovec[0].iov_base,
> tiovec[0].iov_len);
> >       } else {
> >         r = socketManager.readv(vc->con.fd, &tiovec[0], niov);
> >       }
> > @@ -801,7 +883,8 @@ UnixNetVConnection::UnixNetVConnection()
> > #endif
> >     active_timeout(NULL), nh(NULL),
> >     id(0), flags(0), recursion(0), submit_time(0), oob_ptr(0),
> > -    from_accept_thread(false)
> > +    from_accept_thread(false), probe_state(SPDY_PROBE_STATE_NONE),
> > +    selected_next_protocol(NULL)
> > {
> >   memset(&local_addr, 0, sizeof local_addr);
> >   memset(&server_addr, 0, sizeof server_addr);
> > @@ -988,7 +1071,13 @@ UnixNetVConnection::acceptEvent(int event, Event
> *e)
> >     UnixNetVConnection::set_inactivity_timeout(inactivity_timeout_in);
> >   if (active_timeout_in)
> >     UnixNetVConnection::set_active_timeout(active_timeout_in);
> > -  action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
> > +  if (probe_state == SPDY_PROBE_STATE_NONE)
> > +    action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
> > +  else {
> > +    ink_assert(probe_state == SPDY_PROBE_STATE_BEGIN);
> > +    SpdyProbeStart(this);
> > +  }
> > +
> >   return EVENT_DONE;
> > }
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/lib/ts/apidefs.h.in
> > ----------------------------------------------------------------------
> > diff --git a/lib/ts/apidefs.h.in b/lib/ts/apidefs.h.in
> > index 86a650f..0356921 100644
> > --- a/lib/ts/apidefs.h.in
> > +++ b/lib/ts/apidefs.h.in
> > @@ -117,6 +117,9 @@ extern "C"
> >     TS_PROTO_SPDY = 13,
> >     TS_PROTO_RTMP = 14,
> >     TS_PROTO_WBSK = 15, /* WebSocket */
> > +
> > +    /* MAX value of TSProtoType  */
> > +    TS_PROTO_MAX
> >   } TSProtoType;
> >
> >   typedef uint32_t TSClientProtoStack;
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/InkAPI.cc
> > ----------------------------------------------------------------------
> > diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
> > index 00a78e1..682f889 100644
> > --- a/proxy/InkAPI.cc
> > +++ b/proxy/InkAPI.cc
> > @@ -51,7 +51,7 @@
> > #include "PluginVC.h"
> > #include "api/ts/experimental.h"
> > #include "ICP.h"
> > -#include "HttpAccept.h"
> > +#include "HttpAcceptCont.h"
> > #include "PluginVC.h"
> > #include "FetchSM.h"
> > #include "HttpDebugNames.h"
> > @@ -6082,8 +6082,8 @@ TSHttpAltInfoQualitySet(TSHttpAltInfo infop, float
> quality)
> >   info->m_qvalue = quality;
> > }
> >
> > -extern HttpAccept *plugin_http_accept;
> > -extern HttpAccept *plugin_http_transparent_accept;
> > +extern HttpAcceptCont *plugin_http_accept;
> > +extern HttpAcceptCont *plugin_http_transparent_accept;
> >
> > TSVConn
> > TSHttpConnect(sockaddr const* addr)
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/Makefile.am
> > ----------------------------------------------------------------------
> > diff --git a/proxy/Makefile.am b/proxy/Makefile.am
> > index 8ef6104..6dfc816 100644
> > --- a/proxy/Makefile.am
> > +++ b/proxy/Makefile.am
> > @@ -17,7 +17,7 @@
> > #  limitations under the License.
> >
> > # Note that hdrs is targeted from ../Makefile.am
> > -SUBDIRS = congest http logging config
> > +SUBDIRS = congest http spdy logging config
> > noinst_LIBRARIES =
> > bin_PROGRAMS = \
> >   traffic_server \
> > @@ -52,11 +52,10 @@ noinst_HEADERS = \
> >   ConfigParse.h \
> >   Show.h
> >
> > -
> > if STATIC_LIBTS
> > -   which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
> > +    which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
> > else
> > -   which_libts = $(top_builddir)/lib/ts/libtsutil.la
> > +    which_libts = $(top_builddir)/lib/ts/libtsutil.la
> > endif
> >
> > EXTRA_DIST = InkAPITestTool.cc example_alarm_bin.sh example_prep.sh
> > @@ -121,6 +120,7 @@ endif
> > traffic_server_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
> > traffic_server_LDADD = \
> >   http/libhttp.a \
> > +  spdy/libspdy.a \
> >   http/remap/libhttp_remap.a \
> >   congest/libCongestionControl.a \
> >   logging/liblogging.a \
> > @@ -157,6 +157,7 @@ traffic_server_LDADD = \
> >   @LIBZ@ \
> >   @LIBLZMA@ \
> >   @LIBPROFILER@ \
> > +  @SPDYLAY_LIBS@ \
> >   -lm
> >
> > if BUILD_LUA_SUPPORT
> > @@ -223,6 +224,7 @@ traffic_sac_SOURCES = \
> > traffic_sac_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
> > traffic_sac_LDADD = \
> >   http/libhttp.a \
> > +  spdy/libspdy.a \
> >   shared/libdiagsconfig.a \
> >   http/remap/libhttp_remap.a \
> >   congest/libCongestionControl.a \
> > @@ -247,7 +249,7 @@ traffic_sac_LDADD = \
> >   $(top_builddir)/lib/records/librecprocess.a \
> >   $(top_builddir)/lib/ts/libtsutil.la \
> >   @LIBRESOLV@ @LIBPCRE@ @OPENSSL_LIBS@ @LIBTCL@ @hwloc_LIBS@ \
> > -  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@ -lm
> > +  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@@SPDYLAY_LIBS@-lm
> >
> > if BUILD_TESTS
> >   traffic_sac_SOURCES += RegressionSM.cc
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/SocksProxy.cc
> > ----------------------------------------------------------------------
> > diff --git a/proxy/SocksProxy.cc b/proxy/SocksProxy.cc
> > index e447f12..d855689 100644
> > --- a/proxy/SocksProxy.cc
> > +++ b/proxy/SocksProxy.cc
> > @@ -31,7 +31,7 @@
> > #include "StatSystem.h"
> > #include "P_Net.h"
> > #include "I_OneWayTunnel.h"
> > -#include "HttpAccept.h"
> > +#include "HttpAcceptCont.h"
> >
> > enum
> > {
> > @@ -145,14 +145,14 @@ SocksProxy::mainEvent(int event, void *data)
> >
> >     switch (state) {
> >     case HTTP_REQ:{
> > -      HttpAccept::Options ha_opt;
> > +      HttpAcceptCont::Options ha_opt;
> >       //This is a WRITE_COMPLETE. vio->nbytes == vio->ndone is true
> >
> >       SOCKSPROXY_INC_STAT(socksproxy_http_connections_stat);
> >       Debug("SocksProxy", "Handing over the HTTP request\n");
> >
> >       ha_opt.transport_type = clientVC->attributes;
> > -      HttpAccept http_accept(ha_opt);
> > +      HttpAcceptCont http_accept(ha_opt);
> >       http_accept.mainEvent(NET_EVENT_ACCEPT, clientVC);
> >       state = ALL_DONE;
> >       break;
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/api/ts/InkAPIPrivateIOCore.h
> > ----------------------------------------------------------------------
> > diff --git a/proxy/api/ts/InkAPIPrivateIOCore.h
> b/proxy/api/ts/InkAPIPrivateIOCore.h
> > index 26e3fac..97a9bcd 100644
> > --- a/proxy/api/ts/InkAPIPrivateIOCore.h
> > +++ b/proxy/api/ts/InkAPIPrivateIOCore.h
> > @@ -51,6 +51,10 @@ public:
> >
> >   void handle_event_count(int event);
> >   int handle_event(int event, void *edata);
> > +  virtual void *createNetAccept()
> > +  {
> > +    return (NEW(new NetAccept));
> > +  }
> >
> > public:
> >   void *mdata;
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.cc
> > ----------------------------------------------------------------------
> > diff --git a/proxy/http/HttpAccept.cc b/proxy/http/HttpAccept.cc
> > deleted file mode 100644
> > index 9fb0066..0000000
> > --- a/proxy/http/HttpAccept.cc
> > +++ /dev/null
> > @@ -1,98 +0,0 @@
> > -/** @file
> > -
> > -  A brief file description
> > -
> > -  @section license License
> > -
> > -  Licensed to the Apache Software Foundation (ASF) under one
> > -  or more contributor license agreements.  See the NOTICE file
> > -  distributed with this work for additional information
> > -  regarding copyright ownership.  The ASF licenses this file
> > -  to you under the Apache License, Version 2.0 (the
> > -  "License"); you may not use this file except in compliance
> > -  with the License.  You may obtain a copy of the License at
> > -
> > -      http://www.apache.org/licenses/LICENSE-2.0
> > -
> > -  Unless required by applicable law or agreed to in writing, software
> > -  distributed under the License is distributed on an "AS IS" BASIS,
> > -  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > -  See the License for the specific language governing permissions and
> > -  limitations under the License.
> > - */
> > -
> > -#include "HttpAccept.h"
> > -#include "IPAllow.h"
> > -#include "HttpClientSession.h"
> > -#include "I_Machine.h"
> > -#include "Error.h"
> > -
> > -int
> > -HttpAccept::mainEvent(int event, void *data)
> > -{
> > -  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> > -  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
> > -
> > -  if (event == NET_EVENT_ACCEPT) {
> > -    ////////////////////////////////////////////////////
> > -    // if client address forbidden, close immediately //
> > -    ////////////////////////////////////////////////////
> > -    NetVConnection *netvc = static_cast<NetVConnection *>(data);
> > -    sockaddr const* client_ip = netvc->get_remote_addr();
> > -    uint32_t acl_method_mask = 0;
> > -    ip_port_text_buffer ipb;
> > -    IpAllow::scoped_config ipallow;
> > -
> > -    // The backdoor port is now only bound to "localhost", so no
> > -    // reason to check for if it's incoming from "localhost" or not.
> > -    if (backdoor) {
> > -      acl_method_mask = IpAllow::AllMethodMask();
> > -    } else if (ipallow && ((acl_method_mask =
> ipallow->match(client_ip)) == 0)) {
> > -      Warning("client '%s' prohibited by ip-allow policy",
> ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
> > -      netvc->do_io_close();
> > -
> > -      return VC_EVENT_CONT;
> > -    }
> > -
> > -    netvc->attributes = transport_type;
> > -
> > -    if (is_debug_tag_set("http_seq"))
> > -      Debug("http_seq", "[HttpAccept:mainEvent %p] accepted connection
> from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb,
> sizeof(ipb)), netvc->attributes);
> > -
> > -    HttpClientSession *new_session =
> THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
> > -
> > -   // copy over session related data.
> > -    new_session->f_outbound_transparent = f_outbound_transparent;
> > -    new_session->f_transparent_passthrough = f_transparent_passthrough;
> > -    new_session->outbound_ip4 = outbound_ip4;
> > -    new_session->outbound_ip6 = outbound_ip6;
> > -    new_session->outbound_port = outbound_port;
> > -    new_session->host_res_style =
> ats_host_res_from(client_ip->sa_family, host_res_preference);
> > -    new_session->acl_method_mask = acl_method_mask;
> > -
> > -    new_session->new_connection(netvc, backdoor);
> > -
> > -    return EVENT_CONT;
> > -  }
> > -
> > -  /////////////////
> > -  // EVENT_ERROR //
> > -  /////////////////
> > -  if (((long) data) == -ECONNABORTED) {
> > -    /////////////////////////////////////////////////
> > -    // Under Solaris, when accept() fails and sets //
> > -    // errno to EPROTO, it means the client has    //
> > -    // sent a TCP reset before the connection has  //
> > -    // been accepted by the server...  Note that   //
> > -    // in 2.5.1 with the Internet Server Supplement//
> > -    // and also in 2.6 the errno for this case has //
> > -    // changed from EPROTO to ECONNABORTED.        //
> > -    /////////////////////////////////////////////////
> > -
> > -    // FIX: add time to user_agent_hangup
> > -
>  HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
> > -  }
> > -
> > -  MachineFatal("HTTP accept received fatal error: errno = %d",
> -((int)(intptr_t)data));
> > -  return EVENT_CONT;
> > -}
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.h
> > ----------------------------------------------------------------------
> > diff --git a/proxy/http/HttpAccept.h b/proxy/http/HttpAccept.h
> > deleted file mode 100644
> > index 30d68e6..0000000
> > --- a/proxy/http/HttpAccept.h
> > +++ /dev/null
> > @@ -1,198 +0,0 @@
> > -/** @file
> > -
> > -  A brief file description
> > -
> > -  @section license License
> > -
> > -  Licensed to the Apache Software Foundation (ASF) under one
> > -  or more contributor license agreements.  See the NOTICE file
> > -  distributed with this work for additional information
> > -  regarding copyright ownership.  The ASF licenses this file
> > -  to you under the Apache License, Version 2.0 (the
> > -  "License"); you may not use this file except in compliance
> > -  with the License.  You may obtain a copy of the License at
> > -
> > -      http://www.apache.org/licenses/LICENSE-2.0
> > -
> > -  Unless required by applicable law or agreed to in writing, software
> > -  distributed under the License is distributed on an "AS IS" BASIS,
> > -  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > -  See the License for the specific language governing permissions and
> > -  limitations under the License.
> > - */
> > -
> > -#if !defined (_HttpAccept_h_)
> > -#define _HttpAccept_h_
> > -
> > -#include "libts.h"
> > -#include "P_EventSystem.h"
> > -#include "HttpConfig.h"
> > -#include "HTTP.h"
> > -
> > -namespace detail {
> > -  /** Options for @c HttpAccept.
> > -
> > -      @internal This is done as a separate class for two reasons.
> > -
> > -      The first is that in current usage many instances are created
> > -      with the same options so (for the client) this is easier and
> > -      more efficient than passing options directly to the @c
> > -      HttpAccept or calling setters.
> > -
> > -      The second is that @c HttpAccept is not provided with any thread
> > -      safety because it is intended as an immutable object. Putting
> > -      the setters here and not there makes that clearer.
> > -
> > -      We don't do this directly as nested class because we want to
> > -      inherit the data members rather than duplicate the declarations
> > -      and initializations.
> > -   */
> > -  class HttpAcceptOptions {
> > -  private:
> > -    typedef HttpAcceptOptions self; ///< Self reference type.
> > -  public:
> > -    HttpAcceptOptions();
> > -
> > -    // Connection type (HttpProxyPort::TransportType)
> > -    int transport_type;
> > -    /// Set the transport type.
> > -    self& setTransportType(int);
> > -    /// Local address to bind for outbound connections.
> > -    IpAddr outbound_ip4;
> > -    /// Local address to bind for outbound connections.
> > -    IpAddr outbound_ip6;
> > -    /// Set the outbound IP address to @a ip.
> > -    self& setOutboundIp(IpAddr& ip);
> > -    /// Set the outbound IP address to @a ip.
> > -    self& setOutboundIp(IpEndpoint* ip);
> > -    /// Local port for outbound connection.
> > -    uint16_t outbound_port;
> > -    /// Set outbound port.
> > -    self& setOutboundPort(uint16_t);
> > -    /// Outbound transparent.
> > -    bool f_outbound_transparent;
> > -    /// Set outbound transparency.
> > -    self& setOutboundTransparent(bool);
> > -    /// Transparent pass-through.
> > -    bool f_transparent_passthrough;
> > -    /// Set transparent passthrough.
> > -    self& setTransparentPassthrough(bool);
> > -    /// Accepting backdoor connections.
> > -    bool backdoor;
> > -    /// Set backdoor accept.
> > -    self& setBackdoor(bool);
> > -    /// Host address resolution preference order.
> > -    HostResPreferenceOrder host_res_preference;
> > -    /// Set the host query preference.
> > -    self& setHostResPreference(HostResPreferenceOrder const);
> > -  };
> > -
> > -  inline HttpAcceptOptions::HttpAcceptOptions()
> > -    : transport_type(0)
> > -    , outbound_port(0)
> > -    , f_outbound_transparent(false)
> > -    , f_transparent_passthrough(false)
> > -    , backdoor(false)
> > -  {
> > -    memcpy(host_res_preference, host_res_default_preference_order,
> sizeof(host_res_preference));
> > -  }
> > -
> > -  inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setTransportType(int type) {
> > -    transport_type =  type;
> > -    return *this;
> > -  }
> > -
> > -  inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setOutboundIp(IpAddr& ip) {
> > -    if (ip.isIp4()) outbound_ip4 = ip;
> > -    else if (ip.isIp6()) outbound_ip6 = ip;
> > -    return *this;
> > -  }
> > -
> > -  inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setOutboundIp(IpEndpoint* ip) {
> > -    if (ip->isIp4()) outbound_ip4 = *ip;
> > -    else if (ip->isIp6()) outbound_ip6 = *ip;
> > -    return *this;
> > -  }
> > -
> > -  inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setOutboundPort(uint16_t port) {
> > -    outbound_port = port;
> > -    return *this;
> > -  }
> > -
> > -  inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setOutboundTransparent(bool flag) {
> > -    f_outbound_transparent = flag;
> > -    return *this;
> > -  }
> > -
> > -  inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setTransparentPassthrough(bool flag) {
> > -    f_transparent_passthrough = flag;
> > -    return *this;
> > -  }
> > -
> > - inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setBackdoor(bool flag) {
> > -    backdoor = flag;
> > -    return *this;
> > -  }
> > -
> > -  inline HttpAcceptOptions&
> > -  HttpAcceptOptions::setHostResPreference(HostResPreferenceOrder const
> order) {
> > -    memcpy(host_res_preference, order, sizeof(host_res_preference));
> > -    return *this;
> > -  }
> > -}
> > -
> > -/**
> > -   The continuation mutex is NULL to allow parellel accepts in NT. No
> > -   state is recorded by the handler and values are required to be set
> > -   during construction via the @c Options struct and never changed. So
> > -   a NULL mutex is safe.
> > -
> > -   Most of the state is simply passed on to the @c ClientSession after
> > -   an accept. It is done here because this is the least bad pathway
> > -   from the top level configuration to the HTTP session.
> > -*/
> > -
> > -class HttpAccept: public Continuation, private detail::HttpAcceptOptions
> > -{
> > -private:
> > -  typedef HttpAccept self; ///< Self reference type.
> > -public:
> > -  /** Construction options.
> > -      Provide an easier to remember typedef for clients.
> > -  */
> > -  typedef detail::HttpAcceptOptions Options;
> > -
> > -  /** Default constructor.
> > -
> > -      @internal We don't use a static default options object because of
> > -      initialization order issues. It is important to pick up data that
> is read
> > -      from the config file and a static is initialized long before that
> point.
> > -  */
> > -  HttpAccept(Options const& opt = Options())
> > -    : Continuation(NULL)
> > -    , detail::HttpAcceptOptions(opt) // copy these.
> > -  {
> > -    SET_HANDLER(&HttpAccept::mainEvent);
> > -    return;
> > -  }
> > -
> > -  ~HttpAccept()
> > -  {
> > -    return;
> > -  }
> > -
> > -  int mainEvent(int event, void *netvc);
> > -
> > -private:
> > -    HttpAccept(const HttpAccept &);
> > -    HttpAccept & operator =(const HttpAccept &);
> > -};
> > -
> > -#endif
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.cc
> > ----------------------------------------------------------------------
> > diff --git a/proxy/http/HttpAcceptCont.cc b/proxy/http/HttpAcceptCont.cc
> > new file mode 100644
> > index 0000000..d3744e7
> > --- /dev/null
> > +++ b/proxy/http/HttpAcceptCont.cc
> > @@ -0,0 +1,98 @@
> > +/** @file
> > +
> > +  A brief file description
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#include "HttpAcceptCont.h"
> > +#include "IPAllow.h"
> > +#include "HttpClientSession.h"
> > +#include "I_Machine.h"
> > +#include "Error.h"
> > +
> > +int
> > +HttpAcceptCont::mainEvent(int event, void *data)
> > +{
> > +  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> > +  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
> > +
> > +  if (event == NET_EVENT_ACCEPT) {
> > +    ////////////////////////////////////////////////////
> > +    // if client address forbidden, close immediately //
> > +    ////////////////////////////////////////////////////
> > +    NetVConnection *netvc = static_cast<NetVConnection *>(data);
> > +    sockaddr const* client_ip = netvc->get_remote_addr();
> > +    uint32_t acl_method_mask = 0;
> > +    ip_port_text_buffer ipb;
> > +    IpAllow::scoped_config ipallow;
> > +
> > +    // The backdoor port is now only bound to "localhost", so no
> > +    // reason to check for if it's incoming from "localhost" or not.
> > +    if (backdoor) {
> > +      acl_method_mask = IpAllow::AllMethodMask();
> > +    } else if (ipallow && ((acl_method_mask =
> ipallow->match(client_ip)) == 0)) {
> > +      Warning("client '%s' prohibited by ip-allow policy",
> ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
> > +      netvc->do_io_close();
> > +
> > +      return VC_EVENT_CONT;
> > +    }
> > +
> > +    netvc->attributes = transport_type;
> > +
> > +    if (is_debug_tag_set("http_seq"))
> > +      Debug("http_seq", "[HttpAcceptCont:mainEvent %p] accepted
> connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip,
> ipb, sizeof(ipb)), netvc->attributes);
> > +
> > +    HttpClientSession *new_session =
> THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
> > +
> > +   // copy over session related data.
> > +    new_session->f_outbound_transparent = f_outbound_transparent;
> > +    new_session->f_transparent_passthrough = f_transparent_passthrough;
> > +    new_session->outbound_ip4 = outbound_ip4;
> > +    new_session->outbound_ip6 = outbound_ip6;
> > +    new_session->outbound_port = outbound_port;
> > +    new_session->host_res_style =
> ats_host_res_from(client_ip->sa_family, host_res_preference);
> > +    new_session->acl_method_mask = acl_method_mask;
> > +
> > +    new_session->new_connection(netvc, backdoor);
> > +
> > +    return EVENT_CONT;
> > +  }
> > +
> > +  /////////////////
> > +  // EVENT_ERROR //
> > +  /////////////////
> > +  if (((long) data) == -ECONNABORTED) {
> > +    /////////////////////////////////////////////////
> > +    // Under Solaris, when accept() fails and sets //
> > +    // errno to EPROTO, it means the client has    //
> > +    // sent a TCP reset before the connection has  //
> > +    // been accepted by the server...  Note that   //
> > +    // in 2.5.1 with the Internet Server Supplement//
> > +    // and also in 2.6 the errno for this case has //
> > +    // changed from EPROTO to ECONNABORTED.        //
> > +    /////////////////////////////////////////////////
> > +
> > +    // FIX: add time to user_agent_hangup
> > +
>  HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
> > +  }
> > +
> > +  MachineFatal("HTTP accept received fatal error: errno = %d",
> -((int)(intptr_t)data));
> > +  return EVENT_CONT;
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.h
> > ----------------------------------------------------------------------
> > diff --git a/proxy/http/HttpAcceptCont.h b/proxy/http/HttpAcceptCont.h
> > new file mode 100644
> > index 0000000..5b8c437
> > --- /dev/null
> > +++ b/proxy/http/HttpAcceptCont.h
> > @@ -0,0 +1,198 @@
> > +/** @file
> > +
> > +  A brief file description
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#if !defined (_HttpAcceptCont_h_)
> > +#define _HttpAcceptCont_h_
> > +
> > +#include "libts.h"
> > +#include "P_EventSystem.h"
> > +#include "HttpConfig.h"
> > +#include "HTTP.h"
> > +#include "I_Net.h"
> > +
> > +namespace detail {
> > +  /** Options for @c HttpAcceptCont.
> > +
> > +      @internal This is done as a separate class for two reasons.
> > +
> > +      The first is that in current usage many instances are created
> > +      with the same options so (for the client) this is easier and
> > +      more efficient than passing options directly to the @c
> > +      HttpAcceptCont or calling setters.
> > +
> > +      The second is that @c HttpAcceptCont is not provided with any
> thread
> > +      safety because it is intended as an immutable object. Putting
> > +      the setters here and not there makes that clearer.
> > +
> > +      We don't do this directly as nested class because we want to
> > +      inherit the data members rather than duplicate the declarations
> > +      and initializations.
> > +   */
> > +  class HttpAcceptContOptions {
> > +  private:
> > +    typedef HttpAcceptContOptions self; ///< Self reference type.
> > +  public:
> > +    HttpAcceptContOptions();
> > +
> > +    // Connection type (HttpProxyPort::TransportType)
> > +    int transport_type;
> > +    /// Set the transport type.
> > +    self& setTransportType(int);
> > +    /// Local address to bind for outbound connections.
> > +    IpAddr outbound_ip4;
> > +    /// Local address to bind for outbound connections.
> > +    IpAddr outbound_ip6;
> > +    /// Set the outbound IP address to @a ip.
> > +    self& setOutboundIp(IpAddr& ip);
> > +    /// Set the outbound IP address to @a ip.
> > +    self& setOutboundIp(IpEndpoint* ip);
> > +    /// Local port for outbound connection.
> > +    uint16_t outbound_port;
> > +    /// Set outbound port.
> > +    self& setOutboundPort(uint16_t);
> > +    /// Outbound transparent.
> > +    bool f_outbound_transparent;
> > +    /// Set outbound transparency.
> > +    self& setOutboundTransparent(bool);
> > +    /// Transparent pass-through.
> > +    bool f_transparent_passthrough;
> > +    /// Set transparent passthrough.
> > +    self& setTransparentPassthrough(bool);
> > +    /// Accepting backdoor connections.
> > +    bool backdoor;
> > +    /// Set backdoor accept.
> > +    self& setBackdoor(bool);
> > +    /// Host address resolution preference order.
> > +    HostResPreferenceOrder host_res_preference;
> > +    /// Set the host query preference.
> > +    self& setHostResPreference(HostResPreferenceOrder const);
> > +  };
> > +
> > +  inline HttpAcceptContOptions::HttpAcceptContOptions()
> > +    : transport_type(0)
> > +    , outbound_port(0)
> > +    , f_outbound_transparent(false)
> > +    , f_transparent_passthrough(false)
> > +    , backdoor(false)
> > +  {
> > +    memcpy(host_res_preference, host_res_default_preference_order,
> sizeof(host_res_preference));
> > +  }
> > +
> > +  inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setTransportType(int type) {
> > +    transport_type =  type;
> > +    return *this;
> > +  }
> > +
> > +  inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setOutboundIp(IpAddr& ip) {
> > +    if (ip.isIp4()) outbound_ip4 = ip;
> > +    else if (ip.isIp6()) outbound_ip6 = ip;
> > +    return *this;
> > +  }
> > +
> > +  inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setOutboundIp(IpEndpoint* ip) {
> > +    if (ip->isIp4()) outbound_ip4 = *ip;
> > +    else if (ip->isIp6()) outbound_ip6 = *ip;
> > +    return *this;
> > +  }
> > +
> > +  inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setOutboundPort(uint16_t port) {
> > +    outbound_port = port;
> > +    return *this;
> > +  }
> > +
> > +  inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setOutboundTransparent(bool flag) {
> > +    f_outbound_transparent = flag;
> > +    return *this;
> > +  }
> > +
> > +  inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setTransparentPassthrough(bool flag) {
> > +    f_transparent_passthrough = flag;
> > +    return *this;
> > +  }
> > +
> > + inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setBackdoor(bool flag) {
> > +    backdoor = flag;
> > +    return *this;
> > +  }
> > +
> > +  inline HttpAcceptContOptions&
> > +  HttpAcceptContOptions::setHostResPreference(HostResPreferenceOrder
> const order) {
> > +    memcpy(host_res_preference, order, sizeof(host_res_preference));
> > +    return *this;
> > +  }
> > +}
> > +
> > +/**
> > +   The continuation mutex is NULL to allow parellel accepts in NT. No
> > +   state is recorded by the handler and values are required to be set
> > +   during construction via the @c Options struct and never changed. So
> > +   a NULL mutex is safe.
> > +
> > +   Most of the state is simply passed on to the @c ClientSession after
> > +   an accept. It is done here because this is the least bad pathway
> > +   from the top level configuration to the HTTP session.
> > +*/
> > +
> > +class HttpAcceptCont: public AcceptCont, private
> detail::HttpAcceptContOptions
> > +{
> > +private:
> > +  typedef HttpAcceptCont self; ///< Self reference type.
> > +public:
> > +  /** Construction options.
> > +      Provide an easier to remember typedef for clients.
> > +  */
> > +  typedef detail::HttpAcceptContOptions Options;
> > +
> > +  /** Default constructor.
> > +      @internal We don't use a static default options object because of
> > +      initialization order issues. It is important to pick up data that
> is read
> > +      from the config file and a static is initialized long before that
> point.
> > +  */
> > +  HttpAcceptCont(Options const& opt = Options())
> > +    : AcceptCont(NULL)
> > +    , detail::HttpAcceptContOptions(opt) // copy these.
> > +  {
> > +    SET_HANDLER(&HttpAcceptCont::mainEvent);
> > +    return;
> > +  }
> > +
> > +  ~HttpAcceptCont()
> > +  {
> > +    return;
> > +  }
> > +
> > +  int mainEvent(int event, void *netvc);
> > +
> > +private:
> > +    HttpAcceptCont(const HttpAcceptCont &);
> > +    HttpAcceptCont & operator =(const HttpAcceptCont &);
> > +};
> > +
> > +#endif
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpProxyServerMain.cc
> > ----------------------------------------------------------------------
> > diff --git a/proxy/http/HttpProxyServerMain.cc
> b/proxy/http/HttpProxyServerMain.cc
> > index 662856a..8988253 100644
> > --- a/proxy/http/HttpProxyServerMain.cc
> > +++ b/proxy/http/HttpProxyServerMain.cc
> > @@ -26,7 +26,7 @@
> > #include "Main.h"
> > #include "Error.h"
> > #include "HttpConfig.h"
> > -#include "HttpAccept.h"
> > +#include "HttpAcceptCont.h"
> > #include "ReverseProxy.h"
> > #include "HttpSessionManager.h"
> > #include "HttpUpdateSM.h"
> > @@ -35,9 +35,11 @@
> > #include "HttpTunnel.h"
> > #include "Tokenizer.h"
> > #include "P_SSLNextProtocolAccept.h"
> > +#include "P_ProtocolAcceptCont.h"
> > +#include "P_SpdyAcceptCont.h"
> >
> > -HttpAccept *plugin_http_accept = NULL;
> > -HttpAccept *plugin_http_transparent_accept = 0;
> > +HttpAcceptCont *plugin_http_accept = NULL;
> > +HttpAcceptCont *plugin_http_transparent_accept = 0;
> >
> > static SLL<SSLNextProtocolAccept> ssl_plugin_acceptors;
> > static ink_mutex ssl_plugin_mutex = PTHREAD_MUTEX_INITIALIZER;
> > @@ -135,9 +137,10 @@ static void
> > MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port,
> unsigned nthreads)
> > {
> >   NetProcessor::AcceptOptions& net_opt = acceptor._net_opt;
> > -  HttpAccept::Options         accept_opt;
> > +  HttpAcceptCont::Options         accept_opt;
> >
> >   net_opt = make_net_accept_options(port, nthreads);
> > +  net_opt.create_default_NetAccept = false;
> >   REC_ReadConfigInteger(net_opt.recv_bufsize,
> "proxy.config.net.sock_recv_buffer_size_in");
> >   REC_ReadConfigInteger(net_opt.send_bufsize,
> "proxy.config.net.sock_send_buffer_size_in");
> >   REC_ReadConfigInteger(net_opt.packet_mark,
> "proxy.config.net.sock_packet_mark_in");
> > @@ -160,21 +163,24 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor,
> HttpProxyPort& port, unsigned
> >     accept_opt.outbound_ip6 = HttpConfig::m_master.outbound_ip6;
> >   }
> >
> > -  if (port.isSSL()) {
> > -    HttpAccept * accept = NEW(new HttpAccept(accept_opt));
> > -    SSLNextProtocolAccept * ssl = NEW(new
> SSLNextProtocolAccept(accept));
> > +  HttpAcceptCont *http = NEW(new HttpAcceptCont(accept_opt));
> > +  SpdyAcceptCont *spdy = NEW(new SpdyAcceptCont(http));
> > +  SSLNextProtocolAccept *ssl = NEW(new SSLNextProtocolAccept(http));
> > +  ProtocolAcceptCont *proto = NEW(new ProtocolAcceptCont());
> > +
> > +  proto->registerEndpoint(TS_PROTO_TLS, ssl);
> > +  proto->registerEndpoint(TS_PROTO_HTTP, http);
> > +  proto->registerEndpoint(TS_PROTO_SPDY, spdy);
> >
> > +  if (port.isSSL()) {
> >     // ALPN selects the first server-offered protocol, so make sure that
> we offer HTTP/1.1 first.
> > -    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, accept);
> > -    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, accept);
> > +    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, http);
> > +    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, http);
> >
> >     ink_scoped_mutex lock(ssl_plugin_mutex);
> >     ssl_plugin_acceptors.push(ssl);
> > -
> > -    acceptor._accept = ssl;
> > -  } else {
> > -    acceptor._accept = NEW(new HttpAccept(accept_opt));
> >   }
> > +  acceptor._accept = proto;
> > }
> >
> > /** Set up all the accepts and sockets.
> > @@ -197,14 +203,14 @@ init_HttpProxyServer(int n_accept_threads)
> >   //   port but without going through the operating system
> >   //
> >   if (plugin_http_accept == NULL) {
> > -    plugin_http_accept = NEW(new HttpAccept);
> > +    plugin_http_accept = NEW(new HttpAcceptCont);
> >     plugin_http_accept->mutex = new_ProxyMutex();
> >   }
> >   // Same as plugin_http_accept except outbound transparent.
> >   if (! plugin_http_transparent_accept) {
> > -    HttpAccept::Options ha_opt;
> > +    HttpAcceptCont::Options ha_opt;
> >     ha_opt.setOutboundTransparent(true);
> > -    plugin_http_transparent_accept = NEW(new HttpAccept(ha_opt));
> > +    plugin_http_transparent_accept = NEW(new HttpAcceptCont(ha_opt));
> >     plugin_http_transparent_accept->mutex = new_ProxyMutex();
> >   }
> >   ink_mutex_init(&ssl_plugin_mutex, "SSL Acceptor List");
> > @@ -262,7 +268,7 @@ void
> > start_HttpProxyServerBackDoor(int port, int accept_threads)
> > {
> >   NetProcessor::AcceptOptions opt;
> > -  HttpAccept::Options ha_opt;
> > +  HttpAcceptCont::Options ha_opt;
> >
> >   opt.local_port = port;
> >   opt.accept_threads = accept_threads;
> > @@ -271,5 +277,5 @@ start_HttpProxyServerBackDoor(int port, int
> accept_threads)
> >   opt.backdoor = true;
> >
> >   // The backdoor only binds the loopback interface
> > -  netProcessor.main_accept(NEW(new HttpAccept(ha_opt)), NO_FD, opt);
> > +  netProcessor.main_accept(NEW(new HttpAcceptCont(ha_opt)), NO_FD, opt);
> > }
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/Makefile.am
> > ----------------------------------------------------------------------
> > diff --git a/proxy/http/Makefile.am b/proxy/http/Makefile.am
> > index c413ed7..a71d287 100644
> > --- a/proxy/http/Makefile.am
> > +++ b/proxy/http/Makefile.am
> > @@ -31,14 +31,15 @@ AM_CPPFLAGS = \
> >   -I$(top_srcdir)/proxy/hdrs \
> >   -I$(top_srcdir)/proxy/shared \
> >   -I$(top_srcdir)/proxy/http/remap \
> > -  -I$(top_srcdir)/proxy/logging
> > +  -I$(top_srcdir)/proxy/logging \
> > +  -I$(top_srcdir)/proxy/spdy
> >
> > noinst_HEADERS = HttpProxyServerMain.h
> > noinst_LIBRARIES = libhttp.a
> >
> > libhttp_a_SOURCES = \
> > -  HttpAccept.cc \
> > -  HttpAccept.h \
> > +  HttpAcceptCont.cc \
> > +  HttpAcceptCont.h \
> >   HttpBodyFactory.cc \
> >   HttpBodyFactory.h \
> >   HttpCacheSM.cc \
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/Makefile.am
> > ----------------------------------------------------------------------
> > diff --git a/proxy/spdy/Makefile.am b/proxy/spdy/Makefile.am
> > new file mode 100644
> > index 0000000..63911e1
> > --- /dev/null
> > +++ b/proxy/spdy/Makefile.am
> > @@ -0,0 +1,38 @@
> > +# Makefile.am for the traffic/iocore/net hierarchy
> > +#
> > +#  Licensed to the Apache Software Foundation (ASF) under one
> > +#  or more contributor license agreements.  See the NOTICE file
> > +#  distributed with this work for additional information
> > +#  regarding copyright ownership.  The ASF licenses this file
> > +#  to you under the Apache License, Version 2.0 (the
> > +#  "License"); you may not use this file except in compliance
> > +#  with the License.  You may obtain a copy of the License at
> > +#
> > +#      http://www.apache.org/licenses/LICENSE-2.0
> > +#
> > +#  Unless required by applicable law or agreed to in writing, software
> > +#  distributed under the License is distributed on an "AS IS" BASIS,
> > +#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +#  See the License for the specific language governing permissions and
> > +#  limitations under the License.
> > +
> > +AM_CPPFLAGS = \
> > +  $(iocore_include_dirs) \
> > +  -I$(top_srcdir)/lib \
> > +  -I$(top_srcdir)/lib/records \
> > +  -I$(top_srcdir)/lib/ts \
> > +  -I$(top_srcdir)/proxy \
> > +  -I$(top_srcdir)/proxy/api \
> > +  -I$(top_srcdir)/proxy/hdrs \
> > +  -I$(top_srcdir)/proxy/shared \
> > +  -I$(top_srcdir)/proxy/spdy \
> > +  -I$(top_srcdir)/mgmt \
> > +  -I$(top_srcdir)/mgmt/preparse \
> > +  -I$(top_srcdir)/mgmt/utils \
> > +  @SPDYLAY_CFLAGS@
> > +
> > +noinst_LIBRARIES = libspdy.a
> > +
> > +libspdy_a_SOURCES = \
> > +  P_SpdyAcceptCont.h \
> > +  SpdyAcceptCont.cc
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/P_SpdyAcceptCont.h
> > ----------------------------------------------------------------------
> > diff --git a/proxy/spdy/P_SpdyAcceptCont.h
> b/proxy/spdy/P_SpdyAcceptCont.h
> > new file mode 100644
> > index 0000000..88aea02
> > --- /dev/null
> > +++ b/proxy/spdy/P_SpdyAcceptCont.h
> > @@ -0,0 +1,46 @@
> > +/** @file
> > +
> > +  SpdyAcceptCont
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#ifndef P_SpdyAcceptCont_H_
> > +#define P_SpdyAcceptCont_H_
> > +
> > +#include "P_Net.h"
> > +#include "P_EventSystem.h"
> > +#include "P_UnixNet.h"
> > +#include "I_IOBuffer.h"
> > +
> > +class SpdyAcceptCont: public AcceptCont
> > +{
> > +public:
> > +  SpdyAcceptCont(Continuation *ep);
> > +  ~SpdyAcceptCont() {}
> > +
> > +private:
> > +  int mainEvent(int event, void *netvc);
> > +  SpdyAcceptCont(const SpdyAcceptCont &); // disabled
> > +  SpdyAcceptCont& operator =(const SpdyAcceptCont&); // disabled
> > +
> > +  Continuation *endpoint;
> > +};
> > +
> > +#endif /* P_SpdyAcceptCont_H_ */
> >
> >
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/SpdyAcceptCont.cc
> > ----------------------------------------------------------------------
> > diff --git a/proxy/spdy/SpdyAcceptCont.cc b/proxy/spdy/SpdyAcceptCont.cc
> > new file mode 100644
> > index 0000000..b9efc80
> > --- /dev/null
> > +++ b/proxy/spdy/SpdyAcceptCont.cc
> > @@ -0,0 +1,37 @@
> > +/** @file
> > +
> > +  SpdyNetAccept
> > +
> > +  @section license License
> > +
> > +  Licensed to the Apache Software Foundation (ASF) under one
> > +  or more contributor license agreements.  See the NOTICE file
> > +  distributed with this work for additional information
> > +  regarding copyright ownership.  The ASF licenses this file
> > +  to you under the Apache License, Version 2.0 (the
> > +  "License"); you may not use this file except in compliance
> > +  with the License.  You may obtain a copy of the License at
> > +
> > +      http://www.apache.org/licenses/LICENSE-2.0
> > +
> > +  Unless required by applicable law or agreed to in writing, software
> > +  distributed under the License is distributed on an "AS IS" BASIS,
> > +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> > +  See the License for the specific language governing permissions and
> > +  limitations under the License.
> > + */
> > +
> > +#include "P_SpdyAcceptCont.h"
> > +
> > +SpdyAcceptCont::SpdyAcceptCont(Continuation *ep)
> > +    : AcceptCont(new_ProxyMutex()), endpoint(ep)
> > +{
> > +  SET_HANDLER(&SpdyAcceptCont::mainEvent);
> > +}
> > +
> > +int
> > +SpdyAcceptCont::mainEvent(int event, void *netvc)
> > +{
> > +  printf("spdy accepted\n");
> > +  return 0;
> > +}
> >
>
>


-- 
Yunkai Zhang
Work at Taobao

Re: [2/3] git commit: TS-2431: Preparation of SPDY protocol

Posted by James Peach <jp...@apache.org>.
I just started reviewing this, a few high level comments ...

I don't like the renaming of *Accept to *AcceptCont. This results in names like HttpAcceptCont::Options, which is pretty ugly.

I think we need to separate SPDY probe from the UnixNetAccept better. A good goal to have is for iocore to know nothing at all about SPDY. The way I imagine this is for the generic protocol acceptor to take a table of probe functions or continuations that can detect the next protocol. Enabling detection should be a per-port option.

I'll work on this and other SPDY issues over the coming weeks, hopefully we can keep improving it :)

On Mar 20, 2014, at 10:32 AM, yunkai@apache.org wrote:

> TS-2431: Preparation of SPDY protocol
> 
> *) Create basic data structures for SPDY protocal, such as:
>   SpdyAcceptCont.
> 
> *) SPDY will share the same port number with HTTP protocol, ATS can
>   recognize them by detecting the first byte of client request.
> 
> *) HttpAccept looks like a subclass of NetAccept. To avoid confusing,
>   rename HttpAccept to HttpAcceptCont.
> 
> Signed-off-by: Yunkai Zhang <qi...@taobao.com>
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/f90f3a48
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/f90f3a48
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/f90f3a48
> 
> Branch: refs/heads/master
> Commit: f90f3a488323ccb8b98c5571ef6cf035f1be51a9
> Parents: f1a005e
> Author: Yunkai Zhang <qi...@taobao.com>
> Authored: Wed Dec 11 21:21:53 2013 +0800
> Committer: Yunkai Zhang <qi...@taobao.com>
> Committed: Fri Mar 21 01:29:21 2014 +0800
> 
> ----------------------------------------------------------------------
> configure.ac                         |   1 +
> iocore/net/AcceptCont.cc             |  41 +++++++
> iocore/net/I_AcceptCont.h            |  48 ++++++++
> iocore/net/I_Net.h                   |   1 +
> iocore/net/I_NetProcessor.h          |   1 +
> iocore/net/Makefile.am               |   8 ++
> iocore/net/P_Net.h                   |   1 +
> iocore/net/P_NetAccept.h             |   2 +
> iocore/net/P_ProtocolAcceptCont.h    |  50 ++++++++
> iocore/net/P_ProtocolNetAccept.h     |  62 ++++++++++
> iocore/net/P_SSLNetAccept.h          |   1 +
> iocore/net/P_SSLNextProtocolAccept.h |   2 +-
> iocore/net/P_SSLNextProtocolSet.h    |   2 +-
> iocore/net/P_UnixNetVConnection.h    |   9 ++
> iocore/net/ProtocolAcceptCont.cc     |  73 +++++++++++
> iocore/net/ProtocolNetAccept.cc      |  68 ++++++++++
> iocore/net/SSLNetAccept.cc           |  16 ++-
> iocore/net/SSLNetVConnection.cc      |   3 +-
> iocore/net/SSLNextProtocolAccept.cc  |   2 +-
> iocore/net/SSLNextProtocolSet.cc     |   5 +-
> iocore/net/UnixNetAccept.cc          |  69 +++++++----
> iocore/net/UnixNetProcessor.cc       |  20 ++-
> iocore/net/UnixNetVConnection.cc     |  95 +++++++++++++-
> lib/ts/apidefs.h.in                  |   3 +
> proxy/InkAPI.cc                      |   6 +-
> proxy/Makefile.am                    |  12 +-
> proxy/SocksProxy.cc                  |   6 +-
> proxy/api/ts/InkAPIPrivateIOCore.h   |   4 +
> proxy/http/HttpAccept.cc             |  98 ---------------
> proxy/http/HttpAccept.h              | 198 ------------------------------
> proxy/http/HttpAcceptCont.cc         |  98 +++++++++++++++
> proxy/http/HttpAcceptCont.h          | 198 ++++++++++++++++++++++++++++++
> proxy/http/HttpProxyServerMain.cc    |  42 ++++---
> proxy/http/Makefile.am               |   7 +-
> proxy/spdy/Makefile.am               |  38 ++++++
> proxy/spdy/P_SpdyAcceptCont.h        |  46 +++++++
> proxy/spdy/SpdyAcceptCont.cc         |  37 ++++++
> 37 files changed, 1004 insertions(+), 369 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/configure.ac
> ----------------------------------------------------------------------
> diff --git a/configure.ac b/configure.ac
> index f87674d..796158e 100644
> --- a/configure.ac
> +++ b/configure.ac
> @@ -2010,6 +2010,7 @@ AC_CONFIG_FILES([
>   proxy/http/Makefile
>   proxy/http/remap/Makefile
>   proxy/logging/Makefile
> +  proxy/spdy/Makefile
>   rc/Makefile
>   rc/trafficserver
>   rc/trafficserver.conf
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/AcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/AcceptCont.cc b/iocore/net/AcceptCont.cc
> new file mode 100644
> index 0000000..e112230
> --- /dev/null
> +++ b/iocore/net/AcceptCont.cc
> @@ -0,0 +1,41 @@
> +/** @file
> +
> +  AcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "I_AcceptCont.h"
> +#include "P_Net.h"
> +
> +AcceptCont::AcceptCont(ProxyMutex *amutex)
> +    : Continuation(amutex)
> +{
> +  SET_HANDLER(&AcceptCont::mainEvent);
> +}
> +
> +AcceptCont::~AcceptCont()
> +{
> +}
> +
> +void *
> +AcceptCont::createNetAccept()
> +{
> +    return (NEW(new NetAccept));
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_AcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/I_AcceptCont.h b/iocore/net/I_AcceptCont.h
> new file mode 100644
> index 0000000..2d59e1e
> --- /dev/null
> +++ b/iocore/net/I_AcceptCont.h
> @@ -0,0 +1,48 @@
> +/** @file
> +
> +  AcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#ifndef I_AcceptCont_H_
> +#define I_AcceptCont_H_
> +#include "I_Net.h"
> +#include "I_VConnection.h"
> +
> +class AcceptCont: public Continuation
> +{
> +public:
> +  AcceptCont(ProxyMutex *amutex);
> +  ~AcceptCont();
> +
> +  //
> +  // Virtual function allows creation of an SSLNetAccept
> +  // or NetAccept transparent to NetProcessor.
> +  //
> +  // This function should return a pointer
> +  // of NetAccept or its subclass.
> +  //
> +  virtual void *createNetAccept();
> +
> +private:
> +  virtual int mainEvent(int event, void * netvc) = 0;
> +};
> +
> +#endif /* I_AcceptCont_H_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_Net.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/I_Net.h b/iocore/net/I_Net.h
> index 78f3d62..a98b6a3 100644
> --- a/iocore/net/I_Net.h
> +++ b/iocore/net/I_Net.h
> @@ -91,6 +91,7 @@ extern int net_config_poll_timeout;
> 
> #include "I_NetVConnection.h"
> #include "I_NetProcessor.h"
> +#include "I_AcceptCont.h"
> 
> void ink_net_init(ModuleVersion version);
> #endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_NetProcessor.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/I_NetProcessor.h b/iocore/net/I_NetProcessor.h
> index 3deae4c..68ce117 100644
> --- a/iocore/net/I_NetProcessor.h
> +++ b/iocore/net/I_NetProcessor.h
> @@ -100,6 +100,7 @@ public:
>     AcceptOptions() { this->reset(); }
>     /// Reset all values to defaults.
>     self& reset();
> +    bool create_default_NetAccept;
>   };
> 
>   /**
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/iocore/net/Makefile.am b/iocore/net/Makefile.am
> index 4575e9e..efbfa86 100644
> --- a/iocore/net/Makefile.am
> +++ b/iocore/net/Makefile.am
> @@ -52,6 +52,8 @@ libinknet_a_SOURCES = \
>   I_UDPNet.h \
>   I_UDPPacket.h \
>   Inline.cc \
> +  I_AcceptCont.h \
> +  AcceptCont.cc \
>   Net.cc \
>   NetVConnection.cc \
>   P_CompletionUtil.h \
> @@ -61,6 +63,9 @@ libinknet_a_SOURCES = \
>   P_Net.h \
>   P_NetAccept.h \
>   P_NetVConnection.h \
> +  P_ProtocolNetAccept.h \
> +  P_ProtocolAcceptCont.h \
> +  P_Socks.h \
>   P_SSLCertLookup.h \
>   P_SSLConfig.h \
>   P_SSLNetAccept.h \
> @@ -81,6 +86,9 @@ libinknet_a_SOURCES = \
>   P_UnixNetVConnection.h \
>   P_UnixPollDescriptor.h \
>   P_UnixUDPConnection.h \
> +  ProtocolNetAccept.cc \
> +  ProtocolAcceptCont.cc \
> +  Socks.cc \
>   SSLCertLookup.cc \
>   SSLConfig.cc \
>   SSLNetAccept.cc \
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_Net.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_Net.h b/iocore/net/P_Net.h
> index 13ab617..db9acc6 100644
> --- a/iocore/net/P_Net.h
> +++ b/iocore/net/P_Net.h
> @@ -104,6 +104,7 @@ do { \
> #include "P_NetVCTest.h"
> #include "P_LibBulkIO.h"
> 
> +#include "P_ProtocolNetAccept.h"
> #include "P_SSLNetVConnection.h"
> #include "P_SSLNetProcessor.h"
> #include "P_SSLNetAccept.h"
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_NetAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_NetAccept.h b/iocore/net/P_NetAccept.h
> index 3a82e4b..762a601 100644
> --- a/iocore/net/P_NetAccept.h
> +++ b/iocore/net/P_NetAccept.h
> @@ -106,11 +106,13 @@ struct NetAccept:public Continuation
>   virtual UnixNetVConnection *allocateThread(EThread *t);
>   virtual void freeThread(UnixNetVConnection *vc, EThread *t);
>   virtual UnixNetVConnection *allocateGlobal();
> +  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection &con);
>   virtual EventType getEtype();
> 
>   void init_accept_loop(const char *);
>   virtual void init_accept(EThread * t = NULL);
>   virtual void init_accept_per_thread();
> +  virtual NetAccept *clone();
>   // 0 == success
>   int do_listen(bool non_blocking, bool transparent = false);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolAcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_ProtocolAcceptCont.h b/iocore/net/P_ProtocolAcceptCont.h
> new file mode 100644
> index 0000000..7631e5c
> --- /dev/null
> +++ b/iocore/net/P_ProtocolAcceptCont.h
> @@ -0,0 +1,50 @@
> +/** @file
> +
> +  ProtocolAcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#ifndef P_ProtocolAcceptCont_H_
> +#define P_ProtocolAcceptCont_H_
> +
> +#include "I_AcceptCont.h"
> +
> +class ProtocolAcceptCont: public AcceptCont
> +{
> +public:
> +  ProtocolAcceptCont(): AcceptCont(NULL)
> +  {
> +    memset(endpoint, 0, TS_PROTO_MAX * sizeof(AcceptCont *));
> +    SET_HANDLER(&ProtocolAcceptCont::mainEvent);
> +  }
> +  ~ProtocolAcceptCont() {}
> +
> +  void *createNetAccept();
> +  void registerEndpoint(TSProtoType type, Continuation *ep);
> +
> +private:
> +  int mainEvent(int event, void * netvc);
> +  ProtocolAcceptCont(const ProtocolAcceptCont &); // disabled
> +  ProtocolAcceptCont& operator =(const ProtocolAcceptCont&); // disabled
> +
> +  Continuation *endpoint[TS_PROTO_MAX];
> +};
> +
> +#endif /* P_ProtocolAcceptCont_H_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolNetAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_ProtocolNetAccept.h b/iocore/net/P_ProtocolNetAccept.h
> new file mode 100644
> index 0000000..ac49571
> --- /dev/null
> +++ b/iocore/net/P_ProtocolNetAccept.h
> @@ -0,0 +1,62 @@
> +/** @file
> +
> +  A brief file description
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +/****************************************************************************
> +
> +  P_ProtocolNetAccept.h
> +
> +
> +   NetAccept is a generalized facility which allows
> +   Connections of different classes to be accepted either
> +   from a blockable thread or by adaptive polling.
> +
> +   It is used by the NetProcessor and the ClusterProcessor
> +   and should be considered PRIVATE to processor implementations.
> +
> +
> +
> + ****************************************************************************/
> +#if !defined (_ProtocolNetAccept_h_)
> +#define _ProtocolNetAccept_h_
> +
> +#include "libts.h"
> +#include "P_NetAccept.h"
> +#include "P_Connection.h"
> +
> +class Event;
> +class UnixNetVConnection;
> +
> +//
> +// NetAccept
> +// Handles accepting connections.
> +//
> +struct ProtocolNetAccept: public NetAccept
> +{
> +  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection &con);
> +  virtual NetAccept *clone();
> +
> +  ProtocolNetAccept() {}
> +  virtual ~ProtocolNetAccept() {}
> +
> +};
> +#endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNetAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNetAccept.h b/iocore/net/P_SSLNetAccept.h
> index e9a8f7b..6d963a3 100644
> --- a/iocore/net/P_SSLNetAccept.h
> +++ b/iocore/net/P_SSLNetAccept.h
> @@ -57,6 +57,7 @@ struct SSLNetAccept: public NetAccept
>   virtual UnixNetVConnection *allocateGlobal();
>   virtual EventType getEtype();
>   virtual void init_accept_per_thread();
> +  virtual NetAccept *clone();
> 
>   SSLNetAccept()
>     { };
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNextProtocolAccept.h b/iocore/net/P_SSLNextProtocolAccept.h
> index c3ee575..342485a 100644
> --- a/iocore/net/P_SSLNextProtocolAccept.h
> +++ b/iocore/net/P_SSLNextProtocolAccept.h
> @@ -31,7 +31,7 @@
> #include "P_SSLNextProtocolSet.h"
> #include "I_IOBuffer.h"
> 
> -class SSLNextProtocolAccept: public Continuation
> +class SSLNextProtocolAccept: public AcceptCont
> {
> public:
>   SSLNextProtocolAccept(Continuation *);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolSet.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNextProtocolSet.h b/iocore/net/P_SSLNextProtocolSet.h
> index e25f50d..d3c37b6 100644
> --- a/iocore/net/P_SSLNextProtocolSet.h
> +++ b/iocore/net/P_SSLNextProtocolSet.h
> @@ -40,7 +40,7 @@ public:
>   bool advertiseProtocols(const unsigned char ** out, unsigned * len) const;
> 
>   Continuation * findEndpoint(const char *) const;
> -  Continuation * findEndpoint(const unsigned char *, unsigned, TSClientProtoStack *) const;
> +  Continuation * findEndpoint(const unsigned char *, unsigned, TSClientProtoStack *, const char **) const;
> 
>   struct NextProtocolEndpoint
>   {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_UnixNetVConnection.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_UnixNetVConnection.h b/iocore/net/P_UnixNetVConnection.h
> index abdf4ba..0e4eae8 100644
> --- a/iocore/net/P_UnixNetVConnection.h
> +++ b/iocore/net/P_UnixNetVConnection.h
> @@ -41,6 +41,13 @@ class UnixNetVConnection;
> class NetHandler;
> struct PollDescriptor;
> 
> +enum SpdyProbeState
> +{
> +  SPDY_PROBE_STATE_NONE,
> +  SPDY_PROBE_STATE_BEGIN,
> +  SPDY_PROBE_STATE_END
> +};
> +
> TS_INLINE void
> NetVCOptions::reset()
> {
> @@ -233,6 +240,8 @@ public:
>   ink_hrtime submit_time;
>   OOB_callback *oob_ptr;
>   bool from_accept_thread;
> +  SpdyProbeState probe_state;
> +  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/f90f3a48/iocore/net/ProtocolAcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/ProtocolAcceptCont.cc b/iocore/net/ProtocolAcceptCont.cc
> new file mode 100644
> index 0000000..2e73b77
> --- /dev/null
> +++ b/iocore/net/ProtocolAcceptCont.cc
> @@ -0,0 +1,73 @@
> +/** @file
> +
> +  ProtocolAcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "P_ProtocolAcceptCont.h"
> +#include "P_SSLNextProtocolAccept.h"
> +#include "P_Net.h"
> +#include "I_Machine.h"
> +#include "Error.h"
> +
> +void *
> +ProtocolAcceptCont::createNetAccept()
> +{
> +  return ((NetAccept *) NEW(new ProtocolNetAccept));
> +}
> +
> +void
> +ProtocolAcceptCont::registerEndpoint(TSProtoType type, Continuation *ep)
> +{
> +  endpoint[type] = ep;
> +}
> +
> +int
> +ProtocolAcceptCont::mainEvent(int event, void *netvc)
> +{
> +  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> +  ink_release_assert((event == NET_EVENT_ACCEPT) ? (netvc!= 0) : (1));
> +
> +  if (event == NET_EVENT_ACCEPT) {
> +    TSProtoType proto_type;
> +    UnixNetVConnection *vc = (UnixNetVConnection *)netvc;
> +
> +    if (vc->proto_stack & (1u << TS_PROTO_TLS)) {
> +      proto_type = TS_PROTO_TLS;
> +    } else if (vc->proto_stack & (1u << TS_PROTO_HTTP)) {
> +      proto_type = TS_PROTO_HTTP;
> +    } else if (vc->proto_stack & (1u << TS_PROTO_SPDY)) {
> +      proto_type = TS_PROTO_SPDY;
> +    } else {
> +      Warning("Invalid protocol stack:%x", vc->proto_stack);
> +      return EVENT_CONT;
> +    }
> +
> +    if (endpoint[proto_type])
> +      endpoint[proto_type]->handleEvent(NET_EVENT_ACCEPT, netvc);
> +    else
> +      Warning("Unregistered protocol type:%x", proto_type);
> +
> +    return EVENT_CONT;
> +  }
> +
> +  MachineFatal("Protocol Accept received fatal error: errno = %d", -((int)(intptr_t)netvc));
> +  return EVENT_CONT;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/ProtocolNetAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/ProtocolNetAccept.cc b/iocore/net/ProtocolNetAccept.cc
> new file mode 100644
> index 0000000..c4257e2
> --- /dev/null
> +++ b/iocore/net/ProtocolNetAccept.cc
> @@ -0,0 +1,68 @@
> +/** @file
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "ink_config.h"
> +#include "P_Net.h"
> +#include "P_ProtocolNetAccept.h"
> +
> +UnixNetVConnection *
> +ProtocolNetAccept::createSuitableVC(EThread *t, Connection &con)
> +{
> +  UnixNetVConnection *vc;
> +
> +  if (etype == SSLNetProcessor::ET_SSL && etype) {
> +    // SSL protocol
> +    if (t)
> +      vc = (UnixNetVConnection *)THREAD_ALLOC(sslNetVCAllocator, t);
> +    else
> +      vc = (UnixNetVConnection *)sslNetVCAllocator.alloc();
> +    vc->proto_stack = (1u << TS_PROTO_TLS);
> +  } else {
> +    if (t)
> +      vc = THREAD_ALLOC(netVCAllocator, t);
> +    else
> +      vc = netVCAllocator.alloc();
> +
> +#if TS_HAS_SPDY
> +    vc->probe_state = SPDY_PROBE_STATE_BEGIN;
> +#else
> +    vc->probe_state = SPDY_PROBE_STATE_NONE;
> +#endif
> +
> +    //
> +    // Protocol stack may be changed by
> +    // following call of SpdyProbe()
> +    //
> +    vc->proto_stack = (1u << TS_PROTO_HTTP);
> +  }
> +
> +  vc->con = con;
> +  return vc;
> +}
> +
> +NetAccept *
> +ProtocolNetAccept::clone()
> +{
> +  NetAccept *na;
> +  na = NEW(new ProtocolNetAccept);
> +  *na = *this;
> +  return na;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetAccept.cc b/iocore/net/SSLNetAccept.cc
> index a321f89..6aa5ed9 100644
> --- a/iocore/net/SSLNetAccept.cc
> +++ b/iocore/net/SSLNetAccept.cc
> @@ -70,10 +70,9 @@ SSLNetAccept::init_accept_per_thread()
>   NetAccept *a = this;
>   n = eventProcessor.n_threads_for_type[SSLNetProcessor::ET_SSL];
>   for (i = 0; i < n; i++) {
> -    if (i < n - 1) {
> -      a = NEW(new SSLNetAccept);
> -      *a = *this;
> -    } else
> +    if (i < n - 1)
> +      a = clone();
> +    else
>       a = this;
>     EThread *t = eventProcessor.eventthread[SSLNetProcessor::ET_SSL][i];
> 
> @@ -84,3 +83,12 @@ SSLNetAccept::init_accept_per_thread()
>     t->schedule_every(a, period, etype);
>   }
> }
> +
> +NetAccept *
> +SSLNetAccept::clone()
> +{
> +  NetAccept *na;
> +  na = NEW(new SSLNetAccept);
> +  *na = *this;
> +  return na;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
> index b55dcf7..3925de9 100644
> --- a/iocore/net/SSLNetVConnection.cc
> +++ b/iocore/net/SSLNetVConnection.cc
> @@ -578,7 +578,8 @@ 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->proto_stack);
> +        this->npnEndpoint = this->npnSet->findEndpoint(proto, len, &this->proto_stack,
> +                                                       &this->selected_next_protocol);
>         this->npnSet = NULL;
> 
>         ink_assert(this->npnEndpoint != NULL);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNextProtocolAccept.cc b/iocore/net/SSLNextProtocolAccept.cc
> index bfc2f0b..746995b 100644
> --- a/iocore/net/SSLNextProtocolAccept.cc
> +++ b/iocore/net/SSLNextProtocolAccept.cc
> @@ -147,7 +147,7 @@ SSLNextProtocolAccept::unregisterEndpoint(
> }
> 
> SSLNextProtocolAccept::SSLNextProtocolAccept(Continuation * ep)
> -    : Continuation(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
> +    : AcceptCont(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
> {
>   SET_HANDLER(&SSLNextProtocolAccept::mainEvent);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolSet.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNextProtocolSet.cc b/iocore/net/SSLNextProtocolSet.cc
> index 148a6d1..ed0a5e3 100644
> --- a/iocore/net/SSLNextProtocolSet.cc
> +++ b/iocore/net/SSLNextProtocolSet.cc
> @@ -133,7 +133,8 @@ SSLNextProtocolSet::unregisterEndpoint(const char * proto, Continuation * ep)
> 
> Continuation *
> SSLNextProtocolSet::findEndpoint(const unsigned char * proto, unsigned len,
> -                                 TSClientProtoStack *proto_stack) const
> +                                 TSClientProtoStack *proto_stack,
> +                                 const char **selected_protocol) const
> {
>   for (const NextProtocolEndpoint * ep = this->endpoints.head;
>         ep != NULL; ep = this->endpoints.next(ep)) {
> @@ -141,6 +142,8 @@ SSLNextProtocolSet::findEndpoint(const unsigned char * proto, unsigned len,
>     if (sz == len && memcmp(ep->protocol, proto, len) == 0) {
>       if (proto_stack)
>         *proto_stack = ep->proto_stack;
> +      if (selected_protocol)
> +        *selected_protocol = ep->protocol;
>       return ep->endpoint;
>     }
>   }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/UnixNetAccept.cc b/iocore/net/UnixNetAccept.cc
> index c038a2e..af4d8f9 100644
> --- a/iocore/net/UnixNetAccept.cc
> +++ b/iocore/net/UnixNetAccept.cc
> @@ -223,10 +223,9 @@ NetAccept::init_accept_per_thread()
>   NetAccept *a;
>   n = eventProcessor.n_threads_for_type[ET_NET];
>   for (i = 0; i < n; i++) {
> -    if (i < n - 1) {
> -      a = NEW(new NetAccept);
> -      *a = *this;
> -    } else
> +    if (i < n - 1)
> +      a = clone();
> +    else
>       a = this;
>     EThread *t = eventProcessor.eventthread[ET_NET][i];
>     PollDescriptor *pd = get_PollDescriptor(t);
> @@ -237,6 +236,14 @@ NetAccept::init_accept_per_thread()
>   }
> }
> 
> +NetAccept *
> +NetAccept::clone()
> +{
> +  NetAccept *na;
> +  na = NEW(new NetAccept);
> +  *na = *this;
> +  return na;
> +}
> 
> int
> NetAccept::do_listen(bool non_blocking, bool transparent)
> @@ -264,6 +271,20 @@ NetAccept::do_listen(bool non_blocking, bool transparent)
>   return res;
> }
> 
> +UnixNetVConnection *
> +NetAccept::createSuitableVC(EThread *t, Connection &con)
> +{
> +  UnixNetVConnection *vc;
> +
> +  if (t)
> +    vc = allocateThread(t);
> +  else
> +    vc = allocateGlobal();
> +
> +  vc->con = con;
> +
> +  return vc;
> +}
> 
> int
> NetAccept::do_blocking_accept(EThread * t)
> @@ -271,18 +292,11 @@ NetAccept::do_blocking_accept(EThread * t)
>   int res = 0;
>   int loop = accept_till_done;
>   UnixNetVConnection *vc = NULL;
> +  Connection con;
> 
>   //do-while for accepting all the connections
>   //added by YTS Team, yamsat
>   do {
> -    vc = (UnixNetVConnection *)alloc_cache;
> -    if (likely(!vc)) {
> -      //vc = allocateThread(t);
> -      vc = allocateGlobal(); // Bypass proxy / thread allocator
> -      vc->from_accept_thread = true;
> -      vc->id = net_next_connection_number();
> -      alloc_cache = vc;
> -    }
>     ink_hrtime now = ink_get_hrtime();
> 
>     // Throttle accepts
> @@ -297,7 +311,7 @@ NetAccept::do_blocking_accept(EThread * t)
>       now = ink_get_hrtime();
>     }
> 
> -    if ((res = server.accept(&vc->con)) < 0) {
> +    if ((res = server.accept(&con)) < 0) {
>     Lerror:
>       int seriousness = accept_error_seriousness(res);
>       if (seriousness >= 0) {   // not so bad
> @@ -314,16 +328,23 @@ NetAccept::do_blocking_accept(EThread * t)
>       }
>       return -1;
>     }
> -    check_emergency_throttle(vc->con);
> +
> +    // Use 'NULL' to Bypass thread allocator
> +    vc = createSuitableVC(NULL, con);
> +    if (!vc)
> +      return -1;
> +    vc->from_accept_thread = true;
> +    vc->id = net_next_connection_number();
>     alloc_cache = NULL;
> 
> +    check_emergency_throttle(con);
> +
>     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
>     vc->submit_time = now;
>     ats_ip_copy(&vc->server_addr, &vc->con.addr);
>     vc->set_is_transparent(server.f_inbound_transparent);
>     vc->mutex = new_ProxyMutex();
>     vc->action_ = *action_;
> -    vc->proto_stack = (1u << TS_PROTO_HTTP);
>     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) & UnixNetVConnection::acceptEvent);
>     //eventProcessor.schedule_imm(vc, getEtype());
>     eventProcessor.schedule_imm_signal(vc, getEtype());
> @@ -384,6 +405,7 @@ NetAccept::acceptFastEvent(int event, void *ep)
>   (void) event;
>   (void) e;
>   int bufsz, res;
> +  Connection con;
> 
>   PollDescriptor *pd = get_PollDescriptor(e->ethread);
>   UnixNetVConnection *vc = NULL;
> @@ -394,10 +416,10 @@ NetAccept::acceptFastEvent(int event, void *ep)
>       ifd = -1;
>       return EVENT_CONT;
>     }
> -    vc = allocateThread(e->ethread);
> 
> -    socklen_t sz = sizeof(vc->con.addr);
> -    int fd = socketManager.accept(server.fd, &vc->con.addr.sa, &sz);
> +    socklen_t sz = sizeof(con.addr);
> +    int fd = socketManager.accept(server.fd, &con.addr.sa, &sz);
> +    con.fd = fd;
> 
>     if (likely(fd >= 0)) {
>       Debug("iocore_net", "accepted a new socket: %d", fd);
> @@ -443,6 +465,11 @@ NetAccept::acceptFastEvent(int event, void *ep)
>       do {
>         res = safe_nonblocking(fd);
>       } while (res < 0 && (errno == EAGAIN || errno == EINTR));
> +
> +      vc = createSuitableVC(e->ethread, con);
> +      if (!vc)
> +        goto Ldone;
> +
>     } else {
>       res = fd;
>     }
> @@ -453,20 +480,15 @@ NetAccept::acceptFastEvent(int event, void *ep)
>           || res == -EPIPE
> #endif
>         ) {
> -        ink_assert(vc->con.fd == NO_FD);
> -        ink_assert(!vc->link.next && !vc->link.prev);
> -        freeThread(vc, e->ethread);
>         goto Ldone;
>       } else if (accept_error_seriousness(res) >= 0) {
>         check_transient_accept_error(res);
> -        freeThread(vc, e->ethread);
>         goto Ldone;
>       }
>       if (!action_->cancelled)
>         action_->continuation->handleEvent(EVENT_ERROR, (void *)(intptr_t)res);
>       goto Lerror;
>     }
> -    vc->con.fd = fd;
> 
>     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
>     vc->id = net_next_connection_number();
> @@ -478,7 +500,6 @@ NetAccept::acceptFastEvent(int event, void *ep)
>     vc->thread = e->ethread;
> 
>     vc->nh = get_NetHandler(e->ethread);
> -    vc->proto_stack = (1u << TS_PROTO_HTTP);
> 
>     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) & UnixNetVConnection::mainEvent);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetProcessor.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/UnixNetProcessor.cc b/iocore/net/UnixNetProcessor.cc
> index eaae439..ce59bbe 100644
> --- a/iocore/net/UnixNetProcessor.cc
> +++ b/iocore/net/UnixNetProcessor.cc
> @@ -46,6 +46,7 @@ NetProcessor::AcceptOptions::reset()
>   packet_mark = 0;
>   packet_tos = 0;
>   f_inbound_transparent = false;
> +  create_default_NetAccept = true;
>   return *this;
> }
> 
> @@ -84,16 +85,22 @@ NetProcessor::main_accept(Continuation *cont, SOCKET fd, AcceptOptions const& op
> Action *
> UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions const& opt)
> {
> -  EventType et = opt.etype; // setEtype requires non-const ref.
> -  NetAccept *na = createNetAccept();
> +  EventType upgraded_etype = opt.etype; // setEtype requires non-const ref.
> +  AcceptCont *acceptCont = static_cast<AcceptCont *>(cont);
>   EThread *thread = this_ethread();
>   ProxyMutex *mutex = thread->mutex;
>   int accept_threads = opt.accept_threads; // might be changed.
>   IpEndpoint accept_ip; // local binding address.
>   char thr_name[MAX_THREAD_NAME_LENGTH];
> 
> +  NetAccept *na;
> +  if (opt.create_default_NetAccept)
> +    na = createNetAccept();
> +  else
> +    na = (NetAccept *)acceptCont->createNetAccept();
> +
>   // Potentially upgrade to SSL.
> -  upgradeEtype(et);
> +  upgradeEtype(upgraded_etype);
> 
>   // Fill in accept thread from configuration if necessary.
>   if (opt.accept_threads < 0) {
> @@ -137,7 +144,7 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
>   na->sockopt_flags = opt.sockopt_flags;
>   na->packet_mark = opt.packet_mark;
>   na->packet_tos = opt.packet_tos;
> -  na->etype = opt.etype;
> +  na->etype = upgraded_etype;
>   na->backdoor = opt.backdoor;
>   if (na->callback_on_open)
>     na->mutex = cont->mutex;
> @@ -147,7 +154,10 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
>         NetAccept *a;
> 
>         for (int i=1; i < accept_threads; ++i) {
> -          a = createNetAccept();
> +          if (opt.create_default_NetAccept)
> +            a = createNetAccept();
> +          else
> +            a = (NetAccept *)acceptCont->createNetAccept();
>           *a = *na;
>           snprintf(thr_name, MAX_THREAD_NAME_LENGTH, "[ACCEPT %d:%d]", i-1, ats_ip_port_host_order(&accept_ip));
>           a->init_accept_loop(thr_name);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
> index bf1dc0a..8a919c5 100644
> --- a/iocore/net/UnixNetVConnection.cc
> +++ b/iocore/net/UnixNetVConnection.cc
> @@ -38,6 +38,84 @@ typedef struct iovec IOVec;
> #define NET_MAX_IOV UIO_MAXIOV
> #endif
> 
> +struct SpdyProbeCont:public Continuation
> +{
> +  MIOBuffer buf;
> +  unsigned char data;
> +  SpdyProbeCont(): data(0)
> +  {
> +    SET_HANDLER(&SpdyProbeCont::mainEvent);
> +  }
> +
> +  int mainEvent(int event, void *e);
> +};
> +
> +static ClassAllocator<SpdyProbeCont> spdyProberContAllocator("spdyProberContAllocator");
> +
> +SpdyProbeCont *
> +new_SpdyProbeCont(UnixNetVConnection *vc)
> +{
> +  SpdyProbeCont *c = spdyProberContAllocator.alloc();
> +  c->buf.clear();
> +  c->buf.set(&c->data, sizeof c->data);
> +  c->buf._writer->fill(-(sizeof c->data));
> +  c->mutex = vc->mutex;
> +  return c;
> +}
> +void
> +free_SpdyProbeCont(SpdyProbeCont *c)
> +{
> +  c->mutex.clear();
> +  c->buf.clear();
> +  spdyProberContAllocator.free(c);
> +}
> +
> +inline int
> +SpdyProbeCont::mainEvent(int event, void *e) {
> +  UnixNetVConnection *vc = (UnixNetVConnection *) ((VIO *) e)->vc_server;
> +  vc->probe_state = SPDY_PROBE_STATE_END;
> +
> +  switch (event) {
> +  case VC_EVENT_EOS:
> +  case VC_EVENT_ERROR:
> +  case VC_EVENT_INACTIVITY_TIMEOUT:
> +  case VC_EVENT_ACTIVE_TIMEOUT:
> +    vc->do_io_close();
> +    free_SpdyProbeCont(this);
> +    return EVENT_DONE;
> +  case VC_EVENT_READ_COMPLETE:
> +    if ((data & 0x80) != 0) {
> +      //
> +      // SPDY Request
> +      //
> +      free_SpdyProbeCont(this);
> +      vc->proto_stack = (1u << TS_PROTO_SPDY);
> +      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
> +      return EVENT_DONE;
> +    } else {
> +      //
> +      // HTTP Request
> +      //
> +      free_SpdyProbeCont(this);
> +      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
> +      return EVENT_DONE;
> +    }
> +  default:
> +    ink_release_assert(!"unexpected event");
> +  }
> +  return EVENT_CONT;
> +}
> +
> +int SpdyProbeStart(UnixNetVConnection *vc)
> +{
> +  SpdyProbeCont *spdyProbeCont= new_SpdyProbeCont(vc);
> +  //
> +  // TODO: make it configurable
> +  //
> +  vc->set_inactivity_timeout(HRTIME_SECONDS(30));
> +  vc->do_io_read(spdyProbeCont, 1, &spdyProbeCont->buf);
> +  return EVENT_CONT;
> +}
> // Global
> ClassAllocator<UnixNetVConnection> netVCAllocator("netVCAllocator");
> 
> @@ -259,8 +337,12 @@ read_from_net(NetHandler *nh, UnixNetVConnection *vc, EThread *thread)
>         }
>         b = b->next;
>       }
> +      ink_assert(vc->probe_state != SPDY_PROBE_STATE_BEGIN || niov == 1);
>       if (niov == 1) {
> -        r = socketManager.read(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len);
> +        if (vc->probe_state == SPDY_PROBE_STATE_BEGIN) {
> +          r = recv(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len, MSG_PEEK);
> +        } else
> +          r = socketManager.read(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len);
>       } else {
>         r = socketManager.readv(vc->con.fd, &tiovec[0], niov);
>       }
> @@ -801,7 +883,8 @@ UnixNetVConnection::UnixNetVConnection()
> #endif
>     active_timeout(NULL), nh(NULL),
>     id(0), flags(0), recursion(0), submit_time(0), oob_ptr(0),
> -    from_accept_thread(false)
> +    from_accept_thread(false), probe_state(SPDY_PROBE_STATE_NONE),
> +    selected_next_protocol(NULL)
> {
>   memset(&local_addr, 0, sizeof local_addr);
>   memset(&server_addr, 0, sizeof server_addr);
> @@ -988,7 +1071,13 @@ UnixNetVConnection::acceptEvent(int event, Event *e)
>     UnixNetVConnection::set_inactivity_timeout(inactivity_timeout_in);
>   if (active_timeout_in)
>     UnixNetVConnection::set_active_timeout(active_timeout_in);
> -  action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
> +  if (probe_state == SPDY_PROBE_STATE_NONE)
> +    action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
> +  else {
> +    ink_assert(probe_state == SPDY_PROBE_STATE_BEGIN);
> +    SpdyProbeStart(this);
> +  }
> +
>   return EVENT_DONE;
> }
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/lib/ts/apidefs.h.in
> ----------------------------------------------------------------------
> diff --git a/lib/ts/apidefs.h.in b/lib/ts/apidefs.h.in
> index 86a650f..0356921 100644
> --- a/lib/ts/apidefs.h.in
> +++ b/lib/ts/apidefs.h.in
> @@ -117,6 +117,9 @@ extern "C"
>     TS_PROTO_SPDY = 13,
>     TS_PROTO_RTMP = 14,
>     TS_PROTO_WBSK = 15, /* WebSocket */
> +
> +    /* MAX value of TSProtoType  */
> +    TS_PROTO_MAX
>   } TSProtoType;
> 
>   typedef uint32_t TSClientProtoStack;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/InkAPI.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
> index 00a78e1..682f889 100644
> --- a/proxy/InkAPI.cc
> +++ b/proxy/InkAPI.cc
> @@ -51,7 +51,7 @@
> #include "PluginVC.h"
> #include "api/ts/experimental.h"
> #include "ICP.h"
> -#include "HttpAccept.h"
> +#include "HttpAcceptCont.h"
> #include "PluginVC.h"
> #include "FetchSM.h"
> #include "HttpDebugNames.h"
> @@ -6082,8 +6082,8 @@ TSHttpAltInfoQualitySet(TSHttpAltInfo infop, float quality)
>   info->m_qvalue = quality;
> }
> 
> -extern HttpAccept *plugin_http_accept;
> -extern HttpAccept *plugin_http_transparent_accept;
> +extern HttpAcceptCont *plugin_http_accept;
> +extern HttpAcceptCont *plugin_http_transparent_accept;
> 
> TSVConn
> TSHttpConnect(sockaddr const* addr)
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/proxy/Makefile.am b/proxy/Makefile.am
> index 8ef6104..6dfc816 100644
> --- a/proxy/Makefile.am
> +++ b/proxy/Makefile.am
> @@ -17,7 +17,7 @@
> #  limitations under the License.
> 
> # Note that hdrs is targeted from ../Makefile.am
> -SUBDIRS = congest http logging config
> +SUBDIRS = congest http spdy logging config
> noinst_LIBRARIES =
> bin_PROGRAMS = \
>   traffic_server \
> @@ -52,11 +52,10 @@ noinst_HEADERS = \
>   ConfigParse.h \
>   Show.h
> 
> -
> if STATIC_LIBTS
> -   which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
> +    which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
> else
> -   which_libts = $(top_builddir)/lib/ts/libtsutil.la
> +    which_libts = $(top_builddir)/lib/ts/libtsutil.la
> endif
> 
> EXTRA_DIST = InkAPITestTool.cc example_alarm_bin.sh example_prep.sh
> @@ -121,6 +120,7 @@ endif
> traffic_server_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
> traffic_server_LDADD = \
>   http/libhttp.a \
> +  spdy/libspdy.a \
>   http/remap/libhttp_remap.a \
>   congest/libCongestionControl.a \
>   logging/liblogging.a \
> @@ -157,6 +157,7 @@ traffic_server_LDADD = \
>   @LIBZ@ \
>   @LIBLZMA@ \
>   @LIBPROFILER@ \
> +  @SPDYLAY_LIBS@ \
>   -lm
> 
> if BUILD_LUA_SUPPORT
> @@ -223,6 +224,7 @@ traffic_sac_SOURCES = \
> traffic_sac_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
> traffic_sac_LDADD = \
>   http/libhttp.a \
> +  spdy/libspdy.a \
>   shared/libdiagsconfig.a \
>   http/remap/libhttp_remap.a \
>   congest/libCongestionControl.a \
> @@ -247,7 +249,7 @@ traffic_sac_LDADD = \
>   $(top_builddir)/lib/records/librecprocess.a \
>   $(top_builddir)/lib/ts/libtsutil.la \
>   @LIBRESOLV@ @LIBPCRE@ @OPENSSL_LIBS@ @LIBTCL@ @hwloc_LIBS@ \
> -  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@ -lm
> +  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@ @SPDYLAY_LIBS@ -lm
> 
> if BUILD_TESTS
>   traffic_sac_SOURCES += RegressionSM.cc
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/SocksProxy.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/SocksProxy.cc b/proxy/SocksProxy.cc
> index e447f12..d855689 100644
> --- a/proxy/SocksProxy.cc
> +++ b/proxy/SocksProxy.cc
> @@ -31,7 +31,7 @@
> #include "StatSystem.h"
> #include "P_Net.h"
> #include "I_OneWayTunnel.h"
> -#include "HttpAccept.h"
> +#include "HttpAcceptCont.h"
> 
> enum
> {
> @@ -145,14 +145,14 @@ SocksProxy::mainEvent(int event, void *data)
> 
>     switch (state) {
>     case HTTP_REQ:{
> -      HttpAccept::Options ha_opt;
> +      HttpAcceptCont::Options ha_opt;
>       //This is a WRITE_COMPLETE. vio->nbytes == vio->ndone is true
> 
>       SOCKSPROXY_INC_STAT(socksproxy_http_connections_stat);
>       Debug("SocksProxy", "Handing over the HTTP request\n");
> 
>       ha_opt.transport_type = clientVC->attributes;
> -      HttpAccept http_accept(ha_opt);
> +      HttpAcceptCont http_accept(ha_opt);
>       http_accept.mainEvent(NET_EVENT_ACCEPT, clientVC);
>       state = ALL_DONE;
>       break;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/api/ts/InkAPIPrivateIOCore.h
> ----------------------------------------------------------------------
> diff --git a/proxy/api/ts/InkAPIPrivateIOCore.h b/proxy/api/ts/InkAPIPrivateIOCore.h
> index 26e3fac..97a9bcd 100644
> --- a/proxy/api/ts/InkAPIPrivateIOCore.h
> +++ b/proxy/api/ts/InkAPIPrivateIOCore.h
> @@ -51,6 +51,10 @@ public:
> 
>   void handle_event_count(int event);
>   int handle_event(int event, void *edata);
> +  virtual void *createNetAccept()
> +  {
> +    return (NEW(new NetAccept));
> +  }
> 
> public:
>   void *mdata;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAccept.cc b/proxy/http/HttpAccept.cc
> deleted file mode 100644
> index 9fb0066..0000000
> --- a/proxy/http/HttpAccept.cc
> +++ /dev/null
> @@ -1,98 +0,0 @@
> -/** @file
> -
> -  A brief file description
> -
> -  @section license License
> -
> -  Licensed to the Apache Software Foundation (ASF) under one
> -  or more contributor license agreements.  See the NOTICE file
> -  distributed with this work for additional information
> -  regarding copyright ownership.  The ASF licenses this file
> -  to you under the Apache License, Version 2.0 (the
> -  "License"); you may not use this file except in compliance
> -  with the License.  You may obtain a copy of the License at
> -
> -      http://www.apache.org/licenses/LICENSE-2.0
> -
> -  Unless required by applicable law or agreed to in writing, software
> -  distributed under the License is distributed on an "AS IS" BASIS,
> -  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> -  See the License for the specific language governing permissions and
> -  limitations under the License.
> - */
> -
> -#include "HttpAccept.h"
> -#include "IPAllow.h"
> -#include "HttpClientSession.h"
> -#include "I_Machine.h"
> -#include "Error.h"
> -
> -int
> -HttpAccept::mainEvent(int event, void *data)
> -{
> -  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> -  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
> -
> -  if (event == NET_EVENT_ACCEPT) {
> -    ////////////////////////////////////////////////////
> -    // if client address forbidden, close immediately //
> -    ////////////////////////////////////////////////////
> -    NetVConnection *netvc = static_cast<NetVConnection *>(data);
> -    sockaddr const* client_ip = netvc->get_remote_addr();
> -    uint32_t acl_method_mask = 0;
> -    ip_port_text_buffer ipb;
> -    IpAllow::scoped_config ipallow;
> -
> -    // The backdoor port is now only bound to "localhost", so no
> -    // reason to check for if it's incoming from "localhost" or not.
> -    if (backdoor) {
> -      acl_method_mask = IpAllow::AllMethodMask();
> -    } else if (ipallow && ((acl_method_mask = ipallow->match(client_ip)) == 0)) {
> -      Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
> -      netvc->do_io_close();
> -
> -      return VC_EVENT_CONT;
> -    }
> -
> -    netvc->attributes = transport_type;
> -
> -    if (is_debug_tag_set("http_seq"))
> -      Debug("http_seq", "[HttpAccept:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
> -
> -    HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
> -
> -   // copy over session related data.
> -    new_session->f_outbound_transparent = f_outbound_transparent;
> -    new_session->f_transparent_passthrough = f_transparent_passthrough;
> -    new_session->outbound_ip4 = outbound_ip4;
> -    new_session->outbound_ip6 = outbound_ip6;
> -    new_session->outbound_port = outbound_port;
> -    new_session->host_res_style = ats_host_res_from(client_ip->sa_family, host_res_preference);
> -    new_session->acl_method_mask = acl_method_mask;
> -
> -    new_session->new_connection(netvc, backdoor);
> -
> -    return EVENT_CONT;
> -  }
> -
> -  /////////////////
> -  // EVENT_ERROR //
> -  /////////////////
> -  if (((long) data) == -ECONNABORTED) {
> -    /////////////////////////////////////////////////
> -    // Under Solaris, when accept() fails and sets //
> -    // errno to EPROTO, it means the client has    //
> -    // sent a TCP reset before the connection has  //
> -    // been accepted by the server...  Note that   //
> -    // in 2.5.1 with the Internet Server Supplement//
> -    // and also in 2.6 the errno for this case has //
> -    // changed from EPROTO to ECONNABORTED.        //
> -    /////////////////////////////////////////////////
> -
> -    // FIX: add time to user_agent_hangup
> -    HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
> -  }
> -
> -  MachineFatal("HTTP accept received fatal error: errno = %d", -((int)(intptr_t)data));
> -  return EVENT_CONT;
> -}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAccept.h b/proxy/http/HttpAccept.h
> deleted file mode 100644
> index 30d68e6..0000000
> --- a/proxy/http/HttpAccept.h
> +++ /dev/null
> @@ -1,198 +0,0 @@
> -/** @file
> -
> -  A brief file description
> -
> -  @section license License
> -
> -  Licensed to the Apache Software Foundation (ASF) under one
> -  or more contributor license agreements.  See the NOTICE file
> -  distributed with this work for additional information
> -  regarding copyright ownership.  The ASF licenses this file
> -  to you under the Apache License, Version 2.0 (the
> -  "License"); you may not use this file except in compliance
> -  with the License.  You may obtain a copy of the License at
> -
> -      http://www.apache.org/licenses/LICENSE-2.0
> -
> -  Unless required by applicable law or agreed to in writing, software
> -  distributed under the License is distributed on an "AS IS" BASIS,
> -  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> -  See the License for the specific language governing permissions and
> -  limitations under the License.
> - */
> -
> -#if !defined (_HttpAccept_h_)
> -#define _HttpAccept_h_
> -
> -#include "libts.h"
> -#include "P_EventSystem.h"
> -#include "HttpConfig.h"
> -#include "HTTP.h"
> -
> -namespace detail {
> -  /** Options for @c HttpAccept.
> -
> -      @internal This is done as a separate class for two reasons.
> -
> -      The first is that in current usage many instances are created
> -      with the same options so (for the client) this is easier and
> -      more efficient than passing options directly to the @c
> -      HttpAccept or calling setters.
> -
> -      The second is that @c HttpAccept is not provided with any thread
> -      safety because it is intended as an immutable object. Putting
> -      the setters here and not there makes that clearer.
> -
> -      We don't do this directly as nested class because we want to
> -      inherit the data members rather than duplicate the declarations
> -      and initializations.
> -   */
> -  class HttpAcceptOptions {
> -  private:
> -    typedef HttpAcceptOptions self; ///< Self reference type.
> -  public:
> -    HttpAcceptOptions();
> -
> -    // Connection type (HttpProxyPort::TransportType)
> -    int transport_type;
> -    /// Set the transport type.
> -    self& setTransportType(int);
> -    /// Local address to bind for outbound connections.
> -    IpAddr outbound_ip4;
> -    /// Local address to bind for outbound connections.
> -    IpAddr outbound_ip6;
> -    /// Set the outbound IP address to @a ip.
> -    self& setOutboundIp(IpAddr& ip);
> -    /// Set the outbound IP address to @a ip.
> -    self& setOutboundIp(IpEndpoint* ip);
> -    /// Local port for outbound connection.
> -    uint16_t outbound_port;
> -    /// Set outbound port.
> -    self& setOutboundPort(uint16_t);
> -    /// Outbound transparent.
> -    bool f_outbound_transparent;
> -    /// Set outbound transparency.
> -    self& setOutboundTransparent(bool);
> -    /// Transparent pass-through.
> -    bool f_transparent_passthrough;
> -    /// Set transparent passthrough.
> -    self& setTransparentPassthrough(bool);
> -    /// Accepting backdoor connections.
> -    bool backdoor;
> -    /// Set backdoor accept.
> -    self& setBackdoor(bool);
> -    /// Host address resolution preference order.
> -    HostResPreferenceOrder host_res_preference;
> -    /// Set the host query preference.
> -    self& setHostResPreference(HostResPreferenceOrder const);
> -  };
> -
> -  inline HttpAcceptOptions::HttpAcceptOptions()
> -    : transport_type(0)
> -    , outbound_port(0)
> -    , f_outbound_transparent(false)
> -    , f_transparent_passthrough(false)
> -    , backdoor(false)
> -  {
> -    memcpy(host_res_preference, host_res_default_preference_order, sizeof(host_res_preference));
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setTransportType(int type) {
> -    transport_type =  type;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundIp(IpAddr& ip) {
> -    if (ip.isIp4()) outbound_ip4 = ip;
> -    else if (ip.isIp6()) outbound_ip6 = ip;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundIp(IpEndpoint* ip) {
> -    if (ip->isIp4()) outbound_ip4 = *ip;
> -    else if (ip->isIp6()) outbound_ip6 = *ip;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundPort(uint16_t port) {
> -    outbound_port = port;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundTransparent(bool flag) {
> -    f_outbound_transparent = flag;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setTransparentPassthrough(bool flag) {
> -    f_transparent_passthrough = flag;
> -    return *this;
> -  }
> -
> - inline HttpAcceptOptions&
> -  HttpAcceptOptions::setBackdoor(bool flag) {
> -    backdoor = flag;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setHostResPreference(HostResPreferenceOrder const order) {
> -    memcpy(host_res_preference, order, sizeof(host_res_preference));
> -    return *this;
> -  }
> -}
> -
> -/**
> -   The continuation mutex is NULL to allow parellel accepts in NT. No
> -   state is recorded by the handler and values are required to be set
> -   during construction via the @c Options struct and never changed. So
> -   a NULL mutex is safe.
> -
> -   Most of the state is simply passed on to the @c ClientSession after
> -   an accept. It is done here because this is the least bad pathway
> -   from the top level configuration to the HTTP session.
> -*/
> -
> -class HttpAccept: public Continuation, private detail::HttpAcceptOptions
> -{
> -private:
> -  typedef HttpAccept self; ///< Self reference type.
> -public:
> -  /** Construction options.
> -      Provide an easier to remember typedef for clients.
> -  */
> -  typedef detail::HttpAcceptOptions Options;
> -
> -  /** Default constructor.
> -    
> -      @internal We don't use a static default options object because of
> -      initialization order issues. It is important to pick up data that is read
> -      from the config file and a static is initialized long before that point.
> -  */
> -  HttpAccept(Options const& opt = Options())
> -    : Continuation(NULL)
> -    , detail::HttpAcceptOptions(opt) // copy these.
> -  {
> -    SET_HANDLER(&HttpAccept::mainEvent);
> -    return;
> -  }
> -
> -  ~HttpAccept()
> -  {
> -    return;
> -  }
> -
> -  int mainEvent(int event, void *netvc);
> -
> -private:
> -    HttpAccept(const HttpAccept &);
> -    HttpAccept & operator =(const HttpAccept &);
> -};
> -
> -#endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAcceptCont.cc b/proxy/http/HttpAcceptCont.cc
> new file mode 100644
> index 0000000..d3744e7
> --- /dev/null
> +++ b/proxy/http/HttpAcceptCont.cc
> @@ -0,0 +1,98 @@
> +/** @file
> +
> +  A brief file description
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "HttpAcceptCont.h"
> +#include "IPAllow.h"
> +#include "HttpClientSession.h"
> +#include "I_Machine.h"
> +#include "Error.h"
> +
> +int
> +HttpAcceptCont::mainEvent(int event, void *data)
> +{
> +  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> +  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
> +
> +  if (event == NET_EVENT_ACCEPT) {
> +    ////////////////////////////////////////////////////
> +    // if client address forbidden, close immediately //
> +    ////////////////////////////////////////////////////
> +    NetVConnection *netvc = static_cast<NetVConnection *>(data);
> +    sockaddr const* client_ip = netvc->get_remote_addr();
> +    uint32_t acl_method_mask = 0;
> +    ip_port_text_buffer ipb;
> +    IpAllow::scoped_config ipallow;
> +
> +    // The backdoor port is now only bound to "localhost", so no
> +    // reason to check for if it's incoming from "localhost" or not.
> +    if (backdoor) {
> +      acl_method_mask = IpAllow::AllMethodMask();
> +    } else if (ipallow && ((acl_method_mask = ipallow->match(client_ip)) == 0)) {
> +      Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
> +      netvc->do_io_close();
> +
> +      return VC_EVENT_CONT;
> +    }
> +
> +    netvc->attributes = transport_type;
> +
> +    if (is_debug_tag_set("http_seq"))
> +      Debug("http_seq", "[HttpAcceptCont:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
> +
> +    HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
> +
> +   // copy over session related data.
> +    new_session->f_outbound_transparent = f_outbound_transparent;
> +    new_session->f_transparent_passthrough = f_transparent_passthrough;
> +    new_session->outbound_ip4 = outbound_ip4;
> +    new_session->outbound_ip6 = outbound_ip6;
> +    new_session->outbound_port = outbound_port;
> +    new_session->host_res_style = ats_host_res_from(client_ip->sa_family, host_res_preference);
> +    new_session->acl_method_mask = acl_method_mask;
> +
> +    new_session->new_connection(netvc, backdoor);
> +
> +    return EVENT_CONT;
> +  }
> +
> +  /////////////////
> +  // EVENT_ERROR //
> +  /////////////////
> +  if (((long) data) == -ECONNABORTED) {
> +    /////////////////////////////////////////////////
> +    // Under Solaris, when accept() fails and sets //
> +    // errno to EPROTO, it means the client has    //
> +    // sent a TCP reset before the connection has  //
> +    // been accepted by the server...  Note that   //
> +    // in 2.5.1 with the Internet Server Supplement//
> +    // and also in 2.6 the errno for this case has //
> +    // changed from EPROTO to ECONNABORTED.        //
> +    /////////////////////////////////////////////////
> +
> +    // FIX: add time to user_agent_hangup
> +    HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
> +  }
> +
> +  MachineFatal("HTTP accept received fatal error: errno = %d", -((int)(intptr_t)data));
> +  return EVENT_CONT;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAcceptCont.h b/proxy/http/HttpAcceptCont.h
> new file mode 100644
> index 0000000..5b8c437
> --- /dev/null
> +++ b/proxy/http/HttpAcceptCont.h
> @@ -0,0 +1,198 @@
> +/** @file
> +
> +  A brief file description
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#if !defined (_HttpAcceptCont_h_)
> +#define _HttpAcceptCont_h_
> +
> +#include "libts.h"
> +#include "P_EventSystem.h"
> +#include "HttpConfig.h"
> +#include "HTTP.h"
> +#include "I_Net.h"
> +
> +namespace detail {
> +  /** Options for @c HttpAcceptCont.
> +
> +      @internal This is done as a separate class for two reasons.
> +
> +      The first is that in current usage many instances are created
> +      with the same options so (for the client) this is easier and
> +      more efficient than passing options directly to the @c
> +      HttpAcceptCont or calling setters.
> +
> +      The second is that @c HttpAcceptCont is not provided with any thread
> +      safety because it is intended as an immutable object. Putting
> +      the setters here and not there makes that clearer.
> +
> +      We don't do this directly as nested class because we want to
> +      inherit the data members rather than duplicate the declarations
> +      and initializations.
> +   */
> +  class HttpAcceptContOptions {
> +  private:
> +    typedef HttpAcceptContOptions self; ///< Self reference type.
> +  public:
> +    HttpAcceptContOptions();
> +
> +    // Connection type (HttpProxyPort::TransportType)
> +    int transport_type;
> +    /// Set the transport type.
> +    self& setTransportType(int);
> +    /// Local address to bind for outbound connections.
> +    IpAddr outbound_ip4;
> +    /// Local address to bind for outbound connections.
> +    IpAddr outbound_ip6;
> +    /// Set the outbound IP address to @a ip.
> +    self& setOutboundIp(IpAddr& ip);
> +    /// Set the outbound IP address to @a ip.
> +    self& setOutboundIp(IpEndpoint* ip);
> +    /// Local port for outbound connection.
> +    uint16_t outbound_port;
> +    /// Set outbound port.
> +    self& setOutboundPort(uint16_t);
> +    /// Outbound transparent.
> +    bool f_outbound_transparent;
> +    /// Set outbound transparency.
> +    self& setOutboundTransparent(bool);
> +    /// Transparent pass-through.
> +    bool f_transparent_passthrough;
> +    /// Set transparent passthrough.
> +    self& setTransparentPassthrough(bool);
> +    /// Accepting backdoor connections.
> +    bool backdoor;
> +    /// Set backdoor accept.
> +    self& setBackdoor(bool);
> +    /// Host address resolution preference order.
> +    HostResPreferenceOrder host_res_preference;
> +    /// Set the host query preference.
> +    self& setHostResPreference(HostResPreferenceOrder const);
> +  };
> +
> +  inline HttpAcceptContOptions::HttpAcceptContOptions()
> +    : transport_type(0)
> +    , outbound_port(0)
> +    , f_outbound_transparent(false)
> +    , f_transparent_passthrough(false)
> +    , backdoor(false)
> +  {
> +    memcpy(host_res_preference, host_res_default_preference_order, sizeof(host_res_preference));
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setTransportType(int type) {
> +    transport_type =  type;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundIp(IpAddr& ip) {
> +    if (ip.isIp4()) outbound_ip4 = ip;
> +    else if (ip.isIp6()) outbound_ip6 = ip;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundIp(IpEndpoint* ip) {
> +    if (ip->isIp4()) outbound_ip4 = *ip;
> +    else if (ip->isIp6()) outbound_ip6 = *ip;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundPort(uint16_t port) {
> +    outbound_port = port;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundTransparent(bool flag) {
> +    f_outbound_transparent = flag;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setTransparentPassthrough(bool flag) {
> +    f_transparent_passthrough = flag;
> +    return *this;
> +  }
> +
> + inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setBackdoor(bool flag) {
> +    backdoor = flag;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setHostResPreference(HostResPreferenceOrder const order) {
> +    memcpy(host_res_preference, order, sizeof(host_res_preference));
> +    return *this;
> +  }
> +}
> +
> +/**
> +   The continuation mutex is NULL to allow parellel accepts in NT. No
> +   state is recorded by the handler and values are required to be set
> +   during construction via the @c Options struct and never changed. So
> +   a NULL mutex is safe.
> +
> +   Most of the state is simply passed on to the @c ClientSession after
> +   an accept. It is done here because this is the least bad pathway
> +   from the top level configuration to the HTTP session.
> +*/
> +
> +class HttpAcceptCont: public AcceptCont, private detail::HttpAcceptContOptions
> +{
> +private:
> +  typedef HttpAcceptCont self; ///< Self reference type.
> +public:
> +  /** Construction options.
> +      Provide an easier to remember typedef for clients.
> +  */
> +  typedef detail::HttpAcceptContOptions Options;
> +
> +  /** Default constructor.
> +      @internal We don't use a static default options object because of
> +      initialization order issues. It is important to pick up data that is read
> +      from the config file and a static is initialized long before that point.
> +  */
> +  HttpAcceptCont(Options const& opt = Options())
> +    : AcceptCont(NULL)
> +    , detail::HttpAcceptContOptions(opt) // copy these.
> +  {
> +    SET_HANDLER(&HttpAcceptCont::mainEvent);
> +    return;
> +  }
> +
> +  ~HttpAcceptCont()
> +  {
> +    return;
> +  }
> +
> +  int mainEvent(int event, void *netvc);
> +
> +private:
> +    HttpAcceptCont(const HttpAcceptCont &);
> +    HttpAcceptCont & operator =(const HttpAcceptCont &);
> +};
> +
> +#endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpProxyServerMain.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
> index 662856a..8988253 100644
> --- a/proxy/http/HttpProxyServerMain.cc
> +++ b/proxy/http/HttpProxyServerMain.cc
> @@ -26,7 +26,7 @@
> #include "Main.h"
> #include "Error.h"
> #include "HttpConfig.h"
> -#include "HttpAccept.h"
> +#include "HttpAcceptCont.h"
> #include "ReverseProxy.h"
> #include "HttpSessionManager.h"
> #include "HttpUpdateSM.h"
> @@ -35,9 +35,11 @@
> #include "HttpTunnel.h"
> #include "Tokenizer.h"
> #include "P_SSLNextProtocolAccept.h"
> +#include "P_ProtocolAcceptCont.h"
> +#include "P_SpdyAcceptCont.h"
> 
> -HttpAccept *plugin_http_accept = NULL;
> -HttpAccept *plugin_http_transparent_accept = 0;
> +HttpAcceptCont *plugin_http_accept = NULL;
> +HttpAcceptCont *plugin_http_transparent_accept = 0;
> 
> static SLL<SSLNextProtocolAccept> ssl_plugin_acceptors;
> static ink_mutex ssl_plugin_mutex = PTHREAD_MUTEX_INITIALIZER;
> @@ -135,9 +137,10 @@ static void
> MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned nthreads)
> {
>   NetProcessor::AcceptOptions& net_opt = acceptor._net_opt;
> -  HttpAccept::Options         accept_opt;
> +  HttpAcceptCont::Options         accept_opt;
> 
>   net_opt = make_net_accept_options(port, nthreads);
> +  net_opt.create_default_NetAccept = false;
>   REC_ReadConfigInteger(net_opt.recv_bufsize, "proxy.config.net.sock_recv_buffer_size_in");
>   REC_ReadConfigInteger(net_opt.send_bufsize, "proxy.config.net.sock_send_buffer_size_in");
>   REC_ReadConfigInteger(net_opt.packet_mark, "proxy.config.net.sock_packet_mark_in");
> @@ -160,21 +163,24 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
>     accept_opt.outbound_ip6 = HttpConfig::m_master.outbound_ip6;
>   }
> 
> -  if (port.isSSL()) {
> -    HttpAccept * accept = NEW(new HttpAccept(accept_opt));
> -    SSLNextProtocolAccept * ssl = NEW(new SSLNextProtocolAccept(accept));
> +  HttpAcceptCont *http = NEW(new HttpAcceptCont(accept_opt));
> +  SpdyAcceptCont *spdy = NEW(new SpdyAcceptCont(http));
> +  SSLNextProtocolAccept *ssl = NEW(new SSLNextProtocolAccept(http));
> +  ProtocolAcceptCont *proto = NEW(new ProtocolAcceptCont());
> +
> +  proto->registerEndpoint(TS_PROTO_TLS, ssl);
> +  proto->registerEndpoint(TS_PROTO_HTTP, http);
> +  proto->registerEndpoint(TS_PROTO_SPDY, spdy);
> 
> +  if (port.isSSL()) {
>     // ALPN selects the first server-offered protocol, so make sure that we offer HTTP/1.1 first.
> -    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, accept);
> -    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, accept);
> +    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, http);
> +    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, http);
> 
>     ink_scoped_mutex lock(ssl_plugin_mutex);
>     ssl_plugin_acceptors.push(ssl);
> -
> -    acceptor._accept = ssl;
> -  } else {
> -    acceptor._accept = NEW(new HttpAccept(accept_opt));
>   }
> +  acceptor._accept = proto;
> }
> 
> /** Set up all the accepts and sockets.
> @@ -197,14 +203,14 @@ init_HttpProxyServer(int n_accept_threads)
>   //   port but without going through the operating system
>   //
>   if (plugin_http_accept == NULL) {
> -    plugin_http_accept = NEW(new HttpAccept);
> +    plugin_http_accept = NEW(new HttpAcceptCont);
>     plugin_http_accept->mutex = new_ProxyMutex();
>   }
>   // Same as plugin_http_accept except outbound transparent.
>   if (! plugin_http_transparent_accept) {
> -    HttpAccept::Options ha_opt;
> +    HttpAcceptCont::Options ha_opt;
>     ha_opt.setOutboundTransparent(true);
> -    plugin_http_transparent_accept = NEW(new HttpAccept(ha_opt));
> +    plugin_http_transparent_accept = NEW(new HttpAcceptCont(ha_opt));
>     plugin_http_transparent_accept->mutex = new_ProxyMutex();
>   }
>   ink_mutex_init(&ssl_plugin_mutex, "SSL Acceptor List");
> @@ -262,7 +268,7 @@ void
> start_HttpProxyServerBackDoor(int port, int accept_threads)
> {
>   NetProcessor::AcceptOptions opt;
> -  HttpAccept::Options ha_opt;
> +  HttpAcceptCont::Options ha_opt;
> 
>   opt.local_port = port;
>   opt.accept_threads = accept_threads;
> @@ -271,5 +277,5 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
>   opt.backdoor = true;
> 
>   // The backdoor only binds the loopback interface
> -  netProcessor.main_accept(NEW(new HttpAccept(ha_opt)), NO_FD, opt);
> +  netProcessor.main_accept(NEW(new HttpAcceptCont(ha_opt)), NO_FD, opt);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/proxy/http/Makefile.am b/proxy/http/Makefile.am
> index c413ed7..a71d287 100644
> --- a/proxy/http/Makefile.am
> +++ b/proxy/http/Makefile.am
> @@ -31,14 +31,15 @@ AM_CPPFLAGS = \
>   -I$(top_srcdir)/proxy/hdrs \
>   -I$(top_srcdir)/proxy/shared \
>   -I$(top_srcdir)/proxy/http/remap \
> -  -I$(top_srcdir)/proxy/logging
> +  -I$(top_srcdir)/proxy/logging \
> +  -I$(top_srcdir)/proxy/spdy
> 
> noinst_HEADERS = HttpProxyServerMain.h
> noinst_LIBRARIES = libhttp.a
> 
> libhttp_a_SOURCES = \
> -  HttpAccept.cc \
> -  HttpAccept.h \
> +  HttpAcceptCont.cc \
> +  HttpAcceptCont.h \
>   HttpBodyFactory.cc \
>   HttpBodyFactory.h \
>   HttpCacheSM.cc \
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/Makefile.am b/proxy/spdy/Makefile.am
> new file mode 100644
> index 0000000..63911e1
> --- /dev/null
> +++ b/proxy/spdy/Makefile.am
> @@ -0,0 +1,38 @@
> +# Makefile.am for the traffic/iocore/net hierarchy
> +#
> +#  Licensed to the Apache Software Foundation (ASF) under one
> +#  or more contributor license agreements.  See the NOTICE file
> +#  distributed with this work for additional information
> +#  regarding copyright ownership.  The ASF licenses this file
> +#  to you under the Apache License, Version 2.0 (the
> +#  "License"); you may not use this file except in compliance
> +#  with the License.  You may obtain a copy of the License at
> +#
> +#      http://www.apache.org/licenses/LICENSE-2.0
> +#
> +#  Unless required by applicable law or agreed to in writing, software
> +#  distributed under the License is distributed on an "AS IS" BASIS,
> +#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +#  See the License for the specific language governing permissions and
> +#  limitations under the License.
> +
> +AM_CPPFLAGS = \
> +  $(iocore_include_dirs) \
> +  -I$(top_srcdir)/lib \
> +  -I$(top_srcdir)/lib/records \
> +  -I$(top_srcdir)/lib/ts \
> +  -I$(top_srcdir)/proxy \
> +  -I$(top_srcdir)/proxy/api \
> +  -I$(top_srcdir)/proxy/hdrs \
> +  -I$(top_srcdir)/proxy/shared \
> +  -I$(top_srcdir)/proxy/spdy \
> +  -I$(top_srcdir)/mgmt \
> +  -I$(top_srcdir)/mgmt/preparse \
> +  -I$(top_srcdir)/mgmt/utils \
> +  @SPDYLAY_CFLAGS@
> +
> +noinst_LIBRARIES = libspdy.a
> +
> +libspdy_a_SOURCES = \
> +  P_SpdyAcceptCont.h \
> +  SpdyAcceptCont.cc
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/P_SpdyAcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/P_SpdyAcceptCont.h b/proxy/spdy/P_SpdyAcceptCont.h
> new file mode 100644
> index 0000000..88aea02
> --- /dev/null
> +++ b/proxy/spdy/P_SpdyAcceptCont.h
> @@ -0,0 +1,46 @@
> +/** @file
> +
> +  SpdyAcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#ifndef P_SpdyAcceptCont_H_
> +#define P_SpdyAcceptCont_H_
> +
> +#include "P_Net.h"
> +#include "P_EventSystem.h"
> +#include "P_UnixNet.h"
> +#include "I_IOBuffer.h"
> +
> +class SpdyAcceptCont: public AcceptCont
> +{
> +public:
> +  SpdyAcceptCont(Continuation *ep);
> +  ~SpdyAcceptCont() {}
> +
> +private:
> +  int mainEvent(int event, void *netvc);
> +  SpdyAcceptCont(const SpdyAcceptCont &); // disabled
> +  SpdyAcceptCont& operator =(const SpdyAcceptCont&); // disabled
> +
> +  Continuation *endpoint;
> +};
> +
> +#endif /* P_SpdyAcceptCont_H_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/SpdyAcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyAcceptCont.cc b/proxy/spdy/SpdyAcceptCont.cc
> new file mode 100644
> index 0000000..b9efc80
> --- /dev/null
> +++ b/proxy/spdy/SpdyAcceptCont.cc
> @@ -0,0 +1,37 @@
> +/** @file
> +
> +  SpdyNetAccept
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "P_SpdyAcceptCont.h"
> +
> +SpdyAcceptCont::SpdyAcceptCont(Continuation *ep)
> +    : AcceptCont(new_ProxyMutex()), endpoint(ep)
> +{
> +  SET_HANDLER(&SpdyAcceptCont::mainEvent);
> +}
> +
> +int
> +SpdyAcceptCont::mainEvent(int event, void *netvc)
> +{
> +  printf("spdy accepted\n");
> +  return 0;
> +}
> 


Re: [2/3] git commit: TS-2431: Preparation of SPDY protocol

Posted by James Peach <jp...@apache.org>.
I just started reviewing this, a few high level comments ...

I don't like the renaming of *Accept to *AcceptCont. This results in names like HttpAcceptCont::Options, which is pretty ugly.

I think we need to separate SPDY probe from the UnixNetAccept better. A good goal to have is for iocore to know nothing at all about SPDY. The way I imagine this is for the generic protocol acceptor to take a table of probe functions or continuations that can detect the next protocol. Enabling detection should be a per-port option.

I'll work on this and other SPDY issues over the coming weeks, hopefully we can keep improving it :)

On Mar 20, 2014, at 10:32 AM, yunkai@apache.org wrote:

> TS-2431: Preparation of SPDY protocol
> 
> *) Create basic data structures for SPDY protocal, such as:
>   SpdyAcceptCont.
> 
> *) SPDY will share the same port number with HTTP protocol, ATS can
>   recognize them by detecting the first byte of client request.
> 
> *) HttpAccept looks like a subclass of NetAccept. To avoid confusing,
>   rename HttpAccept to HttpAcceptCont.
> 
> Signed-off-by: Yunkai Zhang <qi...@taobao.com>
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/f90f3a48
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/f90f3a48
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/f90f3a48
> 
> Branch: refs/heads/master
> Commit: f90f3a488323ccb8b98c5571ef6cf035f1be51a9
> Parents: f1a005e
> Author: Yunkai Zhang <qi...@taobao.com>
> Authored: Wed Dec 11 21:21:53 2013 +0800
> Committer: Yunkai Zhang <qi...@taobao.com>
> Committed: Fri Mar 21 01:29:21 2014 +0800
> 
> ----------------------------------------------------------------------
> configure.ac                         |   1 +
> iocore/net/AcceptCont.cc             |  41 +++++++
> iocore/net/I_AcceptCont.h            |  48 ++++++++
> iocore/net/I_Net.h                   |   1 +
> iocore/net/I_NetProcessor.h          |   1 +
> iocore/net/Makefile.am               |   8 ++
> iocore/net/P_Net.h                   |   1 +
> iocore/net/P_NetAccept.h             |   2 +
> iocore/net/P_ProtocolAcceptCont.h    |  50 ++++++++
> iocore/net/P_ProtocolNetAccept.h     |  62 ++++++++++
> iocore/net/P_SSLNetAccept.h          |   1 +
> iocore/net/P_SSLNextProtocolAccept.h |   2 +-
> iocore/net/P_SSLNextProtocolSet.h    |   2 +-
> iocore/net/P_UnixNetVConnection.h    |   9 ++
> iocore/net/ProtocolAcceptCont.cc     |  73 +++++++++++
> iocore/net/ProtocolNetAccept.cc      |  68 ++++++++++
> iocore/net/SSLNetAccept.cc           |  16 ++-
> iocore/net/SSLNetVConnection.cc      |   3 +-
> iocore/net/SSLNextProtocolAccept.cc  |   2 +-
> iocore/net/SSLNextProtocolSet.cc     |   5 +-
> iocore/net/UnixNetAccept.cc          |  69 +++++++----
> iocore/net/UnixNetProcessor.cc       |  20 ++-
> iocore/net/UnixNetVConnection.cc     |  95 +++++++++++++-
> lib/ts/apidefs.h.in                  |   3 +
> proxy/InkAPI.cc                      |   6 +-
> proxy/Makefile.am                    |  12 +-
> proxy/SocksProxy.cc                  |   6 +-
> proxy/api/ts/InkAPIPrivateIOCore.h   |   4 +
> proxy/http/HttpAccept.cc             |  98 ---------------
> proxy/http/HttpAccept.h              | 198 ------------------------------
> proxy/http/HttpAcceptCont.cc         |  98 +++++++++++++++
> proxy/http/HttpAcceptCont.h          | 198 ++++++++++++++++++++++++++++++
> proxy/http/HttpProxyServerMain.cc    |  42 ++++---
> proxy/http/Makefile.am               |   7 +-
> proxy/spdy/Makefile.am               |  38 ++++++
> proxy/spdy/P_SpdyAcceptCont.h        |  46 +++++++
> proxy/spdy/SpdyAcceptCont.cc         |  37 ++++++
> 37 files changed, 1004 insertions(+), 369 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/configure.ac
> ----------------------------------------------------------------------
> diff --git a/configure.ac b/configure.ac
> index f87674d..796158e 100644
> --- a/configure.ac
> +++ b/configure.ac
> @@ -2010,6 +2010,7 @@ AC_CONFIG_FILES([
>   proxy/http/Makefile
>   proxy/http/remap/Makefile
>   proxy/logging/Makefile
> +  proxy/spdy/Makefile
>   rc/Makefile
>   rc/trafficserver
>   rc/trafficserver.conf
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/AcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/AcceptCont.cc b/iocore/net/AcceptCont.cc
> new file mode 100644
> index 0000000..e112230
> --- /dev/null
> +++ b/iocore/net/AcceptCont.cc
> @@ -0,0 +1,41 @@
> +/** @file
> +
> +  AcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "I_AcceptCont.h"
> +#include "P_Net.h"
> +
> +AcceptCont::AcceptCont(ProxyMutex *amutex)
> +    : Continuation(amutex)
> +{
> +  SET_HANDLER(&AcceptCont::mainEvent);
> +}
> +
> +AcceptCont::~AcceptCont()
> +{
> +}
> +
> +void *
> +AcceptCont::createNetAccept()
> +{
> +    return (NEW(new NetAccept));
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_AcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/I_AcceptCont.h b/iocore/net/I_AcceptCont.h
> new file mode 100644
> index 0000000..2d59e1e
> --- /dev/null
> +++ b/iocore/net/I_AcceptCont.h
> @@ -0,0 +1,48 @@
> +/** @file
> +
> +  AcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#ifndef I_AcceptCont_H_
> +#define I_AcceptCont_H_
> +#include "I_Net.h"
> +#include "I_VConnection.h"
> +
> +class AcceptCont: public Continuation
> +{
> +public:
> +  AcceptCont(ProxyMutex *amutex);
> +  ~AcceptCont();
> +
> +  //
> +  // Virtual function allows creation of an SSLNetAccept
> +  // or NetAccept transparent to NetProcessor.
> +  //
> +  // This function should return a pointer
> +  // of NetAccept or its subclass.
> +  //
> +  virtual void *createNetAccept();
> +
> +private:
> +  virtual int mainEvent(int event, void * netvc) = 0;
> +};
> +
> +#endif /* I_AcceptCont_H_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_Net.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/I_Net.h b/iocore/net/I_Net.h
> index 78f3d62..a98b6a3 100644
> --- a/iocore/net/I_Net.h
> +++ b/iocore/net/I_Net.h
> @@ -91,6 +91,7 @@ extern int net_config_poll_timeout;
> 
> #include "I_NetVConnection.h"
> #include "I_NetProcessor.h"
> +#include "I_AcceptCont.h"
> 
> void ink_net_init(ModuleVersion version);
> #endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/I_NetProcessor.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/I_NetProcessor.h b/iocore/net/I_NetProcessor.h
> index 3deae4c..68ce117 100644
> --- a/iocore/net/I_NetProcessor.h
> +++ b/iocore/net/I_NetProcessor.h
> @@ -100,6 +100,7 @@ public:
>     AcceptOptions() { this->reset(); }
>     /// Reset all values to defaults.
>     self& reset();
> +    bool create_default_NetAccept;
>   };
> 
>   /**
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/iocore/net/Makefile.am b/iocore/net/Makefile.am
> index 4575e9e..efbfa86 100644
> --- a/iocore/net/Makefile.am
> +++ b/iocore/net/Makefile.am
> @@ -52,6 +52,8 @@ libinknet_a_SOURCES = \
>   I_UDPNet.h \
>   I_UDPPacket.h \
>   Inline.cc \
> +  I_AcceptCont.h \
> +  AcceptCont.cc \
>   Net.cc \
>   NetVConnection.cc \
>   P_CompletionUtil.h \
> @@ -61,6 +63,9 @@ libinknet_a_SOURCES = \
>   P_Net.h \
>   P_NetAccept.h \
>   P_NetVConnection.h \
> +  P_ProtocolNetAccept.h \
> +  P_ProtocolAcceptCont.h \
> +  P_Socks.h \
>   P_SSLCertLookup.h \
>   P_SSLConfig.h \
>   P_SSLNetAccept.h \
> @@ -81,6 +86,9 @@ libinknet_a_SOURCES = \
>   P_UnixNetVConnection.h \
>   P_UnixPollDescriptor.h \
>   P_UnixUDPConnection.h \
> +  ProtocolNetAccept.cc \
> +  ProtocolAcceptCont.cc \
> +  Socks.cc \
>   SSLCertLookup.cc \
>   SSLConfig.cc \
>   SSLNetAccept.cc \
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_Net.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_Net.h b/iocore/net/P_Net.h
> index 13ab617..db9acc6 100644
> --- a/iocore/net/P_Net.h
> +++ b/iocore/net/P_Net.h
> @@ -104,6 +104,7 @@ do { \
> #include "P_NetVCTest.h"
> #include "P_LibBulkIO.h"
> 
> +#include "P_ProtocolNetAccept.h"
> #include "P_SSLNetVConnection.h"
> #include "P_SSLNetProcessor.h"
> #include "P_SSLNetAccept.h"
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_NetAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_NetAccept.h b/iocore/net/P_NetAccept.h
> index 3a82e4b..762a601 100644
> --- a/iocore/net/P_NetAccept.h
> +++ b/iocore/net/P_NetAccept.h
> @@ -106,11 +106,13 @@ struct NetAccept:public Continuation
>   virtual UnixNetVConnection *allocateThread(EThread *t);
>   virtual void freeThread(UnixNetVConnection *vc, EThread *t);
>   virtual UnixNetVConnection *allocateGlobal();
> +  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection &con);
>   virtual EventType getEtype();
> 
>   void init_accept_loop(const char *);
>   virtual void init_accept(EThread * t = NULL);
>   virtual void init_accept_per_thread();
> +  virtual NetAccept *clone();
>   // 0 == success
>   int do_listen(bool non_blocking, bool transparent = false);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolAcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_ProtocolAcceptCont.h b/iocore/net/P_ProtocolAcceptCont.h
> new file mode 100644
> index 0000000..7631e5c
> --- /dev/null
> +++ b/iocore/net/P_ProtocolAcceptCont.h
> @@ -0,0 +1,50 @@
> +/** @file
> +
> +  ProtocolAcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#ifndef P_ProtocolAcceptCont_H_
> +#define P_ProtocolAcceptCont_H_
> +
> +#include "I_AcceptCont.h"
> +
> +class ProtocolAcceptCont: public AcceptCont
> +{
> +public:
> +  ProtocolAcceptCont(): AcceptCont(NULL)
> +  {
> +    memset(endpoint, 0, TS_PROTO_MAX * sizeof(AcceptCont *));
> +    SET_HANDLER(&ProtocolAcceptCont::mainEvent);
> +  }
> +  ~ProtocolAcceptCont() {}
> +
> +  void *createNetAccept();
> +  void registerEndpoint(TSProtoType type, Continuation *ep);
> +
> +private:
> +  int mainEvent(int event, void * netvc);
> +  ProtocolAcceptCont(const ProtocolAcceptCont &); // disabled
> +  ProtocolAcceptCont& operator =(const ProtocolAcceptCont&); // disabled
> +
> +  Continuation *endpoint[TS_PROTO_MAX];
> +};
> +
> +#endif /* P_ProtocolAcceptCont_H_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_ProtocolNetAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_ProtocolNetAccept.h b/iocore/net/P_ProtocolNetAccept.h
> new file mode 100644
> index 0000000..ac49571
> --- /dev/null
> +++ b/iocore/net/P_ProtocolNetAccept.h
> @@ -0,0 +1,62 @@
> +/** @file
> +
> +  A brief file description
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +/****************************************************************************
> +
> +  P_ProtocolNetAccept.h
> +
> +
> +   NetAccept is a generalized facility which allows
> +   Connections of different classes to be accepted either
> +   from a blockable thread or by adaptive polling.
> +
> +   It is used by the NetProcessor and the ClusterProcessor
> +   and should be considered PRIVATE to processor implementations.
> +
> +
> +
> + ****************************************************************************/
> +#if !defined (_ProtocolNetAccept_h_)
> +#define _ProtocolNetAccept_h_
> +
> +#include "libts.h"
> +#include "P_NetAccept.h"
> +#include "P_Connection.h"
> +
> +class Event;
> +class UnixNetVConnection;
> +
> +//
> +// NetAccept
> +// Handles accepting connections.
> +//
> +struct ProtocolNetAccept: public NetAccept
> +{
> +  virtual UnixNetVConnection *createSuitableVC(EThread *t, Connection &con);
> +  virtual NetAccept *clone();
> +
> +  ProtocolNetAccept() {}
> +  virtual ~ProtocolNetAccept() {}
> +
> +};
> +#endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNetAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNetAccept.h b/iocore/net/P_SSLNetAccept.h
> index e9a8f7b..6d963a3 100644
> --- a/iocore/net/P_SSLNetAccept.h
> +++ b/iocore/net/P_SSLNetAccept.h
> @@ -57,6 +57,7 @@ struct SSLNetAccept: public NetAccept
>   virtual UnixNetVConnection *allocateGlobal();
>   virtual EventType getEtype();
>   virtual void init_accept_per_thread();
> +  virtual NetAccept *clone();
> 
>   SSLNetAccept()
>     { };
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolAccept.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNextProtocolAccept.h b/iocore/net/P_SSLNextProtocolAccept.h
> index c3ee575..342485a 100644
> --- a/iocore/net/P_SSLNextProtocolAccept.h
> +++ b/iocore/net/P_SSLNextProtocolAccept.h
> @@ -31,7 +31,7 @@
> #include "P_SSLNextProtocolSet.h"
> #include "I_IOBuffer.h"
> 
> -class SSLNextProtocolAccept: public Continuation
> +class SSLNextProtocolAccept: public AcceptCont
> {
> public:
>   SSLNextProtocolAccept(Continuation *);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_SSLNextProtocolSet.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNextProtocolSet.h b/iocore/net/P_SSLNextProtocolSet.h
> index e25f50d..d3c37b6 100644
> --- a/iocore/net/P_SSLNextProtocolSet.h
> +++ b/iocore/net/P_SSLNextProtocolSet.h
> @@ -40,7 +40,7 @@ public:
>   bool advertiseProtocols(const unsigned char ** out, unsigned * len) const;
> 
>   Continuation * findEndpoint(const char *) const;
> -  Continuation * findEndpoint(const unsigned char *, unsigned, TSClientProtoStack *) const;
> +  Continuation * findEndpoint(const unsigned char *, unsigned, TSClientProtoStack *, const char **) const;
> 
>   struct NextProtocolEndpoint
>   {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/P_UnixNetVConnection.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_UnixNetVConnection.h b/iocore/net/P_UnixNetVConnection.h
> index abdf4ba..0e4eae8 100644
> --- a/iocore/net/P_UnixNetVConnection.h
> +++ b/iocore/net/P_UnixNetVConnection.h
> @@ -41,6 +41,13 @@ class UnixNetVConnection;
> class NetHandler;
> struct PollDescriptor;
> 
> +enum SpdyProbeState
> +{
> +  SPDY_PROBE_STATE_NONE,
> +  SPDY_PROBE_STATE_BEGIN,
> +  SPDY_PROBE_STATE_END
> +};
> +
> TS_INLINE void
> NetVCOptions::reset()
> {
> @@ -233,6 +240,8 @@ public:
>   ink_hrtime submit_time;
>   OOB_callback *oob_ptr;
>   bool from_accept_thread;
> +  SpdyProbeState probe_state;
> +  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/f90f3a48/iocore/net/ProtocolAcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/ProtocolAcceptCont.cc b/iocore/net/ProtocolAcceptCont.cc
> new file mode 100644
> index 0000000..2e73b77
> --- /dev/null
> +++ b/iocore/net/ProtocolAcceptCont.cc
> @@ -0,0 +1,73 @@
> +/** @file
> +
> +  ProtocolAcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "P_ProtocolAcceptCont.h"
> +#include "P_SSLNextProtocolAccept.h"
> +#include "P_Net.h"
> +#include "I_Machine.h"
> +#include "Error.h"
> +
> +void *
> +ProtocolAcceptCont::createNetAccept()
> +{
> +  return ((NetAccept *) NEW(new ProtocolNetAccept));
> +}
> +
> +void
> +ProtocolAcceptCont::registerEndpoint(TSProtoType type, Continuation *ep)
> +{
> +  endpoint[type] = ep;
> +}
> +
> +int
> +ProtocolAcceptCont::mainEvent(int event, void *netvc)
> +{
> +  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> +  ink_release_assert((event == NET_EVENT_ACCEPT) ? (netvc!= 0) : (1));
> +
> +  if (event == NET_EVENT_ACCEPT) {
> +    TSProtoType proto_type;
> +    UnixNetVConnection *vc = (UnixNetVConnection *)netvc;
> +
> +    if (vc->proto_stack & (1u << TS_PROTO_TLS)) {
> +      proto_type = TS_PROTO_TLS;
> +    } else if (vc->proto_stack & (1u << TS_PROTO_HTTP)) {
> +      proto_type = TS_PROTO_HTTP;
> +    } else if (vc->proto_stack & (1u << TS_PROTO_SPDY)) {
> +      proto_type = TS_PROTO_SPDY;
> +    } else {
> +      Warning("Invalid protocol stack:%x", vc->proto_stack);
> +      return EVENT_CONT;
> +    }
> +
> +    if (endpoint[proto_type])
> +      endpoint[proto_type]->handleEvent(NET_EVENT_ACCEPT, netvc);
> +    else
> +      Warning("Unregistered protocol type:%x", proto_type);
> +
> +    return EVENT_CONT;
> +  }
> +
> +  MachineFatal("Protocol Accept received fatal error: errno = %d", -((int)(intptr_t)netvc));
> +  return EVENT_CONT;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/ProtocolNetAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/ProtocolNetAccept.cc b/iocore/net/ProtocolNetAccept.cc
> new file mode 100644
> index 0000000..c4257e2
> --- /dev/null
> +++ b/iocore/net/ProtocolNetAccept.cc
> @@ -0,0 +1,68 @@
> +/** @file
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "ink_config.h"
> +#include "P_Net.h"
> +#include "P_ProtocolNetAccept.h"
> +
> +UnixNetVConnection *
> +ProtocolNetAccept::createSuitableVC(EThread *t, Connection &con)
> +{
> +  UnixNetVConnection *vc;
> +
> +  if (etype == SSLNetProcessor::ET_SSL && etype) {
> +    // SSL protocol
> +    if (t)
> +      vc = (UnixNetVConnection *)THREAD_ALLOC(sslNetVCAllocator, t);
> +    else
> +      vc = (UnixNetVConnection *)sslNetVCAllocator.alloc();
> +    vc->proto_stack = (1u << TS_PROTO_TLS);
> +  } else {
> +    if (t)
> +      vc = THREAD_ALLOC(netVCAllocator, t);
> +    else
> +      vc = netVCAllocator.alloc();
> +
> +#if TS_HAS_SPDY
> +    vc->probe_state = SPDY_PROBE_STATE_BEGIN;
> +#else
> +    vc->probe_state = SPDY_PROBE_STATE_NONE;
> +#endif
> +
> +    //
> +    // Protocol stack may be changed by
> +    // following call of SpdyProbe()
> +    //
> +    vc->proto_stack = (1u << TS_PROTO_HTTP);
> +  }
> +
> +  vc->con = con;
> +  return vc;
> +}
> +
> +NetAccept *
> +ProtocolNetAccept::clone()
> +{
> +  NetAccept *na;
> +  na = NEW(new ProtocolNetAccept);
> +  *na = *this;
> +  return na;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetAccept.cc b/iocore/net/SSLNetAccept.cc
> index a321f89..6aa5ed9 100644
> --- a/iocore/net/SSLNetAccept.cc
> +++ b/iocore/net/SSLNetAccept.cc
> @@ -70,10 +70,9 @@ SSLNetAccept::init_accept_per_thread()
>   NetAccept *a = this;
>   n = eventProcessor.n_threads_for_type[SSLNetProcessor::ET_SSL];
>   for (i = 0; i < n; i++) {
> -    if (i < n - 1) {
> -      a = NEW(new SSLNetAccept);
> -      *a = *this;
> -    } else
> +    if (i < n - 1)
> +      a = clone();
> +    else
>       a = this;
>     EThread *t = eventProcessor.eventthread[SSLNetProcessor::ET_SSL][i];
> 
> @@ -84,3 +83,12 @@ SSLNetAccept::init_accept_per_thread()
>     t->schedule_every(a, period, etype);
>   }
> }
> +
> +NetAccept *
> +SSLNetAccept::clone()
> +{
> +  NetAccept *na;
> +  na = NEW(new SSLNetAccept);
> +  *na = *this;
> +  return na;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
> index b55dcf7..3925de9 100644
> --- a/iocore/net/SSLNetVConnection.cc
> +++ b/iocore/net/SSLNetVConnection.cc
> @@ -578,7 +578,8 @@ 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->proto_stack);
> +        this->npnEndpoint = this->npnSet->findEndpoint(proto, len, &this->proto_stack,
> +                                                       &this->selected_next_protocol);
>         this->npnSet = NULL;
> 
>         ink_assert(this->npnEndpoint != NULL);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNextProtocolAccept.cc b/iocore/net/SSLNextProtocolAccept.cc
> index bfc2f0b..746995b 100644
> --- a/iocore/net/SSLNextProtocolAccept.cc
> +++ b/iocore/net/SSLNextProtocolAccept.cc
> @@ -147,7 +147,7 @@ SSLNextProtocolAccept::unregisterEndpoint(
> }
> 
> SSLNextProtocolAccept::SSLNextProtocolAccept(Continuation * ep)
> -    : Continuation(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
> +    : AcceptCont(NULL), buffer(new_empty_MIOBuffer()), endpoint(ep)
> {
>   SET_HANDLER(&SSLNextProtocolAccept::mainEvent);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/SSLNextProtocolSet.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNextProtocolSet.cc b/iocore/net/SSLNextProtocolSet.cc
> index 148a6d1..ed0a5e3 100644
> --- a/iocore/net/SSLNextProtocolSet.cc
> +++ b/iocore/net/SSLNextProtocolSet.cc
> @@ -133,7 +133,8 @@ SSLNextProtocolSet::unregisterEndpoint(const char * proto, Continuation * ep)
> 
> Continuation *
> SSLNextProtocolSet::findEndpoint(const unsigned char * proto, unsigned len,
> -                                 TSClientProtoStack *proto_stack) const
> +                                 TSClientProtoStack *proto_stack,
> +                                 const char **selected_protocol) const
> {
>   for (const NextProtocolEndpoint * ep = this->endpoints.head;
>         ep != NULL; ep = this->endpoints.next(ep)) {
> @@ -141,6 +142,8 @@ SSLNextProtocolSet::findEndpoint(const unsigned char * proto, unsigned len,
>     if (sz == len && memcmp(ep->protocol, proto, len) == 0) {
>       if (proto_stack)
>         *proto_stack = ep->proto_stack;
> +      if (selected_protocol)
> +        *selected_protocol = ep->protocol;
>       return ep->endpoint;
>     }
>   }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetAccept.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/UnixNetAccept.cc b/iocore/net/UnixNetAccept.cc
> index c038a2e..af4d8f9 100644
> --- a/iocore/net/UnixNetAccept.cc
> +++ b/iocore/net/UnixNetAccept.cc
> @@ -223,10 +223,9 @@ NetAccept::init_accept_per_thread()
>   NetAccept *a;
>   n = eventProcessor.n_threads_for_type[ET_NET];
>   for (i = 0; i < n; i++) {
> -    if (i < n - 1) {
> -      a = NEW(new NetAccept);
> -      *a = *this;
> -    } else
> +    if (i < n - 1)
> +      a = clone();
> +    else
>       a = this;
>     EThread *t = eventProcessor.eventthread[ET_NET][i];
>     PollDescriptor *pd = get_PollDescriptor(t);
> @@ -237,6 +236,14 @@ NetAccept::init_accept_per_thread()
>   }
> }
> 
> +NetAccept *
> +NetAccept::clone()
> +{
> +  NetAccept *na;
> +  na = NEW(new NetAccept);
> +  *na = *this;
> +  return na;
> +}
> 
> int
> NetAccept::do_listen(bool non_blocking, bool transparent)
> @@ -264,6 +271,20 @@ NetAccept::do_listen(bool non_blocking, bool transparent)
>   return res;
> }
> 
> +UnixNetVConnection *
> +NetAccept::createSuitableVC(EThread *t, Connection &con)
> +{
> +  UnixNetVConnection *vc;
> +
> +  if (t)
> +    vc = allocateThread(t);
> +  else
> +    vc = allocateGlobal();
> +
> +  vc->con = con;
> +
> +  return vc;
> +}
> 
> int
> NetAccept::do_blocking_accept(EThread * t)
> @@ -271,18 +292,11 @@ NetAccept::do_blocking_accept(EThread * t)
>   int res = 0;
>   int loop = accept_till_done;
>   UnixNetVConnection *vc = NULL;
> +  Connection con;
> 
>   //do-while for accepting all the connections
>   //added by YTS Team, yamsat
>   do {
> -    vc = (UnixNetVConnection *)alloc_cache;
> -    if (likely(!vc)) {
> -      //vc = allocateThread(t);
> -      vc = allocateGlobal(); // Bypass proxy / thread allocator
> -      vc->from_accept_thread = true;
> -      vc->id = net_next_connection_number();
> -      alloc_cache = vc;
> -    }
>     ink_hrtime now = ink_get_hrtime();
> 
>     // Throttle accepts
> @@ -297,7 +311,7 @@ NetAccept::do_blocking_accept(EThread * t)
>       now = ink_get_hrtime();
>     }
> 
> -    if ((res = server.accept(&vc->con)) < 0) {
> +    if ((res = server.accept(&con)) < 0) {
>     Lerror:
>       int seriousness = accept_error_seriousness(res);
>       if (seriousness >= 0) {   // not so bad
> @@ -314,16 +328,23 @@ NetAccept::do_blocking_accept(EThread * t)
>       }
>       return -1;
>     }
> -    check_emergency_throttle(vc->con);
> +
> +    // Use 'NULL' to Bypass thread allocator
> +    vc = createSuitableVC(NULL, con);
> +    if (!vc)
> +      return -1;
> +    vc->from_accept_thread = true;
> +    vc->id = net_next_connection_number();
>     alloc_cache = NULL;
> 
> +    check_emergency_throttle(con);
> +
>     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
>     vc->submit_time = now;
>     ats_ip_copy(&vc->server_addr, &vc->con.addr);
>     vc->set_is_transparent(server.f_inbound_transparent);
>     vc->mutex = new_ProxyMutex();
>     vc->action_ = *action_;
> -    vc->proto_stack = (1u << TS_PROTO_HTTP);
>     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) & UnixNetVConnection::acceptEvent);
>     //eventProcessor.schedule_imm(vc, getEtype());
>     eventProcessor.schedule_imm_signal(vc, getEtype());
> @@ -384,6 +405,7 @@ NetAccept::acceptFastEvent(int event, void *ep)
>   (void) event;
>   (void) e;
>   int bufsz, res;
> +  Connection con;
> 
>   PollDescriptor *pd = get_PollDescriptor(e->ethread);
>   UnixNetVConnection *vc = NULL;
> @@ -394,10 +416,10 @@ NetAccept::acceptFastEvent(int event, void *ep)
>       ifd = -1;
>       return EVENT_CONT;
>     }
> -    vc = allocateThread(e->ethread);
> 
> -    socklen_t sz = sizeof(vc->con.addr);
> -    int fd = socketManager.accept(server.fd, &vc->con.addr.sa, &sz);
> +    socklen_t sz = sizeof(con.addr);
> +    int fd = socketManager.accept(server.fd, &con.addr.sa, &sz);
> +    con.fd = fd;
> 
>     if (likely(fd >= 0)) {
>       Debug("iocore_net", "accepted a new socket: %d", fd);
> @@ -443,6 +465,11 @@ NetAccept::acceptFastEvent(int event, void *ep)
>       do {
>         res = safe_nonblocking(fd);
>       } while (res < 0 && (errno == EAGAIN || errno == EINTR));
> +
> +      vc = createSuitableVC(e->ethread, con);
> +      if (!vc)
> +        goto Ldone;
> +
>     } else {
>       res = fd;
>     }
> @@ -453,20 +480,15 @@ NetAccept::acceptFastEvent(int event, void *ep)
>           || res == -EPIPE
> #endif
>         ) {
> -        ink_assert(vc->con.fd == NO_FD);
> -        ink_assert(!vc->link.next && !vc->link.prev);
> -        freeThread(vc, e->ethread);
>         goto Ldone;
>       } else if (accept_error_seriousness(res) >= 0) {
>         check_transient_accept_error(res);
> -        freeThread(vc, e->ethread);
>         goto Ldone;
>       }
>       if (!action_->cancelled)
>         action_->continuation->handleEvent(EVENT_ERROR, (void *)(intptr_t)res);
>       goto Lerror;
>     }
> -    vc->con.fd = fd;
> 
>     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
>     vc->id = net_next_connection_number();
> @@ -478,7 +500,6 @@ NetAccept::acceptFastEvent(int event, void *ep)
>     vc->thread = e->ethread;
> 
>     vc->nh = get_NetHandler(e->ethread);
> -    vc->proto_stack = (1u << TS_PROTO_HTTP);
> 
>     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler) & UnixNetVConnection::mainEvent);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetProcessor.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/UnixNetProcessor.cc b/iocore/net/UnixNetProcessor.cc
> index eaae439..ce59bbe 100644
> --- a/iocore/net/UnixNetProcessor.cc
> +++ b/iocore/net/UnixNetProcessor.cc
> @@ -46,6 +46,7 @@ NetProcessor::AcceptOptions::reset()
>   packet_mark = 0;
>   packet_tos = 0;
>   f_inbound_transparent = false;
> +  create_default_NetAccept = true;
>   return *this;
> }
> 
> @@ -84,16 +85,22 @@ NetProcessor::main_accept(Continuation *cont, SOCKET fd, AcceptOptions const& op
> Action *
> UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions const& opt)
> {
> -  EventType et = opt.etype; // setEtype requires non-const ref.
> -  NetAccept *na = createNetAccept();
> +  EventType upgraded_etype = opt.etype; // setEtype requires non-const ref.
> +  AcceptCont *acceptCont = static_cast<AcceptCont *>(cont);
>   EThread *thread = this_ethread();
>   ProxyMutex *mutex = thread->mutex;
>   int accept_threads = opt.accept_threads; // might be changed.
>   IpEndpoint accept_ip; // local binding address.
>   char thr_name[MAX_THREAD_NAME_LENGTH];
> 
> +  NetAccept *na;
> +  if (opt.create_default_NetAccept)
> +    na = createNetAccept();
> +  else
> +    na = (NetAccept *)acceptCont->createNetAccept();
> +
>   // Potentially upgrade to SSL.
> -  upgradeEtype(et);
> +  upgradeEtype(upgraded_etype);
> 
>   // Fill in accept thread from configuration if necessary.
>   if (opt.accept_threads < 0) {
> @@ -137,7 +144,7 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
>   na->sockopt_flags = opt.sockopt_flags;
>   na->packet_mark = opt.packet_mark;
>   na->packet_tos = opt.packet_tos;
> -  na->etype = opt.etype;
> +  na->etype = upgraded_etype;
>   na->backdoor = opt.backdoor;
>   if (na->callback_on_open)
>     na->mutex = cont->mutex;
> @@ -147,7 +154,10 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
>         NetAccept *a;
> 
>         for (int i=1; i < accept_threads; ++i) {
> -          a = createNetAccept();
> +          if (opt.create_default_NetAccept)
> +            a = createNetAccept();
> +          else
> +            a = (NetAccept *)acceptCont->createNetAccept();
>           *a = *na;
>           snprintf(thr_name, MAX_THREAD_NAME_LENGTH, "[ACCEPT %d:%d]", i-1, ats_ip_port_host_order(&accept_ip));
>           a->init_accept_loop(thr_name);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/iocore/net/UnixNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
> index bf1dc0a..8a919c5 100644
> --- a/iocore/net/UnixNetVConnection.cc
> +++ b/iocore/net/UnixNetVConnection.cc
> @@ -38,6 +38,84 @@ typedef struct iovec IOVec;
> #define NET_MAX_IOV UIO_MAXIOV
> #endif
> 
> +struct SpdyProbeCont:public Continuation
> +{
> +  MIOBuffer buf;
> +  unsigned char data;
> +  SpdyProbeCont(): data(0)
> +  {
> +    SET_HANDLER(&SpdyProbeCont::mainEvent);
> +  }
> +
> +  int mainEvent(int event, void *e);
> +};
> +
> +static ClassAllocator<SpdyProbeCont> spdyProberContAllocator("spdyProberContAllocator");
> +
> +SpdyProbeCont *
> +new_SpdyProbeCont(UnixNetVConnection *vc)
> +{
> +  SpdyProbeCont *c = spdyProberContAllocator.alloc();
> +  c->buf.clear();
> +  c->buf.set(&c->data, sizeof c->data);
> +  c->buf._writer->fill(-(sizeof c->data));
> +  c->mutex = vc->mutex;
> +  return c;
> +}
> +void
> +free_SpdyProbeCont(SpdyProbeCont *c)
> +{
> +  c->mutex.clear();
> +  c->buf.clear();
> +  spdyProberContAllocator.free(c);
> +}
> +
> +inline int
> +SpdyProbeCont::mainEvent(int event, void *e) {
> +  UnixNetVConnection *vc = (UnixNetVConnection *) ((VIO *) e)->vc_server;
> +  vc->probe_state = SPDY_PROBE_STATE_END;
> +
> +  switch (event) {
> +  case VC_EVENT_EOS:
> +  case VC_EVENT_ERROR:
> +  case VC_EVENT_INACTIVITY_TIMEOUT:
> +  case VC_EVENT_ACTIVE_TIMEOUT:
> +    vc->do_io_close();
> +    free_SpdyProbeCont(this);
> +    return EVENT_DONE;
> +  case VC_EVENT_READ_COMPLETE:
> +    if ((data & 0x80) != 0) {
> +      //
> +      // SPDY Request
> +      //
> +      free_SpdyProbeCont(this);
> +      vc->proto_stack = (1u << TS_PROTO_SPDY);
> +      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
> +      return EVENT_DONE;
> +    } else {
> +      //
> +      // HTTP Request
> +      //
> +      free_SpdyProbeCont(this);
> +      vc->action_.continuation->handleEvent(NET_EVENT_ACCEPT, vc);
> +      return EVENT_DONE;
> +    }
> +  default:
> +    ink_release_assert(!"unexpected event");
> +  }
> +  return EVENT_CONT;
> +}
> +
> +int SpdyProbeStart(UnixNetVConnection *vc)
> +{
> +  SpdyProbeCont *spdyProbeCont= new_SpdyProbeCont(vc);
> +  //
> +  // TODO: make it configurable
> +  //
> +  vc->set_inactivity_timeout(HRTIME_SECONDS(30));
> +  vc->do_io_read(spdyProbeCont, 1, &spdyProbeCont->buf);
> +  return EVENT_CONT;
> +}
> // Global
> ClassAllocator<UnixNetVConnection> netVCAllocator("netVCAllocator");
> 
> @@ -259,8 +337,12 @@ read_from_net(NetHandler *nh, UnixNetVConnection *vc, EThread *thread)
>         }
>         b = b->next;
>       }
> +      ink_assert(vc->probe_state != SPDY_PROBE_STATE_BEGIN || niov == 1);
>       if (niov == 1) {
> -        r = socketManager.read(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len);
> +        if (vc->probe_state == SPDY_PROBE_STATE_BEGIN) {
> +          r = recv(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len, MSG_PEEK);
> +        } else
> +          r = socketManager.read(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len);
>       } else {
>         r = socketManager.readv(vc->con.fd, &tiovec[0], niov);
>       }
> @@ -801,7 +883,8 @@ UnixNetVConnection::UnixNetVConnection()
> #endif
>     active_timeout(NULL), nh(NULL),
>     id(0), flags(0), recursion(0), submit_time(0), oob_ptr(0),
> -    from_accept_thread(false)
> +    from_accept_thread(false), probe_state(SPDY_PROBE_STATE_NONE),
> +    selected_next_protocol(NULL)
> {
>   memset(&local_addr, 0, sizeof local_addr);
>   memset(&server_addr, 0, sizeof server_addr);
> @@ -988,7 +1071,13 @@ UnixNetVConnection::acceptEvent(int event, Event *e)
>     UnixNetVConnection::set_inactivity_timeout(inactivity_timeout_in);
>   if (active_timeout_in)
>     UnixNetVConnection::set_active_timeout(active_timeout_in);
> -  action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
> +  if (probe_state == SPDY_PROBE_STATE_NONE)
> +    action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
> +  else {
> +    ink_assert(probe_state == SPDY_PROBE_STATE_BEGIN);
> +    SpdyProbeStart(this);
> +  }
> +
>   return EVENT_DONE;
> }
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/lib/ts/apidefs.h.in
> ----------------------------------------------------------------------
> diff --git a/lib/ts/apidefs.h.in b/lib/ts/apidefs.h.in
> index 86a650f..0356921 100644
> --- a/lib/ts/apidefs.h.in
> +++ b/lib/ts/apidefs.h.in
> @@ -117,6 +117,9 @@ extern "C"
>     TS_PROTO_SPDY = 13,
>     TS_PROTO_RTMP = 14,
>     TS_PROTO_WBSK = 15, /* WebSocket */
> +
> +    /* MAX value of TSProtoType  */
> +    TS_PROTO_MAX
>   } TSProtoType;
> 
>   typedef uint32_t TSClientProtoStack;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/InkAPI.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
> index 00a78e1..682f889 100644
> --- a/proxy/InkAPI.cc
> +++ b/proxy/InkAPI.cc
> @@ -51,7 +51,7 @@
> #include "PluginVC.h"
> #include "api/ts/experimental.h"
> #include "ICP.h"
> -#include "HttpAccept.h"
> +#include "HttpAcceptCont.h"
> #include "PluginVC.h"
> #include "FetchSM.h"
> #include "HttpDebugNames.h"
> @@ -6082,8 +6082,8 @@ TSHttpAltInfoQualitySet(TSHttpAltInfo infop, float quality)
>   info->m_qvalue = quality;
> }
> 
> -extern HttpAccept *plugin_http_accept;
> -extern HttpAccept *plugin_http_transparent_accept;
> +extern HttpAcceptCont *plugin_http_accept;
> +extern HttpAcceptCont *plugin_http_transparent_accept;
> 
> TSVConn
> TSHttpConnect(sockaddr const* addr)
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/proxy/Makefile.am b/proxy/Makefile.am
> index 8ef6104..6dfc816 100644
> --- a/proxy/Makefile.am
> +++ b/proxy/Makefile.am
> @@ -17,7 +17,7 @@
> #  limitations under the License.
> 
> # Note that hdrs is targeted from ../Makefile.am
> -SUBDIRS = congest http logging config
> +SUBDIRS = congest http spdy logging config
> noinst_LIBRARIES =
> bin_PROGRAMS = \
>   traffic_server \
> @@ -52,11 +52,10 @@ noinst_HEADERS = \
>   ConfigParse.h \
>   Show.h
> 
> -
> if STATIC_LIBTS
> -   which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
> +    which_libts = $(top_builddir)/lib/ts/.libs/libtsutil.a
> else
> -   which_libts = $(top_builddir)/lib/ts/libtsutil.la
> +    which_libts = $(top_builddir)/lib/ts/libtsutil.la
> endif
> 
> EXTRA_DIST = InkAPITestTool.cc example_alarm_bin.sh example_prep.sh
> @@ -121,6 +120,7 @@ endif
> traffic_server_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
> traffic_server_LDADD = \
>   http/libhttp.a \
> +  spdy/libspdy.a \
>   http/remap/libhttp_remap.a \
>   congest/libCongestionControl.a \
>   logging/liblogging.a \
> @@ -157,6 +157,7 @@ traffic_server_LDADD = \
>   @LIBZ@ \
>   @LIBLZMA@ \
>   @LIBPROFILER@ \
> +  @SPDYLAY_LIBS@ \
>   -lm
> 
> if BUILD_LUA_SUPPORT
> @@ -223,6 +224,7 @@ traffic_sac_SOURCES = \
> traffic_sac_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
> traffic_sac_LDADD = \
>   http/libhttp.a \
> +  spdy/libspdy.a \
>   shared/libdiagsconfig.a \
>   http/remap/libhttp_remap.a \
>   congest/libCongestionControl.a \
> @@ -247,7 +249,7 @@ traffic_sac_LDADD = \
>   $(top_builddir)/lib/records/librecprocess.a \
>   $(top_builddir)/lib/ts/libtsutil.la \
>   @LIBRESOLV@ @LIBPCRE@ @OPENSSL_LIBS@ @LIBTCL@ @hwloc_LIBS@ \
> -  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@ -lm
> +  @LIBEXPAT@ @LIBDEMANGLE@ @LIBZ@ @LIBLZMA@ @LIBPROFILER@ @SPDYLAY_LIBS@ -lm
> 
> if BUILD_TESTS
>   traffic_sac_SOURCES += RegressionSM.cc
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/SocksProxy.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/SocksProxy.cc b/proxy/SocksProxy.cc
> index e447f12..d855689 100644
> --- a/proxy/SocksProxy.cc
> +++ b/proxy/SocksProxy.cc
> @@ -31,7 +31,7 @@
> #include "StatSystem.h"
> #include "P_Net.h"
> #include "I_OneWayTunnel.h"
> -#include "HttpAccept.h"
> +#include "HttpAcceptCont.h"
> 
> enum
> {
> @@ -145,14 +145,14 @@ SocksProxy::mainEvent(int event, void *data)
> 
>     switch (state) {
>     case HTTP_REQ:{
> -      HttpAccept::Options ha_opt;
> +      HttpAcceptCont::Options ha_opt;
>       //This is a WRITE_COMPLETE. vio->nbytes == vio->ndone is true
> 
>       SOCKSPROXY_INC_STAT(socksproxy_http_connections_stat);
>       Debug("SocksProxy", "Handing over the HTTP request\n");
> 
>       ha_opt.transport_type = clientVC->attributes;
> -      HttpAccept http_accept(ha_opt);
> +      HttpAcceptCont http_accept(ha_opt);
>       http_accept.mainEvent(NET_EVENT_ACCEPT, clientVC);
>       state = ALL_DONE;
>       break;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/api/ts/InkAPIPrivateIOCore.h
> ----------------------------------------------------------------------
> diff --git a/proxy/api/ts/InkAPIPrivateIOCore.h b/proxy/api/ts/InkAPIPrivateIOCore.h
> index 26e3fac..97a9bcd 100644
> --- a/proxy/api/ts/InkAPIPrivateIOCore.h
> +++ b/proxy/api/ts/InkAPIPrivateIOCore.h
> @@ -51,6 +51,10 @@ public:
> 
>   void handle_event_count(int event);
>   int handle_event(int event, void *edata);
> +  virtual void *createNetAccept()
> +  {
> +    return (NEW(new NetAccept));
> +  }
> 
> public:
>   void *mdata;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAccept.cc b/proxy/http/HttpAccept.cc
> deleted file mode 100644
> index 9fb0066..0000000
> --- a/proxy/http/HttpAccept.cc
> +++ /dev/null
> @@ -1,98 +0,0 @@
> -/** @file
> -
> -  A brief file description
> -
> -  @section license License
> -
> -  Licensed to the Apache Software Foundation (ASF) under one
> -  or more contributor license agreements.  See the NOTICE file
> -  distributed with this work for additional information
> -  regarding copyright ownership.  The ASF licenses this file
> -  to you under the Apache License, Version 2.0 (the
> -  "License"); you may not use this file except in compliance
> -  with the License.  You may obtain a copy of the License at
> -
> -      http://www.apache.org/licenses/LICENSE-2.0
> -
> -  Unless required by applicable law or agreed to in writing, software
> -  distributed under the License is distributed on an "AS IS" BASIS,
> -  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> -  See the License for the specific language governing permissions and
> -  limitations under the License.
> - */
> -
> -#include "HttpAccept.h"
> -#include "IPAllow.h"
> -#include "HttpClientSession.h"
> -#include "I_Machine.h"
> -#include "Error.h"
> -
> -int
> -HttpAccept::mainEvent(int event, void *data)
> -{
> -  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> -  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
> -
> -  if (event == NET_EVENT_ACCEPT) {
> -    ////////////////////////////////////////////////////
> -    // if client address forbidden, close immediately //
> -    ////////////////////////////////////////////////////
> -    NetVConnection *netvc = static_cast<NetVConnection *>(data);
> -    sockaddr const* client_ip = netvc->get_remote_addr();
> -    uint32_t acl_method_mask = 0;
> -    ip_port_text_buffer ipb;
> -    IpAllow::scoped_config ipallow;
> -
> -    // The backdoor port is now only bound to "localhost", so no
> -    // reason to check for if it's incoming from "localhost" or not.
> -    if (backdoor) {
> -      acl_method_mask = IpAllow::AllMethodMask();
> -    } else if (ipallow && ((acl_method_mask = ipallow->match(client_ip)) == 0)) {
> -      Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
> -      netvc->do_io_close();
> -
> -      return VC_EVENT_CONT;
> -    }
> -
> -    netvc->attributes = transport_type;
> -
> -    if (is_debug_tag_set("http_seq"))
> -      Debug("http_seq", "[HttpAccept:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
> -
> -    HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
> -
> -   // copy over session related data.
> -    new_session->f_outbound_transparent = f_outbound_transparent;
> -    new_session->f_transparent_passthrough = f_transparent_passthrough;
> -    new_session->outbound_ip4 = outbound_ip4;
> -    new_session->outbound_ip6 = outbound_ip6;
> -    new_session->outbound_port = outbound_port;
> -    new_session->host_res_style = ats_host_res_from(client_ip->sa_family, host_res_preference);
> -    new_session->acl_method_mask = acl_method_mask;
> -
> -    new_session->new_connection(netvc, backdoor);
> -
> -    return EVENT_CONT;
> -  }
> -
> -  /////////////////
> -  // EVENT_ERROR //
> -  /////////////////
> -  if (((long) data) == -ECONNABORTED) {
> -    /////////////////////////////////////////////////
> -    // Under Solaris, when accept() fails and sets //
> -    // errno to EPROTO, it means the client has    //
> -    // sent a TCP reset before the connection has  //
> -    // been accepted by the server...  Note that   //
> -    // in 2.5.1 with the Internet Server Supplement//
> -    // and also in 2.6 the errno for this case has //
> -    // changed from EPROTO to ECONNABORTED.        //
> -    /////////////////////////////////////////////////
> -
> -    // FIX: add time to user_agent_hangup
> -    HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
> -  }
> -
> -  MachineFatal("HTTP accept received fatal error: errno = %d", -((int)(intptr_t)data));
> -  return EVENT_CONT;
> -}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAccept.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAccept.h b/proxy/http/HttpAccept.h
> deleted file mode 100644
> index 30d68e6..0000000
> --- a/proxy/http/HttpAccept.h
> +++ /dev/null
> @@ -1,198 +0,0 @@
> -/** @file
> -
> -  A brief file description
> -
> -  @section license License
> -
> -  Licensed to the Apache Software Foundation (ASF) under one
> -  or more contributor license agreements.  See the NOTICE file
> -  distributed with this work for additional information
> -  regarding copyright ownership.  The ASF licenses this file
> -  to you under the Apache License, Version 2.0 (the
> -  "License"); you may not use this file except in compliance
> -  with the License.  You may obtain a copy of the License at
> -
> -      http://www.apache.org/licenses/LICENSE-2.0
> -
> -  Unless required by applicable law or agreed to in writing, software
> -  distributed under the License is distributed on an "AS IS" BASIS,
> -  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> -  See the License for the specific language governing permissions and
> -  limitations under the License.
> - */
> -
> -#if !defined (_HttpAccept_h_)
> -#define _HttpAccept_h_
> -
> -#include "libts.h"
> -#include "P_EventSystem.h"
> -#include "HttpConfig.h"
> -#include "HTTP.h"
> -
> -namespace detail {
> -  /** Options for @c HttpAccept.
> -
> -      @internal This is done as a separate class for two reasons.
> -
> -      The first is that in current usage many instances are created
> -      with the same options so (for the client) this is easier and
> -      more efficient than passing options directly to the @c
> -      HttpAccept or calling setters.
> -
> -      The second is that @c HttpAccept is not provided with any thread
> -      safety because it is intended as an immutable object. Putting
> -      the setters here and not there makes that clearer.
> -
> -      We don't do this directly as nested class because we want to
> -      inherit the data members rather than duplicate the declarations
> -      and initializations.
> -   */
> -  class HttpAcceptOptions {
> -  private:
> -    typedef HttpAcceptOptions self; ///< Self reference type.
> -  public:
> -    HttpAcceptOptions();
> -
> -    // Connection type (HttpProxyPort::TransportType)
> -    int transport_type;
> -    /// Set the transport type.
> -    self& setTransportType(int);
> -    /// Local address to bind for outbound connections.
> -    IpAddr outbound_ip4;
> -    /// Local address to bind for outbound connections.
> -    IpAddr outbound_ip6;
> -    /// Set the outbound IP address to @a ip.
> -    self& setOutboundIp(IpAddr& ip);
> -    /// Set the outbound IP address to @a ip.
> -    self& setOutboundIp(IpEndpoint* ip);
> -    /// Local port for outbound connection.
> -    uint16_t outbound_port;
> -    /// Set outbound port.
> -    self& setOutboundPort(uint16_t);
> -    /// Outbound transparent.
> -    bool f_outbound_transparent;
> -    /// Set outbound transparency.
> -    self& setOutboundTransparent(bool);
> -    /// Transparent pass-through.
> -    bool f_transparent_passthrough;
> -    /// Set transparent passthrough.
> -    self& setTransparentPassthrough(bool);
> -    /// Accepting backdoor connections.
> -    bool backdoor;
> -    /// Set backdoor accept.
> -    self& setBackdoor(bool);
> -    /// Host address resolution preference order.
> -    HostResPreferenceOrder host_res_preference;
> -    /// Set the host query preference.
> -    self& setHostResPreference(HostResPreferenceOrder const);
> -  };
> -
> -  inline HttpAcceptOptions::HttpAcceptOptions()
> -    : transport_type(0)
> -    , outbound_port(0)
> -    , f_outbound_transparent(false)
> -    , f_transparent_passthrough(false)
> -    , backdoor(false)
> -  {
> -    memcpy(host_res_preference, host_res_default_preference_order, sizeof(host_res_preference));
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setTransportType(int type) {
> -    transport_type =  type;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundIp(IpAddr& ip) {
> -    if (ip.isIp4()) outbound_ip4 = ip;
> -    else if (ip.isIp6()) outbound_ip6 = ip;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundIp(IpEndpoint* ip) {
> -    if (ip->isIp4()) outbound_ip4 = *ip;
> -    else if (ip->isIp6()) outbound_ip6 = *ip;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundPort(uint16_t port) {
> -    outbound_port = port;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setOutboundTransparent(bool flag) {
> -    f_outbound_transparent = flag;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setTransparentPassthrough(bool flag) {
> -    f_transparent_passthrough = flag;
> -    return *this;
> -  }
> -
> - inline HttpAcceptOptions&
> -  HttpAcceptOptions::setBackdoor(bool flag) {
> -    backdoor = flag;
> -    return *this;
> -  }
> -
> -  inline HttpAcceptOptions&
> -  HttpAcceptOptions::setHostResPreference(HostResPreferenceOrder const order) {
> -    memcpy(host_res_preference, order, sizeof(host_res_preference));
> -    return *this;
> -  }
> -}
> -
> -/**
> -   The continuation mutex is NULL to allow parellel accepts in NT. No
> -   state is recorded by the handler and values are required to be set
> -   during construction via the @c Options struct and never changed. So
> -   a NULL mutex is safe.
> -
> -   Most of the state is simply passed on to the @c ClientSession after
> -   an accept. It is done here because this is the least bad pathway
> -   from the top level configuration to the HTTP session.
> -*/
> -
> -class HttpAccept: public Continuation, private detail::HttpAcceptOptions
> -{
> -private:
> -  typedef HttpAccept self; ///< Self reference type.
> -public:
> -  /** Construction options.
> -      Provide an easier to remember typedef for clients.
> -  */
> -  typedef detail::HttpAcceptOptions Options;
> -
> -  /** Default constructor.
> -    
> -      @internal We don't use a static default options object because of
> -      initialization order issues. It is important to pick up data that is read
> -      from the config file and a static is initialized long before that point.
> -  */
> -  HttpAccept(Options const& opt = Options())
> -    : Continuation(NULL)
> -    , detail::HttpAcceptOptions(opt) // copy these.
> -  {
> -    SET_HANDLER(&HttpAccept::mainEvent);
> -    return;
> -  }
> -
> -  ~HttpAccept()
> -  {
> -    return;
> -  }
> -
> -  int mainEvent(int event, void *netvc);
> -
> -private:
> -    HttpAccept(const HttpAccept &);
> -    HttpAccept & operator =(const HttpAccept &);
> -};
> -
> -#endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAcceptCont.cc b/proxy/http/HttpAcceptCont.cc
> new file mode 100644
> index 0000000..d3744e7
> --- /dev/null
> +++ b/proxy/http/HttpAcceptCont.cc
> @@ -0,0 +1,98 @@
> +/** @file
> +
> +  A brief file description
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "HttpAcceptCont.h"
> +#include "IPAllow.h"
> +#include "HttpClientSession.h"
> +#include "I_Machine.h"
> +#include "Error.h"
> +
> +int
> +HttpAcceptCont::mainEvent(int event, void *data)
> +{
> +  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
> +  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != 0) : (1));
> +
> +  if (event == NET_EVENT_ACCEPT) {
> +    ////////////////////////////////////////////////////
> +    // if client address forbidden, close immediately //
> +    ////////////////////////////////////////////////////
> +    NetVConnection *netvc = static_cast<NetVConnection *>(data);
> +    sockaddr const* client_ip = netvc->get_remote_addr();
> +    uint32_t acl_method_mask = 0;
> +    ip_port_text_buffer ipb;
> +    IpAllow::scoped_config ipallow;
> +
> +    // The backdoor port is now only bound to "localhost", so no
> +    // reason to check for if it's incoming from "localhost" or not.
> +    if (backdoor) {
> +      acl_method_mask = IpAllow::AllMethodMask();
> +    } else if (ipallow && ((acl_method_mask = ipallow->match(client_ip)) == 0)) {
> +      Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
> +      netvc->do_io_close();
> +
> +      return VC_EVENT_CONT;
> +    }
> +
> +    netvc->attributes = transport_type;
> +
> +    if (is_debug_tag_set("http_seq"))
> +      Debug("http_seq", "[HttpAcceptCont:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
> +
> +    HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, netvc->thread);
> +
> +   // copy over session related data.
> +    new_session->f_outbound_transparent = f_outbound_transparent;
> +    new_session->f_transparent_passthrough = f_transparent_passthrough;
> +    new_session->outbound_ip4 = outbound_ip4;
> +    new_session->outbound_ip6 = outbound_ip6;
> +    new_session->outbound_port = outbound_port;
> +    new_session->host_res_style = ats_host_res_from(client_ip->sa_family, host_res_preference);
> +    new_session->acl_method_mask = acl_method_mask;
> +
> +    new_session->new_connection(netvc, backdoor);
> +
> +    return EVENT_CONT;
> +  }
> +
> +  /////////////////
> +  // EVENT_ERROR //
> +  /////////////////
> +  if (((long) data) == -ECONNABORTED) {
> +    /////////////////////////////////////////////////
> +    // Under Solaris, when accept() fails and sets //
> +    // errno to EPROTO, it means the client has    //
> +    // sent a TCP reset before the connection has  //
> +    // been accepted by the server...  Note that   //
> +    // in 2.5.1 with the Internet Server Supplement//
> +    // and also in 2.6 the errno for this case has //
> +    // changed from EPROTO to ECONNABORTED.        //
> +    /////////////////////////////////////////////////
> +
> +    // FIX: add time to user_agent_hangup
> +    HTTP_SUM_DYN_STAT(http_ua_msecs_counts_errors_pre_accept_hangups_stat, 0);
> +  }
> +
> +  MachineFatal("HTTP accept received fatal error: errno = %d", -((int)(intptr_t)data));
> +  return EVENT_CONT;
> +}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpAcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpAcceptCont.h b/proxy/http/HttpAcceptCont.h
> new file mode 100644
> index 0000000..5b8c437
> --- /dev/null
> +++ b/proxy/http/HttpAcceptCont.h
> @@ -0,0 +1,198 @@
> +/** @file
> +
> +  A brief file description
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#if !defined (_HttpAcceptCont_h_)
> +#define _HttpAcceptCont_h_
> +
> +#include "libts.h"
> +#include "P_EventSystem.h"
> +#include "HttpConfig.h"
> +#include "HTTP.h"
> +#include "I_Net.h"
> +
> +namespace detail {
> +  /** Options for @c HttpAcceptCont.
> +
> +      @internal This is done as a separate class for two reasons.
> +
> +      The first is that in current usage many instances are created
> +      with the same options so (for the client) this is easier and
> +      more efficient than passing options directly to the @c
> +      HttpAcceptCont or calling setters.
> +
> +      The second is that @c HttpAcceptCont is not provided with any thread
> +      safety because it is intended as an immutable object. Putting
> +      the setters here and not there makes that clearer.
> +
> +      We don't do this directly as nested class because we want to
> +      inherit the data members rather than duplicate the declarations
> +      and initializations.
> +   */
> +  class HttpAcceptContOptions {
> +  private:
> +    typedef HttpAcceptContOptions self; ///< Self reference type.
> +  public:
> +    HttpAcceptContOptions();
> +
> +    // Connection type (HttpProxyPort::TransportType)
> +    int transport_type;
> +    /// Set the transport type.
> +    self& setTransportType(int);
> +    /// Local address to bind for outbound connections.
> +    IpAddr outbound_ip4;
> +    /// Local address to bind for outbound connections.
> +    IpAddr outbound_ip6;
> +    /// Set the outbound IP address to @a ip.
> +    self& setOutboundIp(IpAddr& ip);
> +    /// Set the outbound IP address to @a ip.
> +    self& setOutboundIp(IpEndpoint* ip);
> +    /// Local port for outbound connection.
> +    uint16_t outbound_port;
> +    /// Set outbound port.
> +    self& setOutboundPort(uint16_t);
> +    /// Outbound transparent.
> +    bool f_outbound_transparent;
> +    /// Set outbound transparency.
> +    self& setOutboundTransparent(bool);
> +    /// Transparent pass-through.
> +    bool f_transparent_passthrough;
> +    /// Set transparent passthrough.
> +    self& setTransparentPassthrough(bool);
> +    /// Accepting backdoor connections.
> +    bool backdoor;
> +    /// Set backdoor accept.
> +    self& setBackdoor(bool);
> +    /// Host address resolution preference order.
> +    HostResPreferenceOrder host_res_preference;
> +    /// Set the host query preference.
> +    self& setHostResPreference(HostResPreferenceOrder const);
> +  };
> +
> +  inline HttpAcceptContOptions::HttpAcceptContOptions()
> +    : transport_type(0)
> +    , outbound_port(0)
> +    , f_outbound_transparent(false)
> +    , f_transparent_passthrough(false)
> +    , backdoor(false)
> +  {
> +    memcpy(host_res_preference, host_res_default_preference_order, sizeof(host_res_preference));
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setTransportType(int type) {
> +    transport_type =  type;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundIp(IpAddr& ip) {
> +    if (ip.isIp4()) outbound_ip4 = ip;
> +    else if (ip.isIp6()) outbound_ip6 = ip;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundIp(IpEndpoint* ip) {
> +    if (ip->isIp4()) outbound_ip4 = *ip;
> +    else if (ip->isIp6()) outbound_ip6 = *ip;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundPort(uint16_t port) {
> +    outbound_port = port;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setOutboundTransparent(bool flag) {
> +    f_outbound_transparent = flag;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setTransparentPassthrough(bool flag) {
> +    f_transparent_passthrough = flag;
> +    return *this;
> +  }
> +
> + inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setBackdoor(bool flag) {
> +    backdoor = flag;
> +    return *this;
> +  }
> +
> +  inline HttpAcceptContOptions&
> +  HttpAcceptContOptions::setHostResPreference(HostResPreferenceOrder const order) {
> +    memcpy(host_res_preference, order, sizeof(host_res_preference));
> +    return *this;
> +  }
> +}
> +
> +/**
> +   The continuation mutex is NULL to allow parellel accepts in NT. No
> +   state is recorded by the handler and values are required to be set
> +   during construction via the @c Options struct and never changed. So
> +   a NULL mutex is safe.
> +
> +   Most of the state is simply passed on to the @c ClientSession after
> +   an accept. It is done here because this is the least bad pathway
> +   from the top level configuration to the HTTP session.
> +*/
> +
> +class HttpAcceptCont: public AcceptCont, private detail::HttpAcceptContOptions
> +{
> +private:
> +  typedef HttpAcceptCont self; ///< Self reference type.
> +public:
> +  /** Construction options.
> +      Provide an easier to remember typedef for clients.
> +  */
> +  typedef detail::HttpAcceptContOptions Options;
> +
> +  /** Default constructor.
> +      @internal We don't use a static default options object because of
> +      initialization order issues. It is important to pick up data that is read
> +      from the config file and a static is initialized long before that point.
> +  */
> +  HttpAcceptCont(Options const& opt = Options())
> +    : AcceptCont(NULL)
> +    , detail::HttpAcceptContOptions(opt) // copy these.
> +  {
> +    SET_HANDLER(&HttpAcceptCont::mainEvent);
> +    return;
> +  }
> +
> +  ~HttpAcceptCont()
> +  {
> +    return;
> +  }
> +
> +  int mainEvent(int event, void *netvc);
> +
> +private:
> +    HttpAcceptCont(const HttpAcceptCont &);
> +    HttpAcceptCont & operator =(const HttpAcceptCont &);
> +};
> +
> +#endif
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/HttpProxyServerMain.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
> index 662856a..8988253 100644
> --- a/proxy/http/HttpProxyServerMain.cc
> +++ b/proxy/http/HttpProxyServerMain.cc
> @@ -26,7 +26,7 @@
> #include "Main.h"
> #include "Error.h"
> #include "HttpConfig.h"
> -#include "HttpAccept.h"
> +#include "HttpAcceptCont.h"
> #include "ReverseProxy.h"
> #include "HttpSessionManager.h"
> #include "HttpUpdateSM.h"
> @@ -35,9 +35,11 @@
> #include "HttpTunnel.h"
> #include "Tokenizer.h"
> #include "P_SSLNextProtocolAccept.h"
> +#include "P_ProtocolAcceptCont.h"
> +#include "P_SpdyAcceptCont.h"
> 
> -HttpAccept *plugin_http_accept = NULL;
> -HttpAccept *plugin_http_transparent_accept = 0;
> +HttpAcceptCont *plugin_http_accept = NULL;
> +HttpAcceptCont *plugin_http_transparent_accept = 0;
> 
> static SLL<SSLNextProtocolAccept> ssl_plugin_acceptors;
> static ink_mutex ssl_plugin_mutex = PTHREAD_MUTEX_INITIALIZER;
> @@ -135,9 +137,10 @@ static void
> MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned nthreads)
> {
>   NetProcessor::AcceptOptions& net_opt = acceptor._net_opt;
> -  HttpAccept::Options         accept_opt;
> +  HttpAcceptCont::Options         accept_opt;
> 
>   net_opt = make_net_accept_options(port, nthreads);
> +  net_opt.create_default_NetAccept = false;
>   REC_ReadConfigInteger(net_opt.recv_bufsize, "proxy.config.net.sock_recv_buffer_size_in");
>   REC_ReadConfigInteger(net_opt.send_bufsize, "proxy.config.net.sock_send_buffer_size_in");
>   REC_ReadConfigInteger(net_opt.packet_mark, "proxy.config.net.sock_packet_mark_in");
> @@ -160,21 +163,24 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
>     accept_opt.outbound_ip6 = HttpConfig::m_master.outbound_ip6;
>   }
> 
> -  if (port.isSSL()) {
> -    HttpAccept * accept = NEW(new HttpAccept(accept_opt));
> -    SSLNextProtocolAccept * ssl = NEW(new SSLNextProtocolAccept(accept));
> +  HttpAcceptCont *http = NEW(new HttpAcceptCont(accept_opt));
> +  SpdyAcceptCont *spdy = NEW(new SpdyAcceptCont(http));
> +  SSLNextProtocolAccept *ssl = NEW(new SSLNextProtocolAccept(http));
> +  ProtocolAcceptCont *proto = NEW(new ProtocolAcceptCont());
> +
> +  proto->registerEndpoint(TS_PROTO_TLS, ssl);
> +  proto->registerEndpoint(TS_PROTO_HTTP, http);
> +  proto->registerEndpoint(TS_PROTO_SPDY, spdy);
> 
> +  if (port.isSSL()) {
>     // ALPN selects the first server-offered protocol, so make sure that we offer HTTP/1.1 first.
> -    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, accept);
> -    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, accept);
> +    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_1, http);
> +    ssl->registerEndpoint(TS_NPN_PROTOCOL_HTTP_1_0, http);
> 
>     ink_scoped_mutex lock(ssl_plugin_mutex);
>     ssl_plugin_acceptors.push(ssl);
> -
> -    acceptor._accept = ssl;
> -  } else {
> -    acceptor._accept = NEW(new HttpAccept(accept_opt));
>   }
> +  acceptor._accept = proto;
> }
> 
> /** Set up all the accepts and sockets.
> @@ -197,14 +203,14 @@ init_HttpProxyServer(int n_accept_threads)
>   //   port but without going through the operating system
>   //
>   if (plugin_http_accept == NULL) {
> -    plugin_http_accept = NEW(new HttpAccept);
> +    plugin_http_accept = NEW(new HttpAcceptCont);
>     plugin_http_accept->mutex = new_ProxyMutex();
>   }
>   // Same as plugin_http_accept except outbound transparent.
>   if (! plugin_http_transparent_accept) {
> -    HttpAccept::Options ha_opt;
> +    HttpAcceptCont::Options ha_opt;
>     ha_opt.setOutboundTransparent(true);
> -    plugin_http_transparent_accept = NEW(new HttpAccept(ha_opt));
> +    plugin_http_transparent_accept = NEW(new HttpAcceptCont(ha_opt));
>     plugin_http_transparent_accept->mutex = new_ProxyMutex();
>   }
>   ink_mutex_init(&ssl_plugin_mutex, "SSL Acceptor List");
> @@ -262,7 +268,7 @@ void
> start_HttpProxyServerBackDoor(int port, int accept_threads)
> {
>   NetProcessor::AcceptOptions opt;
> -  HttpAccept::Options ha_opt;
> +  HttpAcceptCont::Options ha_opt;
> 
>   opt.local_port = port;
>   opt.accept_threads = accept_threads;
> @@ -271,5 +277,5 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
>   opt.backdoor = true;
> 
>   // The backdoor only binds the loopback interface
> -  netProcessor.main_accept(NEW(new HttpAccept(ha_opt)), NO_FD, opt);
> +  netProcessor.main_accept(NEW(new HttpAcceptCont(ha_opt)), NO_FD, opt);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/http/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/proxy/http/Makefile.am b/proxy/http/Makefile.am
> index c413ed7..a71d287 100644
> --- a/proxy/http/Makefile.am
> +++ b/proxy/http/Makefile.am
> @@ -31,14 +31,15 @@ AM_CPPFLAGS = \
>   -I$(top_srcdir)/proxy/hdrs \
>   -I$(top_srcdir)/proxy/shared \
>   -I$(top_srcdir)/proxy/http/remap \
> -  -I$(top_srcdir)/proxy/logging
> +  -I$(top_srcdir)/proxy/logging \
> +  -I$(top_srcdir)/proxy/spdy
> 
> noinst_HEADERS = HttpProxyServerMain.h
> noinst_LIBRARIES = libhttp.a
> 
> libhttp_a_SOURCES = \
> -  HttpAccept.cc \
> -  HttpAccept.h \
> +  HttpAcceptCont.cc \
> +  HttpAcceptCont.h \
>   HttpBodyFactory.cc \
>   HttpBodyFactory.h \
>   HttpCacheSM.cc \
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/Makefile.am b/proxy/spdy/Makefile.am
> new file mode 100644
> index 0000000..63911e1
> --- /dev/null
> +++ b/proxy/spdy/Makefile.am
> @@ -0,0 +1,38 @@
> +# Makefile.am for the traffic/iocore/net hierarchy
> +#
> +#  Licensed to the Apache Software Foundation (ASF) under one
> +#  or more contributor license agreements.  See the NOTICE file
> +#  distributed with this work for additional information
> +#  regarding copyright ownership.  The ASF licenses this file
> +#  to you under the Apache License, Version 2.0 (the
> +#  "License"); you may not use this file except in compliance
> +#  with the License.  You may obtain a copy of the License at
> +#
> +#      http://www.apache.org/licenses/LICENSE-2.0
> +#
> +#  Unless required by applicable law or agreed to in writing, software
> +#  distributed under the License is distributed on an "AS IS" BASIS,
> +#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +#  See the License for the specific language governing permissions and
> +#  limitations under the License.
> +
> +AM_CPPFLAGS = \
> +  $(iocore_include_dirs) \
> +  -I$(top_srcdir)/lib \
> +  -I$(top_srcdir)/lib/records \
> +  -I$(top_srcdir)/lib/ts \
> +  -I$(top_srcdir)/proxy \
> +  -I$(top_srcdir)/proxy/api \
> +  -I$(top_srcdir)/proxy/hdrs \
> +  -I$(top_srcdir)/proxy/shared \
> +  -I$(top_srcdir)/proxy/spdy \
> +  -I$(top_srcdir)/mgmt \
> +  -I$(top_srcdir)/mgmt/preparse \
> +  -I$(top_srcdir)/mgmt/utils \
> +  @SPDYLAY_CFLAGS@
> +
> +noinst_LIBRARIES = libspdy.a
> +
> +libspdy_a_SOURCES = \
> +  P_SpdyAcceptCont.h \
> +  SpdyAcceptCont.cc
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/P_SpdyAcceptCont.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/P_SpdyAcceptCont.h b/proxy/spdy/P_SpdyAcceptCont.h
> new file mode 100644
> index 0000000..88aea02
> --- /dev/null
> +++ b/proxy/spdy/P_SpdyAcceptCont.h
> @@ -0,0 +1,46 @@
> +/** @file
> +
> +  SpdyAcceptCont
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#ifndef P_SpdyAcceptCont_H_
> +#define P_SpdyAcceptCont_H_
> +
> +#include "P_Net.h"
> +#include "P_EventSystem.h"
> +#include "P_UnixNet.h"
> +#include "I_IOBuffer.h"
> +
> +class SpdyAcceptCont: public AcceptCont
> +{
> +public:
> +  SpdyAcceptCont(Continuation *ep);
> +  ~SpdyAcceptCont() {}
> +
> +private:
> +  int mainEvent(int event, void *netvc);
> +  SpdyAcceptCont(const SpdyAcceptCont &); // disabled
> +  SpdyAcceptCont& operator =(const SpdyAcceptCont&); // disabled
> +
> +  Continuation *endpoint;
> +};
> +
> +#endif /* P_SpdyAcceptCont_H_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f90f3a48/proxy/spdy/SpdyAcceptCont.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyAcceptCont.cc b/proxy/spdy/SpdyAcceptCont.cc
> new file mode 100644
> index 0000000..b9efc80
> --- /dev/null
> +++ b/proxy/spdy/SpdyAcceptCont.cc
> @@ -0,0 +1,37 @@
> +/** @file
> +
> +  SpdyNetAccept
> +
> +  @section license License
> +
> +  Licensed to the Apache Software Foundation (ASF) under one
> +  or more contributor license agreements.  See the NOTICE file
> +  distributed with this work for additional information
> +  regarding copyright ownership.  The ASF licenses this file
> +  to you under the Apache License, Version 2.0 (the
> +  "License"); you may not use this file except in compliance
> +  with the License.  You may obtain a copy of the License at
> +
> +      http://www.apache.org/licenses/LICENSE-2.0
> +
> +  Unless required by applicable law or agreed to in writing, software
> +  distributed under the License is distributed on an "AS IS" BASIS,
> +  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> +  See the License for the specific language governing permissions and
> +  limitations under the License.
> + */
> +
> +#include "P_SpdyAcceptCont.h"
> +
> +SpdyAcceptCont::SpdyAcceptCont(Continuation *ep)
> +    : AcceptCont(new_ProxyMutex()), endpoint(ep)
> +{
> +  SET_HANDLER(&SpdyAcceptCont::mainEvent);
> +}
> +
> +int
> +SpdyAcceptCont::mainEvent(int event, void *netvc)
> +{
> +  printf("spdy accepted\n");
> +  return 0;
> +}
>