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

git commit: TS-2751: Remove the ProtocolNetAccept layer

Repository: trafficserver
Updated Branches:
  refs/heads/master 110608611 -> 727811ef0


TS-2751: Remove the ProtocolNetAccept layer

Remove the ProtocolNetAccept mechanism. Remove the protocol probe
state from UnixNetVConnection. Move the protocol probe detection
all the way up to the session accept layer. Refactor NetVConnection
allocation so they are always allocated by NetProcessors.


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

Branch: refs/heads/master
Commit: 727811ef0870701ade427b0ed374ac42daec8c54
Parents: 1106086
Author: James Peach <jp...@apache.org>
Authored: Mon May 12 09:25:10 2014 -0700
Committer: James Peach <jp...@apache.org>
Committed: Fri May 16 15:08:39 2014 -0700

----------------------------------------------------------------------
 CHANGES                                   |   4 +-
 iocore/net/I_NetProcessor.h               |   3 +-
 iocore/net/I_SessionAccept.h              |   3 +-
 iocore/net/Makefile.am                    |   4 -
 iocore/net/P_Net.h                        |   4 +-
 iocore/net/P_NetAccept.h                  |  12 +--
 iocore/net/P_ProtocolNetAccept.h          |  62 -----------
 iocore/net/P_ProtocolProbeSessionAccept.h |  50 ---------
 iocore/net/P_SSLNetAccept.h               |   8 +-
 iocore/net/P_SSLNetProcessor.h            |   8 +-
 iocore/net/P_SSLNextProtocolAccept.h      |   2 +
 iocore/net/P_UnixNetProcessor.h           |   6 +-
 iocore/net/P_UnixNetVConnection.h         |   8 --
 iocore/net/ProtocolNetAccept.cc           |  68 ------------
 iocore/net/ProtocolProbeSessionAccept.cc  |  73 -------------
 iocore/net/SSLNetAccept.cc                |  27 +----
 iocore/net/SSLNetProcessor.cc             |  26 +++--
 iocore/net/SSLNextProtocolAccept.cc       |  21 +++-
 iocore/net/UnixNetAccept.cc               |  92 ++++++----------
 iocore/net/UnixNetProcessor.cc            |  56 ++++------
 iocore/net/UnixNetVConnection.cc          |  99 ++---------------
 proxy/Makefile.am                         |  15 ++-
 proxy/ProtocolProbeSessionAccept.cc       | 141 +++++++++++++++++++++++++
 proxy/ProtocolProbeSessionAccept.h        |  53 ++++++++++
 proxy/api/ts/InkAPIPrivateIOCore.h        |   6 --
 proxy/http/HttpClientSession.cc           |  13 +--
 proxy/http/HttpClientSession.h            |   4 +-
 proxy/http/HttpProxyServerMain.cc         |  29 +++--
 proxy/http/HttpSM.cc                      |  12 +--
 proxy/http/HttpSM.h                       |   5 +-
 proxy/http/HttpSessionAccept.cc           |  73 +++++++------
 proxy/http/HttpSessionAccept.h            |   1 +
 proxy/shared/UglyLogStubs.cc              |  10 +-
 proxy/spdy/SpdyCallbacks.cc               |  11 +-
 proxy/spdy/SpdySM.cc                      |  81 ++++++--------
 proxy/spdy/SpdySM.h                       |  16 ++-
 proxy/spdy/SpdySessionAccept.cc           |  27 ++++-
 proxy/spdy/SpdySessionAccept.h            |   2 +
 38 files changed, 469 insertions(+), 666 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 137f5be..f772da3 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.0.0
 
+  *) [TS-2751] Remove the ProtocolNetAccept layer.
+
   *) [TS-2815] SSL orgin server connection hangs if ssl handshake is slow
 
   *) [TS-2788] Make proxy.config.alarm.bin reloadable.
@@ -10,7 +12,7 @@ Changes with Apache Traffic Server 5.0.0
   *) [TS-2791] SPDY POST transactions failing with ERR_CLIENT_ABORT
 
   *) [TS-2805] Client connections are connecting with SPDY 3 instead of 3.1.
-  
+
   *) [TS-2619] Changed TSRecordDump declaration from TSRecordType to int to
    accommodate bit-masks. Also changed TSRecordType enums to hexidecimal, as
    this is easier to read for bit arguments.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/I_NetProcessor.h
----------------------------------------------------------------------
diff --git a/iocore/net/I_NetProcessor.h b/iocore/net/I_NetProcessor.h
index 68ce117..2cfa2ae 100644
--- a/iocore/net/I_NetProcessor.h
+++ b/iocore/net/I_NetProcessor.h
@@ -100,7 +100,6 @@ public:
     AcceptOptions() { this->reset(); }
     /// Reset all values to defaults.
     self& reset();
-    bool create_default_NetAccept;
   };
 
   /**
@@ -222,6 +221,8 @@ public:
   */
   virtual int start(int number_of_net_threads, size_t stacksize) = 0;
 
+  inkcoreapi virtual NetVConnection *allocate_vc(EThread *) = 0;
+
   /** Private constructor. */
   NetProcessor()
   {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/I_SessionAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/I_SessionAccept.h b/iocore/net/I_SessionAccept.h
index c7fc9c1..e6e3b4d 100644
--- a/iocore/net/I_SessionAccept.h
+++ b/iocore/net/I_SessionAccept.h
@@ -33,7 +33,8 @@ public:
   SessionAccept(ProxyMutex *amutex);
   ~SessionAccept();
 
-  //
+  virtual void accept(NetVConnection *, MIOBuffer *, IOBufferReader *) = 0;
+
   // Virtual function allows creation of an SSLNetAccept
   // or NetAccept transparent to NetProcessor.
   //

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/Makefile.am
----------------------------------------------------------------------
diff --git a/iocore/net/Makefile.am b/iocore/net/Makefile.am
index d7d3cc5..b58e10b 100644
--- a/iocore/net/Makefile.am
+++ b/iocore/net/Makefile.am
@@ -62,8 +62,6 @@ libinknet_a_SOURCES = \
   P_Net.h \
   P_NetAccept.h \
   P_NetVConnection.h \
-  P_ProtocolNetAccept.h \
-  P_ProtocolProbeSessionAccept.h \
   P_Socks.h \
   P_SSLCertLookup.h \
   P_SSLConfig.h \
@@ -85,8 +83,6 @@ libinknet_a_SOURCES = \
   P_UnixNetVConnection.h \
   P_UnixPollDescriptor.h \
   P_UnixUDPConnection.h \
-  ProtocolNetAccept.cc \
-  ProtocolProbeSessionAccept.cc \
   Socks.cc \
   SSLCertLookup.cc \
   SSLConfig.cc \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_Net.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_Net.h b/iocore/net/P_Net.h
index db9acc6..97a6184 100644
--- a/iocore/net/P_Net.h
+++ b/iocore/net/P_Net.h
@@ -104,7 +104,6 @@ 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"
@@ -123,5 +122,8 @@ do { \
 #define NetDebug Debug
 #endif
 
+/// Default amount of buffer space to use for the initial read on an incoming connection.
+/// This is an IOBufferBlock index, not the size in bytes.
+static size_t const CLIENT_CONNECTION_FIRST_READ_BUFFER_SIZE_INDEX = BUFFER_SIZE_INDEX_4K;
 
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_NetAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_NetAccept.h b/iocore/net/P_NetAccept.h
index 762a601..3efac3a 100644
--- a/iocore/net/P_NetAccept.h
+++ b/iocore/net/P_NetAccept.h
@@ -100,19 +100,13 @@ struct NetAccept:public Continuation
   UnixNetVConnection *epoll_vc; // only storage for epoll events
   EventIO ep;
 
-  // Functions all THREAD_FREE and THREAD_ALLOC to be performed
-  // for both SSL and regular NetVConnection transparent to
-  // accept functions.
-  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();
+  virtual EventType getEtype() const;
+  virtual NetProcessor * getNetProcessor() const;
 
   void init_accept_loop(const char *);
   virtual void init_accept(EThread * t = NULL);
   virtual void init_accept_per_thread();
-  virtual NetAccept *clone();
+  virtual NetAccept *clone() const;
   // 0 == success
   int do_listen(bool non_blocking, bool transparent = false);
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_ProtocolNetAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_ProtocolNetAccept.h b/iocore/net/P_ProtocolNetAccept.h
deleted file mode 100644
index ac49571..0000000
--- a/iocore/net/P_ProtocolNetAccept.h
+++ /dev/null
@@ -1,62 +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.
- */
-
-/****************************************************************************
-
-  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/727811ef/iocore/net/P_ProtocolProbeSessionAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_ProtocolProbeSessionAccept.h b/iocore/net/P_ProtocolProbeSessionAccept.h
deleted file mode 100644
index 424751c..0000000
--- a/iocore/net/P_ProtocolProbeSessionAccept.h
+++ /dev/null
@@ -1,50 +0,0 @@
-/** @file
-
-  ProtocolProbeSessionAccept
-
-  @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_ProtocolProbeSessionAccept_H_
-#define P_ProtocolProbeSessionAccept_H_
-
-#include "I_SessionAccept.h"
-
-class ProtocolProbeSessionAccept: public SessionAccept
-{
-public:
-  ProtocolProbeSessionAccept(): SessionAccept(NULL)
-  {
-    memset(endpoint, 0, sizeof(endpoint));
-    SET_HANDLER(&ProtocolProbeSessionAccept::mainEvent);
-  }
-  ~ProtocolProbeSessionAccept() {}
-
-  void *createNetAccept();
-  void registerEndpoint(TSProtoType type, Continuation *ep);
-
-private:
-  int mainEvent(int event, void * netvc);
-  ProtocolProbeSessionAccept(const ProtocolProbeSessionAccept &); // disabled
-  ProtocolProbeSessionAccept& operator =(const ProtocolProbeSessionAccept&); // disabled
-
-  Continuation *endpoint[sizeof(TSClientProtoStack) * CHAR_BIT];
-};
-
-#endif /* P_ProtocolProbeSessionAccept_H_ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_SSLNetAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNetAccept.h b/iocore/net/P_SSLNetAccept.h
index 6d963a3..d50d7ea 100644
--- a/iocore/net/P_SSLNetAccept.h
+++ b/iocore/net/P_SSLNetAccept.h
@@ -52,12 +52,10 @@ class UnixNetVConnection;
 //
 struct SSLNetAccept: public NetAccept
 {
-  virtual UnixNetVConnection *allocateThread(EThread * t);
-  virtual void freeThread(UnixNetVConnection * vc, EThread * t);
-  virtual UnixNetVConnection *allocateGlobal();
-  virtual EventType getEtype();
+  virtual NetProcessor * getNetProcessor() const;
+  virtual EventType getEtype() const;
   virtual void init_accept_per_thread();
-  virtual NetAccept *clone();
+  virtual NetAccept *clone() const;
 
   SSLNetAccept()
     { };

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_SSLNetProcessor.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNetProcessor.h b/iocore/net/P_SSLNetProcessor.h
index bdc00c3..2ff93d6 100644
--- a/iocore/net/P_SSLNetProcessor.h
+++ b/iocore/net/P_SSLNetProcessor.h
@@ -70,12 +70,8 @@ public:
   // to be upgraded to ET_SSL for SSLNetProcessor.
   virtual void upgradeEtype(EventType &etype);
 
-  // Functions all THREAD_FREE and THREAD_ALLOC to be performed
-  // for both SSL and regular NetVConnection transparent to
-  // netProcessor connect functions.
-  virtual UnixNetVConnection *allocateThread(EThread * t);
-  virtual void freeThread(UnixNetVConnection * vc, EThread * t);
-  virtual NetAccept *createNetAccept();
+  virtual NetAccept * createNetAccept();
+  virtual NetVConnection * allocate_vc(EThread *t);
 
 private:
   SSLNetProcessor(const SSLNetProcessor &);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_SSLNextProtocolAccept.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNextProtocolAccept.h b/iocore/net/P_SSLNextProtocolAccept.h
index 1e0a96d..2c53f20 100644
--- a/iocore/net/P_SSLNextProtocolAccept.h
+++ b/iocore/net/P_SSLNextProtocolAccept.h
@@ -37,6 +37,8 @@ public:
   SSLNextProtocolAccept(Continuation *);
   ~SSLNextProtocolAccept();
 
+  void accept(NetVConnection *, MIOBuffer *, IOBufferReader*);
+
   // Register handler as an endpoint for the specified protocol. Neither
   // handler nor protocol are copied, so the caller must guarantee their
   // lifetime is at least as long as that of the acceptor.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_UnixNetProcessor.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_UnixNetProcessor.h b/iocore/net/P_UnixNetProcessor.h
index a0ac3c5..47994c1 100644
--- a/iocore/net/P_UnixNetProcessor.h
+++ b/iocore/net/P_UnixNetProcessor.h
@@ -58,12 +58,8 @@ public:
   // nothing for NetProcessor
   virtual void upgradeEtype(EventType & /* etype ATS_UNUSED */) { };
 
-  // Functions all THREAD_FREE and THREAD_ALLOC to be performed
-  // for both SSL and regular NetVConnection transparent to
-  // netProcessor connect functions.
-  virtual UnixNetVConnection *allocateThread(EThread * t);
-  virtual void freeThread(UnixNetVConnection * vc, EThread * t);
   virtual NetAccept *createNetAccept();
+  virtual NetVConnection * allocate_vc(EThread *t);
 
   virtual int start(int number_of_net_threads, size_t stacksize);
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/P_UnixNetVConnection.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_UnixNetVConnection.h b/iocore/net/P_UnixNetVConnection.h
index 0e4eae8..0034654 100644
--- a/iocore/net/P_UnixNetVConnection.h
+++ b/iocore/net/P_UnixNetVConnection.h
@@ -41,13 +41,6 @@ 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()
 {
@@ -240,7 +233,6 @@ 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);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/ProtocolNetAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/ProtocolNetAccept.cc b/iocore/net/ProtocolNetAccept.cc
deleted file mode 100644
index c4257e2..0000000
--- a/iocore/net/ProtocolNetAccept.cc
+++ /dev/null
@@ -1,68 +0,0 @@
-/** @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/727811ef/iocore/net/ProtocolProbeSessionAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/ProtocolProbeSessionAccept.cc b/iocore/net/ProtocolProbeSessionAccept.cc
deleted file mode 100644
index a7aee6e..0000000
--- a/iocore/net/ProtocolProbeSessionAccept.cc
+++ /dev/null
@@ -1,73 +0,0 @@
-/** @file
-
-  ProtocolProbeSessionAccept
-
-  @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_ProtocolProbeSessionAccept.h"
-#include "P_SSLNextProtocolAccept.h"
-#include "P_Net.h"
-#include "I_Machine.h"
-#include "Error.h"
-
-void *
-ProtocolProbeSessionAccept::createNetAccept()
-{
-  return ((NetAccept *) NEW(new ProtocolNetAccept));
-}
-
-void
-ProtocolProbeSessionAccept::registerEndpoint(TSProtoType type, Continuation *ep)
-{
-  endpoint[type] = ep;
-}
-
-int
-ProtocolProbeSessionAccept::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/727811ef/iocore/net/SSLNetAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetAccept.cc b/iocore/net/SSLNetAccept.cc
index 6aa5ed9..ecd103e 100644
--- a/iocore/net/SSLNetAccept.cc
+++ b/iocore/net/SSLNetAccept.cc
@@ -28,32 +28,15 @@ typedef int (SSLNetAccept::*SSLNetAcceptHandler) (int, void *);
 // etype to be used in NetAccept functions (ET_SSL
 // or ET_NET).
 EventType
-SSLNetAccept::getEtype()
+SSLNetAccept::getEtype() const
 {
   return SSLNetProcessor::ET_SSL;
 }
 
-// Functions all THREAD_FREE and THREAD_ALLOC to be performed
-// for both SSL and regular NetVConnection transparent to
-// accept functions.
-UnixNetVConnection *
-SSLNetAccept::allocateThread(EThread *t)
+NetProcessor *
+SSLNetAccept::getNetProcessor() const
 {
-  return ((UnixNetVConnection *) THREAD_ALLOC(sslNetVCAllocator, t));
-}
-
-void
-SSLNetAccept::freeThread(UnixNetVConnection *vc, EThread *t)
-{
-  ink_assert(!vc->from_accept_thread);
-  THREAD_FREE((SSLNetVConnection *) vc, sslNetVCAllocator, t);
-}
-
-// This allocates directly on the class allocator, used for accept threads.
-UnixNetVConnection *
-SSLNetAccept::allocateGlobal()
-{
-  return (UnixNetVConnection *)sslNetVCAllocator.alloc();
+  return &sslNetProcessor;
 }
 
 void
@@ -85,7 +68,7 @@ SSLNetAccept::init_accept_per_thread()
 }
 
 NetAccept *
-SSLNetAccept::clone()
+SSLNetAccept::clone() const
 {
   NetAccept *na;
   na = NEW(new SSLNetAccept);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/SSLNetProcessor.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetProcessor.cc b/iocore/net/SSLNetProcessor.cc
index 094eeaf..4df64c3 100644
--- a/iocore/net/SSLNetProcessor.cc
+++ b/iocore/net/SSLNetProcessor.cc
@@ -92,22 +92,20 @@ SSLNetProcessor::upgradeEtype(EventType & etype)
   }
 }
 
-// Functions all THREAD_FREE and THREAD_ALLOC to be performed
-// for both SSL and regular NetVConnection transparent to
-// netProcessor connect functions. Yes it looks goofy to
-// have them in both places, but it saves a bunch of
-// connect code from being duplicated.
-UnixNetVConnection *
-SSLNetProcessor::allocateThread(EThread *t)
+NetVConnection *
+SSLNetProcessor::allocate_vc(EThread *t)
 {
-  return ((UnixNetVConnection *) THREAD_ALLOC(sslNetVCAllocator, t));
-}
+  SSLNetVConnection *vc;
+
+  if (t) {
+    vc = THREAD_ALLOC(sslNetVCAllocator, t);
+  } else {
+    if (likely(vc = sslNetVCAllocator.alloc())) {
+      vc->from_accept_thread = true;
+    }
+  }
 
-void
-SSLNetProcessor::freeThread(UnixNetVConnection *vc, EThread *t)
-{
-  ink_assert(!vc->from_accept_thread);
-  THREAD_FREE((SSLNetVConnection *) vc, sslNetVCAllocator, t);
+  return vc;
 }
 
 SSLNetProcessor::SSLNetProcessor()

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/SSLNextProtocolAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNextProtocolAccept.cc b/iocore/net/SSLNextProtocolAccept.cc
index 31c8cd4..39528b7 100644
--- a/iocore/net/SSLNextProtocolAccept.cc
+++ b/iocore/net/SSLNextProtocolAccept.cc
@@ -78,19 +78,24 @@ struct SSLNextProtocolTrampoline : public Continuation
     Continuation * plugin;
     SSLNetVConnection * netvc;
 
+    vio = static_cast<VIO *>(edata);
+    netvc = dynamic_cast<SSLNetVConnection *>(vio->vc_server);
+    ink_assert(netvc != NULL);
+
     switch (event) {
+    case VC_EVENT_EOS:
+    case VC_EVENT_ERROR:
+    case VC_EVENT_ACTIVE_TIMEOUT:
     case VC_EVENT_INACTIVITY_TIMEOUT:
+      netvc->do_io(VIO::CLOSE);
+      delete this;
+      return EVENT_CONT;
     case VC_EVENT_READ_COMPLETE:
-    case VC_EVENT_ERROR:
-      vio = static_cast<VIO *>(edata);
       break;
     default:
       return EVENT_ERROR;
     }
 
-    netvc = dynamic_cast<SSLNetVConnection *>(vio->vc_server);
-    ink_assert(netvc != NULL);
-
     plugin = netvc->endpoint();
     if (plugin) {
       send_plugin_event(plugin, NET_EVENT_ACCEPT, netvc);
@@ -132,6 +137,12 @@ SSLNextProtocolAccept::mainEvent(int event, void * edata)
   }
 }
 
+void
+SSLNextProtocolAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader* read)
+{
+  ink_release_assert(0);
+}
+
 bool
 SSLNextProtocolAccept::registerEndpoint(
     const char * protocol, Continuation * handler)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/UnixNetAccept.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetAccept.cc b/iocore/net/UnixNetAccept.cc
index 4962917..1e2c410 100644
--- a/iocore/net/UnixNetAccept.cc
+++ b/iocore/net/UnixNetAccept.cc
@@ -32,7 +32,7 @@ typedef int (NetAccept::*NetAcceptHandler) (int, void *);
 volatile int dummy_volatile = 0;
 int accept_till_done = 1;
 
-void
+static void
 safe_delay(int msec)
 {
   socketManager.poll(0, 0, msec);
@@ -43,7 +43,7 @@ safe_delay(int msec)
 // Send the throttling message to up to THROTTLE_AT_ONCE connections,
 // delaying to let some of the current connections complete
 //
-int
+static int
 send_throttle_message(NetAccept * na)
 {
   struct pollfd afd;
@@ -95,7 +95,7 @@ net_accept(NetAccept * na, void *ep, bool blockable)
   do {
     vc = (UnixNetVConnection *) na->alloc_cache;
     if (!vc) {
-      vc = na->allocateThread(e->ethread);
+      vc = (UnixNetVConnection *)na->getNetProcessor()->allocate_vc(e->ethread);
       NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
       vc->id = net_next_connection_number();
       na->alloc_cache = vc;
@@ -137,36 +137,6 @@ Ldone:
   return count;
 }
 
-
-UnixNetVConnection *
-NetAccept::allocateThread(EThread * t)
-{
-  return ((UnixNetVConnection *)THREAD_ALLOC(netVCAllocator, t));
-}
-
-void
-NetAccept::freeThread(UnixNetVConnection * vc, EThread * t)
-{
-  ink_assert(!vc->from_accept_thread);
-  THREAD_FREE(vc, netVCAllocator, t);
-}
-
-// This allocates directly on the class allocator, used for accept threads.
-UnixNetVConnection *
-NetAccept::allocateGlobal()
-{
-  return (UnixNetVConnection *)netVCAllocator.alloc();
-}
-
-// Virtual function allows the correct
-// etype to be used in NetAccept functions (ET_SSL
-// or ET_NET).
-EventType NetAccept::getEtype()
-{
-  return etype;
-}
-
-
 //
 // Initialize the NetAccept for execution in its own thread.
 // This should be done for low latency, high connection rate sockets.
@@ -236,15 +206,6 @@ 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)
 {
@@ -271,21 +232,6 @@ 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)
 {
@@ -330,11 +276,12 @@ NetAccept::do_blocking_accept(EThread * t)
     }
 
     // Use 'NULL' to Bypass thread allocator
-    vc = createSuitableVC(NULL, con);
+    vc = (UnixNetVConnection *)this->getNetProcessor()->allocate_vc(NULL);
     if (!vc) {
       con.close();
       return -1;
     }
+    vc->con = con;
     vc->from_accept_thread = true;
     vc->id = net_next_connection_number();
     alloc_cache = NULL;
@@ -468,12 +415,14 @@ NetAccept::acceptFastEvent(int event, void *ep)
         res = safe_nonblocking(fd);
       } while (res < 0 && (errno == EAGAIN || errno == EINTR));
 
-      vc = createSuitableVC(e->ethread, con);
+      vc = (UnixNetVConnection *)this->getNetProcessor()->allocate_vc(e->ethread);
       if (!vc) {
         con.close();
         goto Ldone;
       }
 
+      vc->con = con;
+
     } else {
       res = fd;
     }
@@ -534,7 +483,7 @@ Ldone:
 Lerror:
   server.close();
   e->cancel();
-  freeThread(vc, e->ethread);
+  vc->free(e->ethread);
   NET_DECREMENT_DYN_STAT(net_accepts_currently_open_stat);
   delete this;
   return EVENT_DONE;
@@ -589,3 +538,26 @@ NetAccept::cancel()
   action_->cancel();
   server.close();
 }
+
+NetAccept *
+NetAccept::clone() const
+{
+  NetAccept *na;
+  na = NEW(new NetAccept);
+  *na = *this;
+  return na;
+}
+
+// Virtual function allows the correct
+// etype to be used in NetAccept functions (ET_SSL
+// or ET_NET).
+EventType NetAccept::getEtype() const
+{
+  return etype;
+}
+
+NetProcessor *
+NetAccept::getNetProcessor() const
+{
+  return &netProcessor;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/UnixNetProcessor.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetProcessor.cc b/iocore/net/UnixNetProcessor.cc
index 825db85..46a2f3a 100644
--- a/iocore/net/UnixNetProcessor.cc
+++ b/iocore/net/UnixNetProcessor.cc
@@ -46,7 +46,6 @@ NetProcessor::AcceptOptions::reset()
   packet_mark = 0;
   packet_tos = 0;
   f_inbound_transparent = false;
-  create_default_NetAccept = true;
   return *this;
 }
 
@@ -86,18 +85,13 @@ Action *
 UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions const& opt)
 {
   EventType upgraded_etype = opt.etype; // setEtype requires non-const ref.
-  SessionAccept *acceptCont = static_cast<SessionAccept *>(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();
+  NetAccept *na = createNetAccept();
 
   // Potentially upgrade to SSL.
   upgradeEtype(upgraded_etype);
@@ -151,18 +145,15 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
   if (opt.frequent_accept) { // true
     if (accept_threads > 0)  {
       if (0 == na->do_listen(BLOCKING, opt.f_inbound_transparent)) {
-        NetAccept *a;
 
         for (int i=1; i < accept_threads; ++i) {
-          if (opt.create_default_NetAccept)
-            a = createNetAccept();
-          else
-            a = (NetAccept *)acceptCont->createNetAccept();
-          *a = *na;
+          NetAccept * a = na->clone();
+
           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);
           Debug("iocore_net_accept", "Created accept thread #%d for port %d", i, ats_ip_port_host_order(&accept_ip));
         }
+
         // Start the "template" accept thread last.
         Debug("iocore_net_accept", "Created accept thread #%d for port %d", accept_threads, ats_ip_port_host_order(&accept_ip));
         snprintf(thr_name, MAX_THREAD_NAME_LENGTH, "[ACCEPT %d:%d]", accept_threads-1, ats_ip_port_host_order(&accept_ip));
@@ -171,8 +162,9 @@ UnixNetProcessor::accept_internal(Continuation *cont, int fd, AcceptOptions cons
     } else {
       na->init_accept_per_thread();
     }
-  } else
+  } else {
     na->init_accept();
+  }
 
 #ifdef TCP_DEFER_ACCEPT
   // set tcp defer accept timeout if it is configured, this will not trigger an accept until there is
@@ -202,7 +194,7 @@ UnixNetProcessor::connect_re_internal(
 ) {
   ProxyMutex *mutex = cont->mutex;
   EThread *t = mutex->thread_holding;
-  UnixNetVConnection *vc = allocateThread(t);
+  UnixNetVConnection *vc = (UnixNetVConnection *)this->allocate_vc(t);
 
   if (opt)
     vc->options = *opt;
@@ -453,24 +445,6 @@ UnixNetProcessor::start(int, size_t)
   return 1;
 }
 
-// Functions all THREAD_FREE and THREAD_ALLOC to be performed
-// for both SSL and regular UnixNetVConnection transparent to
-// netProcessor connect functions. Yes it looks goofy to
-// have them in both places, but it saves a bunch of
-// code from being duplicated.
-UnixNetVConnection *
-UnixNetProcessor::allocateThread(EThread * t)
-{
-  return ((UnixNetVConnection *) THREAD_ALLOC(netVCAllocator, t));
-}
-
-void
-UnixNetProcessor::freeThread(UnixNetVConnection * vc, EThread * t)
-{
-  ink_assert(!vc->from_accept_thread);
-  THREAD_FREE(vc, netVCAllocator, t);
-}
-
 // Virtual function allows creation of an
 // SSLNetAccept or NetAccept transparent to NetProcessor.
 NetAccept *
@@ -479,6 +453,22 @@ UnixNetProcessor::createNetAccept()
   return (NEW(new NetAccept));
 }
 
+NetVConnection *
+UnixNetProcessor::allocate_vc(EThread *t)
+{
+  UnixNetVConnection *vc;
+
+  if (t) {
+    vc = THREAD_ALLOC(netVCAllocator, t);
+  } else {
+    if (likely(vc = netVCAllocator.alloc())) {
+      vc->from_accept_thread = true;
+    }
+  }
+
+  return vc;
+}
+
 struct socks_conf_struct *
 NetProcessor::socks_conf_stuff = NULL;
 int NetProcessor::accept_mss = 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/iocore/net/UnixNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
index 105a6c5..ea9f22a 100644
--- a/iocore/net/UnixNetVConnection.cc
+++ b/iocore/net/UnixNetVConnection.cc
@@ -38,84 +38,6 @@ 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");
 
@@ -337,12 +259,9 @@ 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) {
-        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);
+        r = socketManager.read(vc->con.fd, tiovec[0].iov_base, tiovec[0].iov_len);
       } else {
         r = socketManager.readv(vc->con.fd, &tiovec[0], niov);
       }
@@ -881,7 +800,7 @@ UnixNetVConnection::UnixNetVConnection()
 #endif
     active_timeout(NULL), nh(NULL),
     id(0), flags(0), recursion(0), submit_time(0), oob_ptr(0),
-    from_accept_thread(false), probe_state(SPDY_PROBE_STATE_NONE),
+    from_accept_thread(false),
     selected_next_protocol(NULL)
 {
   memset(&local_addr, 0, sizeof local_addr);
@@ -1065,17 +984,15 @@ UnixNetVConnection::acceptEvent(int event, Event *e)
 
   nh->open_list.enqueue(this);
 
-  if (inactivity_timeout_in)
+  if (inactivity_timeout_in) {
     UnixNetVConnection::set_inactivity_timeout(inactivity_timeout_in);
-  if (active_timeout_in)
+  }
+
+  if (active_timeout_in) {
     UnixNetVConnection::set_active_timeout(active_timeout_in);
-  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);
   }
 
+  action_.continuation->handleEvent(NET_EVENT_ACCEPT, this);
   return EVENT_DONE;
 }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/Makefile.am
----------------------------------------------------------------------
diff --git a/proxy/Makefile.am b/proxy/Makefile.am
index 206fc43..e4fd83d 100644
--- a/proxy/Makefile.am
+++ b/proxy/Makefile.am
@@ -38,6 +38,7 @@ AM_CPPFLAGS = \
   -I$(top_srcdir)/lib/records \
   -I$(top_srcdir)/lib/ts \
   -I$(srcdir)/http \
+  -I$(srcdir)/spdy \
   -I$(srcdir)/logging \
   -I$(srcdir)/http/remap  \
   -I$(srcdir)/hdrs \
@@ -141,21 +142,21 @@ traffic_server_SOURCES = \
   CoreUtils.h \
   DynamicStats.h \
   EventName.cc \
+  FetchSM.cc \
   HttpTransStats.h \
   ICP.cc \
   ICP.h \
   ICPConfig.cc \
-  ICPevents.h \
-  ICPlog.h \
   ICPProcessor.cc \
   ICPProcessor.h \
   ICPStats.cc \
+  ICPevents.h \
+  ICPlog.h \
+  IPAllow.cc \
+  IPAllow.h \
   InkAPI.cc \
-  FetchSM.cc \
   InkAPIInternal.h \
   InkIOCoreAPI.cc \
-  IPAllow.cc \
-  IPAllow.h \
   Main.cc \
   Main.h \
   ParentSelection.cc \
@@ -166,6 +167,8 @@ traffic_server_SOURCES = \
   PluginVC.h \
   Prefetch.cc \
   Prefetch.h \
+  ProtocolProbeSessionAccept.cc \
+  ProtocolProbeSessionAccept.h \
   ReverseProxy.cc \
   ReverseProxy.h \
   SocksProxy.cc \
@@ -284,6 +287,8 @@ traffic_sac_SOURCES = \
   AbstractBuffer.cc \
   Transform.cc \
   Prefetch.cc \
+  ProtocolProbeSessionAccept.cc \
+  ProtocolProbeSessionAccept.h \
   Update.cc \
   Plugin.cc \
   InkAPI.cc \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/ProtocolProbeSessionAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/ProtocolProbeSessionAccept.cc b/proxy/ProtocolProbeSessionAccept.cc
new file mode 100644
index 0000000..33702d5
--- /dev/null
+++ b/proxy/ProtocolProbeSessionAccept.cc
@@ -0,0 +1,141 @@
+/** @file
+
+  ProtocolProbeSessionAccept
+
+  @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_Net.h"
+#include "I_Machine.h"
+#include "ProtocolProbeSessionAccept.h"
+#include "Error.h"
+
+struct ProtocolProbeTrampoline : public Continuation
+{
+  static const size_t minimum_read_size = 1;
+  static const unsigned buffer_size_index = CLIENT_CONNECTION_FIRST_READ_BUFFER_SIZE_INDEX;
+
+  explicit
+  ProtocolProbeTrampoline(const ProtocolProbeSessionAccept * probe, ProxyMutex * mutex)
+    : Continuation(mutex), probeParent(probe)
+  {
+    this->iobuf = new_MIOBuffer(buffer_size_index);
+    SET_HANDLER(&ProtocolProbeTrampoline::ioCompletionEvent);
+  }
+
+  int ioCompletionEvent(int event, void * edata)
+  {
+    VIO *             vio;
+    IOBufferReader *  reader;
+    NetVConnection *  netvc;
+    TSProtoType       proto_type = TS_PROTO_NULL;
+
+    vio = static_cast<VIO *>(edata);
+    netvc = static_cast<NetVConnection *>(vio->vc_server);
+
+    switch (event) {
+    case VC_EVENT_EOS:
+    case VC_EVENT_ERROR:
+    case VC_EVENT_ACTIVE_TIMEOUT:
+    case VC_EVENT_INACTIVITY_TIMEOUT:
+      // Error ....
+      netvc->do_io_close();
+      goto done;
+    case VC_EVENT_READ_READY:
+    case VC_EVENT_READ_COMPLETE:
+      break;
+    default:
+      return EVENT_ERROR;
+    }
+
+    reader = iobuf->alloc_reader();
+    ink_assert(netvc != NULL);
+
+    if (!reader->is_read_avail_more_than(minimum_read_size - 1)) {
+      // Not enough data read. Well, that sucks.
+      netvc->do_io_close();
+      goto done;
+    }
+
+    // SPDY clients have to start by sending a control frame (the high bit is set). Let's assume
+    // that no other protocol could possibly ever set this bit!
+    if ((uint8_t)(*reader->start()) == 0x80u) {
+      proto_type = TS_PROTO_SPDY;
+    } else {
+      proto_type = TS_PROTO_HTTP;
+    }
+
+    netvc->do_io_read(this, 0, NULL); // Disable the read IO that we started.
+    netvc->proto_stack |= (1u << proto_type);
+
+    if (probeParent->endpoint[proto_type] == NULL) {
+      Warning("Unregistered protocol type %d", proto_type);
+      netvc->do_io_close();
+      goto done;
+    }
+
+    // Directly invoke the session acceptor, letting it take ownership of the input buffer.
+    probeParent->endpoint[proto_type]->accept(netvc, this->iobuf, reader);
+    delete this;
+    return EVENT_CONT;
+
+done:
+    free_MIOBuffer(this->iobuf);
+    delete this;
+    return EVENT_CONT;
+  }
+
+  MIOBuffer * iobuf;
+  const ProtocolProbeSessionAccept * probeParent;
+};
+
+int
+ProtocolProbeSessionAccept::mainEvent(int event, void *data)
+{
+  if (event == NET_EVENT_ACCEPT) {
+    ink_assert(data);
+
+    VIO * vio;
+    NetVConnection * netvc = static_cast<NetVConnection*>(data);
+    ProtocolProbeTrampoline * probe = NEW(new ProtocolProbeTrampoline(this, netvc->mutex));
+
+    // XXX we need to apply accept inactivity timeout here ...
+
+    vio = netvc->do_io_read(probe,
+		    BUFFER_SIZE_FOR_INDEX(ProtocolProbeTrampoline::buffer_size_index), probe->iobuf);
+    vio->reenable();
+    return EVENT_CONT;
+  }
+
+  MachineFatal("Protocol probe received a fatal error: errno = %d", -((int)(intptr_t)data));
+  return EVENT_CONT;
+}
+
+void
+ProtocolProbeSessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader* read)
+{
+  ink_release_assert(0);
+}
+
+void
+ProtocolProbeSessionAccept::registerEndpoint(TSProtoType proto_type, SessionAccept * ap)
+{
+  ink_release_assert(endpoint[proto_type] == NULL);
+  this->endpoint[proto_type] = ap;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/ProtocolProbeSessionAccept.h
----------------------------------------------------------------------
diff --git a/proxy/ProtocolProbeSessionAccept.h b/proxy/ProtocolProbeSessionAccept.h
new file mode 100644
index 0000000..9ad8c3d
--- /dev/null
+++ b/proxy/ProtocolProbeSessionAccept.h
@@ -0,0 +1,53 @@
+/** @file
+
+  ProtocolProbeSessionAccept
+
+  @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 ProtocolProbeSessionAccept_H_
+#define ProtocolProbeSessionAccept_H_
+
+#include "I_SessionAccept.h"
+
+class ProtocolProbeSessionAccept: public SessionAccept
+{
+public:
+  ProtocolProbeSessionAccept(): SessionAccept(NULL)
+  {
+    memset(endpoint, 0, sizeof(endpoint));
+    SET_HANDLER(&ProtocolProbeSessionAccept::mainEvent);
+  }
+  ~ProtocolProbeSessionAccept() {}
+
+  void registerEndpoint(TSProtoType proto_type, SessionAccept * ap);
+
+  void accept(NetVConnection *, MIOBuffer *, IOBufferReader*);
+
+private:
+  int mainEvent(int event, void * netvc);
+  ProtocolProbeSessionAccept(const ProtocolProbeSessionAccept &); // disabled
+  ProtocolProbeSessionAccept& operator =(const ProtocolProbeSessionAccept&); // disabled
+
+  SessionAccept * endpoint[sizeof(TSClientProtoStack) * CHAR_BIT];
+
+friend struct ProtocolProbeTrampoline;
+};
+
+#endif /* ProtocolProbeSessionAccept_H_ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/api/ts/InkAPIPrivateIOCore.h
----------------------------------------------------------------------
diff --git a/proxy/api/ts/InkAPIPrivateIOCore.h b/proxy/api/ts/InkAPIPrivateIOCore.h
index 97a9bcd..fab4b4a 100644
--- a/proxy/api/ts/InkAPIPrivateIOCore.h
+++ b/proxy/api/ts/InkAPIPrivateIOCore.h
@@ -51,10 +51,6 @@ public:
 
   void handle_event_count(int event);
   int handle_event(int event, void *edata);
-  virtual void *createNetAccept()
-  {
-    return (NEW(new NetAccept));
-  }
 
 public:
   void *mdata;
@@ -67,8 +63,6 @@ public:
   INKContInternalMagic_t m_free_magic;
 };
 
-
-
 class INKVConnInternal:public INKContInternal
 {
 public:

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/http/HttpClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
index d98d48a..48ca3b6 100644
--- a/proxy/http/HttpClientSession.cc
+++ b/proxy/http/HttpClientSession.cc
@@ -110,13 +110,6 @@ HttpClientSession::destroy()
   THREAD_FREE(this, httpClientSessionAllocator, this_thread());
 }
 
-HttpClientSession *
-HttpClientSession::allocate()
-{
-  ink_assert(0);
-  return NULL;
-}
-
 void
 HttpClientSession::ssn_hook_append(TSHttpHookID id, INKContInternal * cont)
 {
@@ -173,7 +166,7 @@ HttpClientSession::do_api_callout(TSHttpHookID id)
 }
 
 void
-HttpClientSession::new_connection(NetVConnection * new_vc, bool backdoor)
+HttpClientSession::new_connection(NetVConnection * new_vc, bool backdoor, MIOBuffer * iobuf, IOBufferReader * reader)
 {
 
   ink_assert(new_vc != NULL);
@@ -223,8 +216,8 @@ HttpClientSession::new_connection(NetVConnection * new_vc, bool backdoor)
 
   DebugSsn("http_cs", "[%" PRId64 "] session born, netvc %p", con_id, new_vc);
 
-  read_buffer = new_MIOBuffer(HTTP_HEADER_BUFFER_SIZE_INDEX);
-  sm_reader = read_buffer->alloc_reader();
+  read_buffer = iobuf ? iobuf : new_MIOBuffer(HTTP_HEADER_BUFFER_SIZE_INDEX);
+  sm_reader = reader ? reader : read_buffer->alloc_reader();
 
   // INKqa11186: Use a local pointer to the mutex as
   // when we return from do_api_callout, the ClientSession may

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/http/HttpClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpClientSession.h b/proxy/http/HttpClientSession.h
index d41d5a5..8daf04f 100644
--- a/proxy/http/HttpClientSession.h
+++ b/proxy/http/HttpClientSession.h
@@ -52,9 +52,7 @@ public:
   void cleanup();
   virtual void destroy();
 
-  static HttpClientSession *allocate();
-
-  void new_connection(NetVConnection * new_vc, bool backdoor = false);
+  void new_connection(NetVConnection * new_vc, bool backdoor, MIOBuffer * iobuf, IOBufferReader * reader);
 
   virtual VIO *do_io_read(Continuation * c, int64_t nbytes = INT64_MAX, MIOBuffer * buf = 0);
   virtual VIO *do_io_write(Continuation * c = NULL, int64_t nbytes = INT64_MAX, IOBufferReader * buf = 0, bool owner = false);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/http/HttpProxyServerMain.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
index e7a26b9..c9fdbde 100644
--- a/proxy/http/HttpProxyServerMain.cc
+++ b/proxy/http/HttpProxyServerMain.cc
@@ -35,7 +35,7 @@
 #include "HttpTunnel.h"
 #include "Tokenizer.h"
 #include "P_SSLNextProtocolAccept.h"
-#include "P_ProtocolProbeSessionAccept.h"
+#include "ProtocolProbeSessionAccept.h"
 #include "SpdySessionAccept.h"
 
 HttpSessionAccept *plugin_http_accept = NULL;
@@ -140,7 +140,6 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
   HttpSessionAccept::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");
@@ -163,17 +162,26 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
     accept_opt.outbound_ip6 = HttpConfig::m_master.outbound_ip6;
   }
 
+  // OK the way this works is that the fallback for each port is a protocol
+  // probe acceptor. For SSL ports, we can stack a NPN+ALPN acceptor in front
+  // of that, and these ports will fall back to the probe if no NPN+ALPN endpoint
+  // was negotiated.
+
+  // XXX the protocol probe should be a configuration option.
+
+  ProtocolProbeSessionAccept *probe = NEW(new ProtocolProbeSessionAccept());
   HttpSessionAccept *http = NEW(new HttpSessionAccept(accept_opt));
+
+#if TS_HAS_SPDY
   SpdySessionAccept *spdy = NEW(new SpdySessionAccept(http));
-  SSLNextProtocolAccept *ssl = NEW(new SSLNextProtocolAccept(http));
-  ProtocolProbeSessionAccept *proto = NEW(new ProtocolProbeSessionAccept());
+  probe->registerEndpoint(TS_PROTO_SPDY, spdy);
+#endif
 
-  proto->registerEndpoint(TS_PROTO_TLS, ssl);
-  proto->registerEndpoint(TS_PROTO_HTTP, http);
-  proto->registerEndpoint(TS_PROTO_SPDY, spdy);
+  probe->registerEndpoint(TS_PROTO_HTTP, http);
 
   if (port.isSSL()) {
-    //
+    SSLNextProtocolAccept *ssl = NEW(new SSLNextProtocolAccept(probe));
+
     // ALPN selects the first server-offered protocol,
     // so make sure that we offer the newest protocol first.
     // But since registerEndpoint prepends you want to
@@ -193,8 +201,11 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
 
     ink_scoped_mutex lock(ssl_plugin_mutex);
     ssl_plugin_acceptors.push(ssl);
+
+    acceptor._accept = ssl;
+  } else {
+    acceptor._accept = probe;
   }
-  acceptor._accept = proto;
 }
 
 /** Set up all the accepts and sockets.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 578c15d..43b56bc 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -5531,7 +5531,7 @@ HttpSM::setup_server_send_request()
 
   // Send the request header
   server_entry->vc_handler = &HttpSM::state_send_server_request_header;
-  server_entry->write_buffer = new_MIOBuffer(buffer_size_to_index(HTTP_HEADER_BUFFER_SIZE));
+  server_entry->write_buffer = new_MIOBuffer(HTTP_HEADER_BUFFER_SIZE_INDEX);
 
   if (t_state.api_server_request_body_set) {
     msg_len = t_state.internal_msg_buffer_size;
@@ -5637,7 +5637,7 @@ HttpSM::setup_cache_read_transfer()
   ink_assert(cache_sm.cache_read_vc != NULL);
 
   doc_size = t_state.cache_info.object_read->object_size_get();
-  alloc_index = buffer_size_to_index(doc_size + HTTP_HEADER_BUFFER_SIZE);
+  alloc_index = buffer_size_to_index(doc_size + index_to_buffer_size(HTTP_HEADER_BUFFER_SIZE_INDEX));
 
 #ifndef USE_NEW_EMPTY_MIOBUFFER
   MIOBuffer *buf = new_MIOBuffer(alloc_index);
@@ -5735,9 +5735,7 @@ HttpSM::setup_cache_write_transfer(HttpCacheSM * c_sm,
 void
 HttpSM::setup_100_continue_transfer()
 {
-  int64_t buf_size = HTTP_HEADER_BUFFER_SIZE;
-
-  MIOBuffer *buf = new_MIOBuffer(buffer_size_to_index(buf_size));
+  MIOBuffer *buf = new_MIOBuffer(HTTP_HEADER_BUFFER_SIZE_INDEX);
   IOBufferReader *buf_start = buf->alloc_reader();
 
   // First write the client response header into the buffer
@@ -5837,7 +5835,7 @@ HttpSM::setup_internal_transfer(HttpSMHandler handler_arg)
 
   t_state.source = HttpTransact::SOURCE_INTERNAL;
 
-  int64_t buf_size = HTTP_HEADER_BUFFER_SIZE + (is_msg_buf_present ? t_state.internal_msg_buffer_size : 0);
+  int64_t buf_size = index_to_buffer_size(HTTP_HEADER_BUFFER_SIZE_INDEX) + (is_msg_buf_present ? t_state.internal_msg_buffer_size : 0);
 
   MIOBuffer *buf = new_MIOBuffer(buffer_size_to_index(buf_size));
   IOBufferReader *buf_start = buf->alloc_reader();
@@ -5907,7 +5905,7 @@ HttpSM::find_http_resp_buffer_size(int64_t content_length)
 #ifdef WRITE_AND_TRANSFER
     buf_size = HTTP_HEADER_BUFFER_SIZE + content_length - index_to_buffer_size(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
 #else
-    buf_size = HTTP_HEADER_BUFFER_SIZE + content_length;
+    buf_size = index_to_buffer_size(HTTP_HEADER_BUFFER_SIZE_INDEX) + content_length;
 #endif
     alloc_index = buffer_size_to_index(buf_size);
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/http/HttpSM.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
index 9c1b58f..d5675d3 100644
--- a/proxy/http/HttpSM.h
+++ b/proxy/http/HttpSM.h
@@ -62,15 +62,14 @@
 
 // The default size for http header buffers when we don't
 //   need to include extra space for the document
-#define HTTP_HEADER_BUFFER_SIZE       2048
-#define HTTP_HEADER_BUFFER_SIZE_INDEX BUFFER_SIZE_INDEX_4K      //changed by YTS Team, yamsat for BUGID-59651
+static size_t const HTTP_HEADER_BUFFER_SIZE_INDEX = CLIENT_CONNECTION_FIRST_READ_BUFFER_SIZE_INDEX;
 
 // We want to use a larger buffer size when reading response
 //   headers from the origin server since we want to get
 //   as much of the document as possible on the first read
 //   Marco benchmarked about 3% ops/second improvement using
 //   the larger buffer size
-#define HTTP_SERVER_RESP_HDR_BUFFER_INDEX BUFFER_SIZE_INDEX_8K
+static size_t const HTTP_SERVER_RESP_HDR_BUFFER_INDEX = BUFFER_SIZE_INDEX_8K;
 
 class HttpServerSession;
 class AuthHttpAdapter;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/http/HttpSessionAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSessionAccept.cc b/proxy/http/HttpSessionAccept.cc
index 62bbdee..e485764 100644
--- a/proxy/http/HttpSessionAccept.cc
+++ b/proxy/http/HttpSessionAccept.cc
@@ -27,51 +27,58 @@
 #include "I_Machine.h"
 #include "Error.h"
 
-int
-HttpSessionAccept::mainEvent(int event, void *data)
+void
+HttpSessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
 {
-  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) {
+  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)) {
     ////////////////////////////////////////////////////
     // 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;
+    Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
+    netvc->do_io_close();
+
+    return;
+  }
 
-    // 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();
+  netvc->attributes = transport_type;
 
-      return VC_EVENT_CONT;
-    }
+  if (is_debug_tag_set("http_seq")) {
+    Debug("http_seq", "[HttpSessionAccept:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
+  }
 
-    netvc->attributes = transport_type;
+  HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, this_ethread());
 
-    if (is_debug_tag_set("http_seq"))
-      Debug("http_seq", "[HttpSessionAccept:mainEvent %p] accepted connection from %s transport type = %d", netvc, ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
+  // 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;
 
-    HttpClientSession *new_session = THREAD_ALLOC_INIT(httpClientSessionAllocator, this_ethread());
+  new_session->new_connection(netvc, backdoor, iobuf, reader);
 
-   // 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;
+  return;
+}
 
-    new_session->new_connection(netvc, backdoor);
+int
+HttpSessionAccept::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) {
+    this->accept(static_cast<NetVConnection *>(data), NULL, NULL);
     return EVENT_CONT;
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/http/HttpSessionAccept.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSessionAccept.h b/proxy/http/HttpSessionAccept.h
index 49947af..67b7c62 100644
--- a/proxy/http/HttpSessionAccept.h
+++ b/proxy/http/HttpSessionAccept.h
@@ -188,6 +188,7 @@ public:
     return;
   }
 
+  void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
   int mainEvent(int event, void *netvc);
 
 private:

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/shared/UglyLogStubs.cc
----------------------------------------------------------------------
diff --git a/proxy/shared/UglyLogStubs.cc b/proxy/shared/UglyLogStubs.cc
index 9690571..ad9e38e 100644
--- a/proxy/shared/UglyLogStubs.cc
+++ b/proxy/shared/UglyLogStubs.cc
@@ -180,19 +180,13 @@ UnixNetProcessor::accept_internal(Continuation * /* cont ATS_UNUSED */, int /* f
   return NULL;
 }
 
-UnixNetVConnection *
-UnixNetProcessor::allocateThread(EThread * /* t ATS_UNUSED */)
+NetVConnection *
+UnixNetProcessor::allocate_vc(EThread *)
 {
   ink_release_assert(false);
   return NULL;
 }
 
-void
-UnixNetProcessor::freeThread(UnixNetVConnection * /* vc ATS_UNUSED */, EThread * /* t ATS_UNUSED */)
-{
-  ink_release_assert(false);
-}
-
 // For Intel ICC
 int cache_config_mutex_retry_delay = 2;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/spdy/SpdyCallbacks.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyCallbacks.cc b/proxy/spdy/SpdyCallbacks.cc
index 4a80685..647d3b5 100644
--- a/proxy/spdy/SpdyCallbacks.cc
+++ b/proxy/spdy/SpdyCallbacks.cc
@@ -173,7 +173,7 @@ spdy_fetcher_launch(SpdyRequest *req, TSFetchMethod method)
   SpdySM *sm = req->spdy_sm;
 
   url = req->scheme + "://" + req->host + req->path;
-  client_addr = TSNetVConnRemoteAddrGet(sm->net_vc);
+  client_addr = TSNetVConnRemoteAddrGet(reinterpret_cast<TSVConn>(sm->vc));
 
   req->url = url;
   Debug("spdy", "++++Request[%" PRIu64 ":%d] %s", sm->sm_id, req->stream_id, req->url.c_str());
@@ -190,7 +190,7 @@ spdy_fetcher_launch(SpdyRequest *req, TSFetchMethod method)
   //
   // Set client protocol stack in FetchSM that needed by logging module
   //
-  NetVConnection *netvc = (NetVConnection *)sm->net_vc;
+  NetVConnection *netvc = (NetVConnection *)sm->vc;
   TSFetchClientProtoStackSet(req->fetch_sm, netvc->proto_stack);
 
   //
@@ -256,7 +256,12 @@ spdy_recv_callback(spdylay_session * /*session*/, uint8_t *buf, size_t length,
   }
 
   TSIOBufferReaderConsume(sm->req_reader, already);
-  TSVIOReenable(sm->read_vio);
+
+  // This is a bit of a hack. If we are reading out of the buffer the protocol probe acceptor gave us, then we have not
+  // kicked off our own I/O yet. After consuming this data we will come back and do that.
+  if (sm->read_vio) {
+    TSVIOReenable(sm->read_vio);
+  }
 
   if (!already)
     return SPDYLAY_ERR_WOULDBLOCK;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/spdy/SpdySM.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySM.cc b/proxy/spdy/SpdySM.cc
index 4fe2d4d..df66f39 100644
--- a/proxy/spdy/SpdySM.cc
+++ b/proxy/spdy/SpdySM.cc
@@ -56,38 +56,24 @@ SpdyRequest::clear()
   Debug("spdy", "****Delete Request[%" PRIu64 ":%d]", spdy_sm->sm_id, stream_id);
 }
 
-SpdySM::SpdySM():
-  net_vc(NULL), contp(NULL),
-  req_buffer(NULL), req_reader(NULL),
-  resp_buffer(NULL), resp_reader(NULL),
-  read_vio(NULL), write_vio(NULL), session(NULL)
-{}
-
-SpdySM::SpdySM(TSVConn conn):
-  net_vc(NULL), contp(NULL),
-  req_buffer(NULL), req_reader(NULL),
-  resp_buffer(NULL), resp_reader(NULL),
-  read_vio(NULL), write_vio(NULL), session(NULL)
-
-{
-  init(conn);
-}
-
 void
-SpdySM::init(TSVConn conn)
+SpdySM::init(NetVConnection * netvc)
 {
   int version, r;
-  UnixNetVConnection *vc;
 
-  net_vc = conn;
-  req_map.clear();
-  vc = (UnixNetVConnection *)(conn);
+  atomic_inc(g_sm_cnt);
+
+  this->vc = netvc;
+  this->req_map.clear();
 
-  if (vc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_3_1)
+  // XXX this has to die ... TS-2793
+  UnixNetVConnection * unixvc = reinterpret_cast<UnixNetVConnection *>(netvc);
+
+  if (unixvc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_3_1)
     version = SPDYLAY_PROTO_SPDY3_1;
-  else if (vc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_3)
+  else if (unixvc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_3)
     version = SPDYLAY_PROTO_SPDY3;
-  else if (vc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_2)
+  else if (unixvc->selected_next_protocol == TS_NPN_PROTOCOL_SPDY_2)
     version = SPDYLAY_PROTO_SPDY2;
   else
     version = SPDYLAY_PROTO_SPDY3;
@@ -98,6 +84,13 @@ SpdySM::init(TSVConn conn)
   sm_id = atomic_inc(g_sm_id);
   total_size = 0;
   start_time = TShrtime();
+
+  ink_assert(this->contp == NULL);
+  this->contp = TSContCreate(spdy_main_handler, TSMutexCreate());
+  TSContDataSet(this->contp, this);
+
+  this->vc->set_inactivity_timeout(HRTIME_SECONDS(SPDY_CFG.accept_no_activity_timeout));
+  this->current_handler = &spdy_start_handler;
 }
 
 void
@@ -122,9 +115,9 @@ SpdySM::clear()
   }
   req_map.clear();
 
-  if (net_vc) {
-    TSVConnClose(net_vc);
-    net_vc = NULL;
+  if (vc) {
+    TSVConnClose(reinterpret_cast<TSVConn>(vc));
+    vc = NULL;
   }
 
   if (contp) {
@@ -163,21 +156,19 @@ SpdySM::clear()
 }
 
 void
-spdy_sm_create(TSVConn cont)
+spdy_sm_create(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
 {
   SpdySM  *sm;
-  NetVConnection *netvc = (NetVConnection *)cont;
 
   sm = spdySMAllocator.alloc();
-  sm->init(cont);
-  atomic_inc(g_sm_cnt);
+  sm->init(netvc);
 
-  sm->contp = TSContCreate(spdy_main_handler, TSMutexCreate());
-  TSContDataSet(sm->contp, sm);
+  sm->req_buffer = iobuf ? reinterpret_cast<TSIOBuffer>(iobuf) : TSIOBufferCreate();
+  sm->req_reader = reader ? reinterpret_cast<TSIOBufferReader>(reader) : TSIOBufferReaderAlloc(sm->req_buffer);
 
-  netvc->set_inactivity_timeout(HRTIME_SECONDS(SPDY_CFG.accept_no_activity_timeout));
+  sm->resp_buffer = TSIOBufferCreate();
+  sm->resp_reader = TSIOBufferReaderAlloc(sm->resp_buffer);
 
-  sm->current_handler = &spdy_start_handler;
   TSContSchedule(sm->contp, 0, TS_THREAD_POOL_DEFAULT);       // schedule now
 }
 
@@ -201,14 +192,12 @@ spdy_start_handler(TSCont contp, TSEvent /*event*/, void * /*data*/)
 
   SpdySM  *sm = (SpdySM*)TSContDataGet(contp);
 
-  sm->req_buffer = TSIOBufferCreate();
-  sm->req_reader = TSIOBufferReaderAlloc(sm->req_buffer);
-
-  sm->resp_buffer = TSIOBufferCreate();
-  sm->resp_reader = TSIOBufferReaderAlloc(sm->resp_buffer);
+  if (TSIOBufferReaderAvail(sm->req_reader) > 0) {
+    spdy_process_read(TS_EVENT_VCONN_WRITE_READY, sm);
+  }
 
-  sm->read_vio = TSVConnRead(sm->net_vc, contp, sm->req_buffer, INT64_MAX);
-  sm->write_vio = TSVConnWrite(sm->net_vc, contp, sm->resp_reader, INT64_MAX);
+  sm->read_vio = (TSVIO)sm->vc->do_io_read(reinterpret_cast<Continuation *>(contp), INT64_MAX, reinterpret_cast<MIOBuffer *>(sm->req_buffer));
+  sm->write_vio = (TSVIO)sm->vc->do_io_write(reinterpret_cast<Continuation *>(contp), INT64_MAX, reinterpret_cast<IOBufferReader *>(sm->resp_reader));
 
   sm->current_handler = &spdy_default_handler;
 
@@ -229,10 +218,8 @@ spdy_default_handler(TSCont contp, TSEvent event, void *edata)
 {
   int ret = 0;
   bool from_fetch = false;
-  NetVConnection *netvc;
   SpdySM  *sm = (SpdySM*)TSContDataGet(contp);
   sm->event = event;
-  netvc = (NetVConnection *)sm->net_vc;
 
   if (edata == sm->read_vio) {
     Debug("spdy", "++++[READ EVENT]");
@@ -262,7 +249,7 @@ out:
     sm->clear();
     spdySMAllocator.free(sm);
   } else if (!from_fetch) {
-    netvc->set_inactivity_timeout(HRTIME_SECONDS(SPDY_CFG.no_activity_timeout_in));
+    sm->vc->set_inactivity_timeout(HRTIME_SECONDS(SPDY_CFG.no_activity_timeout_in));
   }
 
   return 0;
@@ -382,7 +369,7 @@ spdy_read_fetch_body_callback(spdylay_session * /*session*/, int32_t stream_id,
 
   already = TSFetchReadData(req->fetch_sm, buf, length);
 
-  Debug("spdy", "    stream_id:%d, call:%d, length:%ld, already:%ld",
+  Debug("spdy", "    stream_id:%d, call:%d, length:%ld, already:%" PRId64,
         stream_id, g_call_cnt, length, already);
   if (SPDY_CFG.spdy.verbose)
     MD5_Update(&req->recv_md5, buf, already);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/spdy/SpdySM.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySM.h b/proxy/spdy/SpdySM.h
index f80b4d9..4a6da83 100644
--- a/proxy/spdy/SpdySM.h
+++ b/proxy/spdy/SpdySM.h
@@ -103,23 +103,21 @@ class SpdySM
 
 public:
 
-  SpdySM();
-  SpdySM(TSVConn conn);
-  ~SpdySM()
-  {
+  SpdySM() {
+  }
+
+  ~SpdySM() {
     clear();
   }
 
-  void init(TSVConn conn);
+  void init(NetVConnection * netvc);
   void clear();
 
-public:
-
   int64_t sm_id;
   uint64_t total_size;
   TSHRTime start_time;
 
-  TSVConn net_vc;
+  NetVConnection * vc;
   TSCont  contp;
 
   TSIOBuffer req_buffer;
@@ -139,7 +137,7 @@ public:
   map<int32_t, SpdyRequest*> req_map;
 };
 
-void spdy_sm_create(TSVConn cont);
+void spdy_sm_create(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader);
 
 extern ClassAllocator<SpdySM> spdySMAllocator;
 extern ClassAllocator<SpdyRequest> spdyRequestAllocator;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/spdy/SpdySessionAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
index 1b41e5d..7014759 100644
--- a/proxy/spdy/SpdySessionAccept.cc
+++ b/proxy/spdy/SpdySessionAccept.cc
@@ -22,6 +22,8 @@
  */
 
 #include "SpdySessionAccept.h"
+#include "Error.h"
+
 #if TS_HAS_SPDY
 #include "SpdySM.h"
 #endif
@@ -36,12 +38,29 @@ SpdySessionAccept::SpdySessionAccept(Continuation *ep)
 }
 
 int
-SpdySessionAccept::mainEvent(int /* event */, void *netvc)
+SpdySessionAccept::mainEvent(int event, void * edata)
 {
+  if (event == NET_EVENT_ACCEPT) {
+    NetVConnection * netvc =static_cast<NetVConnection *>(edata);
+
 #if TS_HAS_SPDY
-  spdy_sm_create((TSCont)netvc);
+    spdy_sm_create(netvc, NULL, NULL);
 #else
-  (void)(netvc);
+    Error("accepted a SPDY session, but SPDY support is not available");
+    netvc->do_io_close();
+#endif
+
+    return EVENT_CONT;
+  }
+
+  MachineFatal("SPDY accept received fatal error: errno = %d", -((int)(intptr_t)edata));
+  return EVENT_CONT;
+}
+
+void
+SpdySessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
+{
+#if TS_HAS_SPDY
+  spdy_sm_create(netvc, iobuf, reader);
 #endif
-  return 0;
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/727811ef/proxy/spdy/SpdySessionAccept.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySessionAccept.h b/proxy/spdy/SpdySessionAccept.h
index a0fc822..dfe6484 100644
--- a/proxy/spdy/SpdySessionAccept.h
+++ b/proxy/spdy/SpdySessionAccept.h
@@ -35,6 +35,8 @@ public:
   SpdySessionAccept(Continuation *ep);
   ~SpdySessionAccept() {}
 
+  void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
+
 private:
   int mainEvent(int event, void *netvc);
   SpdySessionAccept(const SpdySessionAccept &); // disabled