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

git commit: TS-2838 Add log fields for plugins on HttpSM and use for SPDY.

Repository: trafficserver
Updated Branches:
  refs/heads/master b6383928a -> aadb75226


TS-2838 Add log fields for plugins on HttpSM and use for SPDY.


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

Branch: refs/heads/master
Commit: aadb752267ba194a3a6f584c0874cda36e6ab859
Parents: b638392
Author: Alan M. Carroll <am...@network-geographics.com>
Authored: Fri May 23 09:30:03 2014 -0700
Committer: Alan M. Carroll <am...@network-geographics.com>
Committed: Fri May 23 09:30:03 2014 -0700

----------------------------------------------------------------------
 CHANGES                           |  2 ++
 proxy/FetchSM.cc                  | 17 ++++++-------
 proxy/FetchSM.h                   |  4 ++-
 proxy/InkAPI.cc                   |  9 ++++++-
 proxy/Plugin.h                    | 29 ++++++++++++++++++++++
 proxy/PluginVC.cc                 | 12 ++++++++-
 proxy/PluginVC.h                  | 23 ++++++++++++++++-
 proxy/api/ts/ts.h                 | 14 +++++++----
 proxy/http/HttpClientSession.cc   |  8 ++++++
 proxy/http/HttpProxyServerMain.cc |  9 +++----
 proxy/http/HttpSM.cc              |  1 +
 proxy/http/HttpSM.h               |  5 ++++
 proxy/logging/Log.cc              | 30 +++++++++--------------
 proxy/logging/LogAccess.cc        | 45 ++++++----------------------------
 proxy/logging/LogAccess.h         |  3 ++-
 proxy/logging/LogAccessHttp.cc    | 23 ++++++++++++-----
 proxy/logging/LogAccessHttp.h     |  3 ++-
 proxy/spdy/SpdyClientSession.cc   | 36 ++++++++++++++++++++++++---
 proxy/spdy/SpdyClientSession.h    | 12 ++++++---
 proxy/spdy/SpdySessionAccept.cc   | 15 +++---------
 proxy/spdy/SpdySessionAccept.h    | 11 +++------
 21 files changed, 198 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 4f8d8af..fa34b15 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.0.0
 
+  *) [TS-2838] Add logging fields for plugins to TS connect API. Use for SPDY.
+
   *) [TS-2833] Remove REC_BUILD_STAND_ALONE, REC_BUILD_MGMT and
    REC_BUILD_STUB. Dead code.
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.cc
----------------------------------------------------------------------
diff --git a/proxy/FetchSM.cc b/proxy/FetchSM.cc
index c119aca..a0a4033 100644
--- a/proxy/FetchSM.cc
+++ b/proxy/FetchSM.cc
@@ -61,23 +61,22 @@ FetchSM::cleanUp()
   client_response_hdr.destroy();
   ats_free(client_response);
   cont_mutex.clear();
-
-  PluginVC *vc = (PluginVC *) http_vc;
-
-  vc->do_io_close();
+  http_vc->do_io_close();
   FetchSMAllocator.free(this);
 }
 
 void
 FetchSM::httpConnect()
 {
-  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
-  http_vc = TSHttpConnect(&_addr.sa);
+  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(contp);
+  char const* tag = pi ? pi->getPluginTag() : "fetchSM";
+  int64_t id = pi ? pi->getPluginId() : 0;
 
-  PluginVC *vc = (PluginVC *) http_vc;
+  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
+  http_vc = reinterpret_cast<PluginVC*>(TSHttpConnectWithPluginId(&_addr.sa, tag, id));
 
-  read_vio = vc->do_io_read(this, INT64_MAX, resp_buffer);
-  write_vio = vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
+  read_vio = http_vc->do_io_read(this, INT64_MAX, resp_buffer);
+  write_vio = http_vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
 }
 
 char* FetchSM::resp_get(int *length) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.h
----------------------------------------------------------------------
diff --git a/proxy/FetchSM.h b/proxy/FetchSM.h
index 0de5d96..c1b94bf 100644
--- a/proxy/FetchSM.h
+++ b/proxy/FetchSM.h
@@ -35,6 +35,8 @@
 #include "HttpSM.h"
 #include "HttpTunnel.h"
 
+class PluginVC;
+
 class FetchSM: public Continuation
 {
 public:
@@ -139,7 +141,7 @@ private:
   int dechunk_body();
 
   int recursion;
-  TSVConn http_vc;
+  PluginVC* http_vc;
   VIO *read_vio;
   VIO *write_vio;
   MIOBuffer *req_buffer;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/InkAPI.cc
----------------------------------------------------------------------
diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
index 0182ebe..dceea6e 100644
--- a/proxy/InkAPI.cc
+++ b/proxy/InkAPI.cc
@@ -6047,7 +6047,7 @@ extern HttpSessionAccept *plugin_http_accept;
 extern HttpSessionAccept *plugin_http_transparent_accept;
 
 TSVConn
-TSHttpConnect(sockaddr const* addr)
+TSHttpConnectWithPluginId(sockaddr const* addr, char const* tag, int64_t id)
 {
   sdk_assert(addr);
 
@@ -6058,6 +6058,8 @@ TSHttpConnect(sockaddr const* addr)
     PluginVCCore *new_pvc = PluginVCCore::alloc();
 
     new_pvc->set_active_addr(addr);
+    new_pvc->set_plugin_id(id);
+    new_pvc->set_plugin_tag(tag);
     new_pvc->set_accept_cont(plugin_http_accept);
 
     PluginVC *return_vc = new_pvc->connect();
@@ -6076,6 +6078,11 @@ TSHttpConnect(sockaddr const* addr)
   return NULL;
 }
 
+TSVConn
+TSHttpConnect(sockaddr const* addr)
+{
+  return TSHttpConnectWithPluginId(addr, "plugin", 0);
+}
 
 TSVConn
 TSHttpConnectTransparent(sockaddr const* client_addr, sockaddr const* server_addr)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/Plugin.h
----------------------------------------------------------------------
diff --git a/proxy/Plugin.h b/proxy/Plugin.h
index 4c1ad09..8f70f1a 100644
--- a/proxy/Plugin.h
+++ b/proxy/Plugin.h
@@ -58,4 +58,33 @@ extern PluginRegInfo *plugin_reg_current;
 
 void plugin_init(void);
 
+/** Abstract interface class for plugin based continuations.
+
+    The primary intended use of this is for logging so that continuations
+    that generate logging messages can generate plugin local data in a
+    generic way.
+
+    The core will at appropriate times dynamically cast the continuation
+    to this class and if successful access the plugin data via these
+    methods.
+
+    Plugins should mix this in to continuations for which it is useful.
+    The default implementations return empty / invalid responses and should
+    be overridden by the plugin.
+ */
+class PluginIdentity
+{
+ public:
+  /** Get the plugin tag.
+      The returned string must have a lifetime at least as long as the plugin.
+      @return A string identifying the plugin or @c NULL.
+  */
+  virtual char const* getPluginTag() const { return NULL; }
+  /** Get the plugin instance ID.
+      A plugin can create multiple subsidiary instances. This is used as the
+      identifier for those to distinguish the instances.
+  */
+  virtual int64_t getPluginId() const { return 0; }
+};
+
 #endif /* __PLUGIN_H__ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.cc
----------------------------------------------------------------------
diff --git a/proxy/PluginVC.cc b/proxy/PluginVC.cc
index 2e4cfa1..16eb5a2 100644
--- a/proxy/PluginVC.cc
+++ b/proxy/PluginVC.cc
@@ -1051,7 +1051,6 @@ void
 PluginVCCore::set_accept_cont(Continuation * c)
 {
   connect_to = c;
-
   // FIX ME - must return action
 }
 
@@ -1202,6 +1201,17 @@ PluginVCCore::set_transparent(bool passive_side, bool active_side)
   active_vc.set_is_transparent(active_side);
 }
 
+void
+PluginVCCore::set_plugin_id(int64_t id)
+{
+  passive_vc.plugin_id = active_vc.plugin_id = id;
+}
+
+void
+PluginVCCore::set_plugin_tag(char const* tag)
+{
+  passive_vc.plugin_tag = active_vc.plugin_tag = tag;
+}
 
 /*************************************************************
  *

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.h
----------------------------------------------------------------------
diff --git a/proxy/PluginVC.h b/proxy/PluginVC.h
index d770de5..6ee3a93 100644
--- a/proxy/PluginVC.h
+++ b/proxy/PluginVC.h
@@ -36,6 +36,7 @@
 #ifndef _PLUGIN_VC_H_
 #define _PLUGIN_VC_H_
 
+#include "Plugin.h"
 #include "P_Net.h"
 #include "ink_atomic.h"
 
@@ -75,7 +76,7 @@ enum
   PLUGIN_VC_MAGIC_DEAD = 0xaabbdead
 };
 
-class PluginVC:public NetVConnection
+class PluginVC:public NetVConnection, public PluginIdentity
 {
   friend class PluginVCCore;
 public:
@@ -114,6 +115,18 @@ public:
 
   virtual PluginVC* get_other_side() { return other_side; }
 
+  //@{ @name Plugin identity.
+  /// Override for @c PluginIdentity.
+  virtual char const* getPluginTag() const { return plugin_tag; }
+  /// Override for @c PluginIdentity.
+  virtual int64_t getPluginId() const { return plugin_id; }
+
+  /// Setter for plugin tag.
+  virtual void setPluginTag(char const* tag) { plugin_tag = tag; }
+  /// Setter for plugin id.
+  virtual void setPluginId(int64_t id) { plugin_id = id; }
+  //@}
+
   int main_handler(int event, void *data);
 
 private:
@@ -152,6 +165,9 @@ private:
   ink_hrtime inactive_timeout;
   ink_hrtime inactive_timeout_at;
   Event *inactive_event;
+
+  char const* plugin_tag;
+  int64_t plugin_id;
 };
 
 class PluginVCCore:public Continuation
@@ -198,6 +214,11 @@ public:
 
   void set_transparent(bool passive_side, bool active_side);
 
+  /// Set the plugin ID for the internal VCs.
+  void set_plugin_id(int64_t id);
+  /// Set the plugin tag for the internal VCs.
+  void set_plugin_tag(char const* tag);
+
   // The active vc is handed to the initiator of
   //   connection.  The passive vc is handled to
   //   receiver of the connection

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/api/ts/ts.h
----------------------------------------------------------------------
diff --git a/proxy/api/ts/ts.h b/proxy/api/ts/ts.h
index 27d0546..917d6b2 100644
--- a/proxy/api/ts/ts.h
+++ b/proxy/api/ts/ts.h
@@ -1588,12 +1588,16 @@ extern "C"
       than TSNetConnect() to localhost since it avoids the overhead of
       passing the data through the operating system.
 
-      @param log_ip ip address (in network byte order) that connection
-        will be logged as coming from.
-      @param log_port port (in network byte order) that connection will
-        be logged as coming from.
-      @param vc will be set to point to the new TSVConn on success.
+      This returns a VConn that connected to the transaction.
+
+      @param addr Target address of the origin server.
+      @param tag A logging tag that can be accessed via the pitag field. May be @c NULL.
+      @param id A logging id that can be access via the piid field.
+   */
+  tsapi TSVConn TSHttpConnectWithPluginId(struct sockaddr const* addr, char const* tag, int64_t id);
 
+  /** Backwards compatible version.
+      This provides a @a tag of "plugin" and an @a id of 0.
    */
   tsapi TSVConn TSHttpConnect(struct sockaddr const* addr);
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
index d2246c8..0620b3b 100644
--- a/proxy/http/HttpClientSession.cc
+++ b/proxy/http/HttpClientSession.cc
@@ -36,6 +36,7 @@
 #include "HttpSM.h"
 #include "HttpDebugNames.h"
 #include "HttpServerSession.h"
+#include "Plugin.h"
 
 #define DebugSsn(tag, ...) DebugSpecific(debug_on, tag, __VA_ARGS__)
 #define STATE_ENTER(state_name, event, vio) { \
@@ -137,6 +138,7 @@ void
 HttpClientSession::new_transaction()
 {
   ink_assert(current_reader == NULL);
+  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(client_vc);
 
   read_state = HCS_ACTIVE_READER;
   current_reader = HttpSM::allocate();
@@ -145,6 +147,12 @@ HttpClientSession::new_transaction()
   DebugSsn("http_cs", "[%" PRId64 "] Starting transaction %d using sm [%" PRId64 "]", con_id, transact_count, current_reader->sm_id);
 
   current_reader->attach_client_session(this, sm_reader);
+  if (pi) {
+    // it's a plugin VC of some sort with identify information.
+    // copy it to the SM.
+    current_reader->plugin_tag = pi->getPluginTag();
+    current_reader->plugin_id = pi->getPluginId();
+  }
 }
 
 inline void

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpProxyServerMain.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
index 2bf7d64..9eb9291 100644
--- a/proxy/http/HttpProxyServerMain.cc
+++ b/proxy/http/HttpProxyServerMain.cc
@@ -180,8 +180,7 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
 
 #if TS_HAS_SPDY
   if (port.m_session_protocol_preference.intersects(SPDY_PROTOCOL_SET)) {
-    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY,
-                            new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
+    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
   }
 #endif
 
@@ -207,11 +206,11 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
     // SPDY
 #if TS_HAS_SPDY
     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3)) {
-      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3));
+      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(spdy::SESSION_VERSION_3));
     }
 
     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3_1)) {
-      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
+      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
     }
 #endif
 
@@ -316,7 +315,7 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
   opt.localhost_only = true;
   ha_opt.backdoor = true;
   opt.backdoor = true;
-  
+
   // The backdoor only binds the loopback interface
   netProcessor.main_accept(new HttpSessionAccept(ha_opt), NO_FD, opt);
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 2eabc1d..02afe4a 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -325,6 +325,7 @@ HttpSM::HttpSM()
     client_response_hdr_bytes(0), client_response_body_bytes(0),
     cache_response_hdr_bytes(0), cache_response_body_bytes(0),
     pushed_response_hdr_bytes(0), pushed_response_body_bytes(0),
+    plugin_tag(0), plugin_id(0),
     hooks_set(0), cur_hook_id(TS_HTTP_LAST_HOOK), cur_hook(NULL),
     cur_hooks(0), callout_state(HTTP_API_NO_CALLOUT), terminate_sm(false), kill_this_async_done(false)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
index 2b5a6fd..13121ec 100644
--- a/proxy/http/HttpSM.h
+++ b/proxy/http/HttpSM.h
@@ -488,6 +488,11 @@ public:
   int pushed_response_hdr_bytes;
   int64_t pushed_response_body_bytes;
   TransactionMilestones milestones;
+  // The next two enable plugins to tag the state machine for
+  // the purposes of logging so the instances can be correlated
+  // with the source plugin.
+  char const* plugin_tag;
+  int64_t plugin_id;
 
   // hooks_set records whether there are any hooks relevant
   //  to this transaction.  Used to avoid costly calls

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/Log.cc
----------------------------------------------------------------------
diff --git a/proxy/logging/Log.cc b/proxy/logging/Log.cc
index 16138b4..9ee618e 100644
--- a/proxy/logging/Log.cc
+++ b/proxy/logging/Log.cc
@@ -365,27 +365,19 @@ Log::init_fields()
   global_field_list.add (field, false);
   ink_hash_table_insert (field_symbol_hash, "caun", field);
 
-# if 0
-  Ptr<LogFieldAliasTable> proto_type_map = make_ptr(new LogFieldAliasTable);
-  proto_type_map->init(7,
-                       // Transport protocols
-                       TS_PROTO_UDP, "UDP",
-                       TS_PROTO_TCP, "TCP",
-                       TS_PROTO_TLS, "TLS",
-                       // Application protocols
-                       TS_PROTO_HTTP, "HTTP",
-                       TS_PROTO_SPDY, "SPDY",
-                       TS_PROTO_RTMP, "RTMP",
-                       TS_PROTO_WS,   "WS");
-
-  field = new LogField("client_protocol_stack", "cps",
+  field = new LogField("plugin_identity_id", "piid",
                        LogField::sINT,
-                       &LogAccess::marshal_client_protocol_stack,
-                       &LogAccess::unmarshal_client_protocol_stack,
-                       (Ptr<LogFieldAliasMap>) proto_type_map);
+                       &LogAccess::marshal_plugin_identity_id,
+                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_int_to_str));
   global_field_list.add(field, false);
-  ink_hash_table_insert(field_symbol_hash, "cps", field);
-# endif
+  ink_hash_table_insert(field_symbol_hash, "piid", field);
+
+  field = new LogField("plugin_identity_tag", "pitag",
+                       LogField::STRING,
+                       &LogAccess::marshal_plugin_identity_tag,
+                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_str));
+  global_field_list.add(field, false);
+  ink_hash_table_insert(field_symbol_hash, "pitag", field);
 
   field = new LogField("client_req_timestamp_sec", "cqts",
                        LogField::sINT,

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.cc
----------------------------------------------------------------------
diff --git a/proxy/logging/LogAccess.cc b/proxy/logging/LogAccess.cc
index 968762e..5009113 100644
--- a/proxy/logging/LogAccess.cc
+++ b/proxy/logging/LogAccess.cc
@@ -75,12 +75,18 @@ LogAccess::init()
   -------------------------------------------------------------------------*/
 
 int
-LogAccess::marshal_client_protocol_stack(char *buf)
+LogAccess::marshal_plugin_identity_id(char *buf)
 {
   DEFAULT_INT_FIELD;
 }
 
 int
+LogAccess::marshal_plugin_identity_tag(char *buf)
+{
+  DEFAULT_STR_FIELD;
+}
+
+int
 LogAccess::marshal_client_host_ip(char *buf)
 {
   DEFAULT_IP_FIELD;
@@ -1311,43 +1317,6 @@ LogAccess::unmarshal_cache_write_code(char **buf, char *dest, int len, Ptr<LogFi
   return (LogAccess::unmarshal_with_map(unmarshal_int(buf), dest, len, map, "UNKNOWN_CACHE_WRITE_CODE"));
 }
 
-# if 0
-int
-LogAccess::unmarshal_client_protocol_stack(char **buf, char *dest, int len, Ptr<LogFieldAliasMap> map)
-{
-  ink_assert(buf != NULL);
-  ink_assert(*buf != NULL);
-  ink_assert(dest != NULL);
-
-  char *p;
-  size_t nr_chars = 0;
-  int i, ret, nr_bits, left_len;
-  TSClientProtoStack proto_stack = (TSClientProtoStack)unmarshal_int(buf);
-
-  p = dest;
-  left_len = len;
-  nr_bits = 8 * sizeof(TSClientProtoStack);
-
-  for (i = 0; i < nr_bits && left_len; i++) {
-    if ((proto_stack >> i) & 0x1) {
-      if (p != dest) {
-        *p++ = '+';
-        left_len--;
-      }
-      ret = map->asString(i, p, left_len, &nr_chars);
-      if (ret == LogFieldAliasMap::ALL_OK) {
-        p += nr_chars;
-        left_len -= nr_chars;
-      } else if (ret == LogFieldAliasMap::BUFFER_TOO_SMALL) {
-        break;
-      }
-    }
-  }
-
-  return (len - left_len);
-}
-# endif
-
 int
 LogAccess::unmarshal_record(char **buf, char *dest, int len)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.h
----------------------------------------------------------------------
diff --git a/proxy/logging/LogAccess.h b/proxy/logging/LogAccess.h
index 940145d..8fb35b5 100644
--- a/proxy/logging/LogAccess.h
+++ b/proxy/logging/LogAccess.h
@@ -172,7 +172,6 @@ public:
   inkcoreapi virtual int marshal_client_auth_user_name(char *); // STR
   int marshal_client_req_timestamp_sec(char *); // INT
 
-  inkcoreapi virtual int marshal_client_protocol_stack(char *); // INT
   inkcoreapi virtual int marshal_client_req_text(char *);       // STR
   inkcoreapi virtual int marshal_client_req_http_method(char *);        // STR
   inkcoreapi virtual int marshal_client_req_url(char *);        // STR
@@ -251,6 +250,8 @@ public:
   inkcoreapi virtual int marshal_transfer_time_ms(char *);      // INT
   inkcoreapi virtual int marshal_transfer_time_s(char *);       // INT
   inkcoreapi virtual int marshal_file_size(char *);     // INT
+  inkcoreapi virtual int marshal_plugin_identity_id(char *); // INT
+  inkcoreapi virtual int marshal_plugin_identity_tag(char *); // STR
   int marshal_entry_type(char *);       // INT
 
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.cc
----------------------------------------------------------------------
diff --git a/proxy/logging/LogAccessHttp.cc b/proxy/logging/LogAccessHttp.cc
index a8dad70..b44e684 100644
--- a/proxy/logging/LogAccessHttp.cc
+++ b/proxy/logging/LogAccessHttp.cc
@@ -206,16 +206,27 @@ LogAccessHttp::set_client_req_url_path(char *buf, int len)
 
 /*-------------------------------------------------------------------------
   -------------------------------------------------------------------------*/
-# if 0
 int
-LogAccessHttp::marshal_client_protocol_stack(char *buf)
+LogAccessHttp::marshal_plugin_identity_id(char *buf)
 {
-  if (buf) {
-    marshal_int(buf, m_http_sm->proto_stack);
-  }
+  if (buf) marshal_int(buf, m_http_sm->plugin_id);
   return INK_MIN_ALIGN;
 }
-# endif
+
+int
+LogAccessHttp::marshal_plugin_identity_tag(char *buf)
+{
+  int len = INK_MIN_ALIGN;
+  char const* tag = m_http_sm->plugin_tag;
+
+  if (!tag) tag = "*";
+  else len = LogAccess::strlen(tag);
+
+  if (buf) marshal_str(buf, tag, len);
+
+  return len;
+}
+
 int
 LogAccessHttp::marshal_client_host_ip(char *buf)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.h
----------------------------------------------------------------------
diff --git a/proxy/logging/LogAccessHttp.h b/proxy/logging/LogAccessHttp.h
index 53a0ff5..51ee9e3 100644
--- a/proxy/logging/LogAccessHttp.h
+++ b/proxy/logging/LogAccessHttp.h
@@ -58,7 +58,6 @@ public:
   virtual int marshal_client_host_ip(char *);   // STR
   virtual int marshal_client_host_port(char *); // INT
   virtual int marshal_client_auth_user_name(char *);    // STR
-  //  virtual int marshal_client_protocol_stack(char *);    // INT
   virtual int marshal_client_req_text(char *);  // STR
   virtual int marshal_client_req_http_method(char *);   // INT
   virtual int marshal_client_req_url(char *);   // STR
@@ -128,6 +127,8 @@ public:
   virtual int marshal_transfer_time_ms(char *); // INT
   virtual int marshal_transfer_time_s(char *);  // INT
   virtual int marshal_file_size(char *); // INT
+  virtual int marshal_plugin_identity_id(char *);    // INT
+  virtual int marshal_plugin_identity_tag(char *);    // STR
 
   //
   // named fields from within a http header

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyClientSession.cc b/proxy/spdy/SpdyClientSession.cc
index 2c77ff6..fcb79fb 100644
--- a/proxy/spdy/SpdyClientSession.cc
+++ b/proxy/spdy/SpdyClientSession.cc
@@ -27,6 +27,22 @@
 static ClassAllocator<SpdyClientSession> spdyClientSessionAllocator("spdyClientSessionAllocator");
 ClassAllocator<SpdyRequest> spdyRequestAllocator("spdyRequestAllocator");
 
+#if TS_HAS_SPDY
+#include "SpdyClientSession.h"
+
+static const spdylay_proto_version versmap[] = {
+  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
+  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
+  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
+};
+
+static char const* const  npnmap[] = {
+  TS_NPN_PROTOCOL_SPDY_2,
+  TS_NPN_PROTOCOL_SPDY_3,
+  TS_NPN_PROTOCOL_SPDY_3_1
+};
+
+#endif
 static int spdy_process_read(TSEvent event, SpdyClientSession *sm);
 static int spdy_process_write(TSEvent event, SpdyClientSession *sm);
 static int spdy_process_fetch(TSEvent event, SpdyClientSession *sm, void *edata);
@@ -69,15 +85,16 @@ SpdyRequest::clear()
 }
 
 void
-SpdyClientSession::init(NetVConnection * netvc, spdylay_proto_version vers)
+SpdyClientSession::init(NetVConnection * netvc, spdy::SessionVersion vers)
 {
   int r;
 
   this->mutex = new_ProxyMutex();
   this->vc = netvc;
   this->req_map.clear();
+  this->version = vers;
 
-  r = spdylay_session_server_new(&session, vers, &SPDY_CFG.spdy.callbacks, this);
+  r = spdylay_session_server_new(&session, versmap[vers], &SPDY_CFG.spdy.callbacks, this);
 
   // A bit ugly but we need a thread and I don't want to wait until the
   // session start event in case of a time out generating a decrement
@@ -157,7 +174,7 @@ SpdyClientSession::clear()
 }
 
 void
-spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader)
+spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader)
 {
   SpdyClientSession  *sm;
 
@@ -242,6 +259,19 @@ out:
   return EVENT_CONT;
 }
 
+int64_t
+SpdyClientSession::getPluginId() const
+{
+  return sm_id;
+}
+
+char const*
+SpdyClientSession::getPluginTag() const
+{
+  return npnmap[this->version];
+}
+
+
 static int
 spdy_process_read(TSEvent /* event ATS_UNUSED */, SpdyClientSession *sm)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdyClientSession.h b/proxy/spdy/SpdyClientSession.h
index 2f37d35..738ff55 100644
--- a/proxy/spdy/SpdyClientSession.h
+++ b/proxy/spdy/SpdyClientSession.h
@@ -24,9 +24,11 @@
 #ifndef __P_SPDY_SM_H__
 #define __P_SPDY_SM_H__
 
+#include "SpdyDefs.h"
 #include "SpdyCommon.h"
 #include "SpdyCallbacks.h"
 #include <openssl/md5.h>
+#include "Plugin.h"
 
 class SpdyClientSession;
 typedef int (*SpdyClientSessionHandler) (TSCont contp, TSEvent event, void *data);
@@ -89,7 +91,7 @@ public:
   MD5_CTX recv_md5;
 };
 
-class SpdyClientSession : public Continuation
+class SpdyClientSession : public Continuation, public PluginIdentity
 {
 
 public:
@@ -101,10 +103,11 @@ public:
     clear();
   }
 
-  void init(NetVConnection * netvc, spdylay_proto_version vers);
+  void init(NetVConnection * netvc, spdy::SessionVersion vers);
   void clear();
 
   int64_t sm_id;
+  spdy::SessionVersion version;
   uint64_t total_size;
   TSHRTime start_time;
 
@@ -124,12 +127,15 @@ public:
 
   map<int32_t, SpdyRequest*> req_map;
 
+  virtual char const* getPluginTag() const;
+  virtual int64_t getPluginId() const;
+
 private:
   int state_session_start(int event, void * edata);
   int state_session_readwrite(int event, void * edata);
 };
 
-void spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader);
+void spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader);
 
 extern ClassAllocator<SpdyRequest> spdyRequestAllocator;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
index e679adc..d7dfa11 100644
--- a/proxy/spdy/SpdySessionAccept.cc
+++ b/proxy/spdy/SpdySessionAccept.cc
@@ -26,20 +26,13 @@
 
 #if TS_HAS_SPDY
 #include "SpdyClientSession.h"
-
-static const spdylay_proto_version versmap[] = {
-  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
-  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
-  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
-};
-
 #endif
 
-SpdySessionAccept::SpdySessionAccept(unsigned vers)
+SpdySessionAccept::SpdySessionAccept(spdy::SessionVersion vers)
     : SessionAccept(new_ProxyMutex()), version(vers)
 {
 #if TS_HAS_SPDY
-  ink_release_assert(vers < countof(versmap));
+  ink_release_assert(spdy::SESSION_VERSION_2 <= vers && vers <= spdy::SESSION_VERSION_3_1);
 #endif
   SET_HANDLER(&SpdySessionAccept::mainEvent);
 }
@@ -51,7 +44,7 @@ SpdySessionAccept::mainEvent(int event, void * edata)
     NetVConnection * netvc =static_cast<NetVConnection *>(edata);
 
 #if TS_HAS_SPDY
-    spdy_sm_create(netvc, versmap[this->version], NULL, NULL);
+    spdy_sm_create(netvc, this->version, NULL, NULL);
 #else
     Error("accepted a SPDY session, but SPDY support is not available");
     netvc->do_io_close();
@@ -68,7 +61,7 @@ void
 SpdySessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
 {
 #if TS_HAS_SPDY
-  spdy_sm_create(netvc, versmap[this->version], iobuf, reader);
+  spdy_sm_create(netvc, this->version, iobuf, reader);
 #else
   (void)netvc;
   (void)iobuf;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.h
----------------------------------------------------------------------
diff --git a/proxy/spdy/SpdySessionAccept.h b/proxy/spdy/SpdySessionAccept.h
index 9d740e2..af3d81d 100644
--- a/proxy/spdy/SpdySessionAccept.h
+++ b/proxy/spdy/SpdySessionAccept.h
@@ -28,18 +28,13 @@
 #include "P_EventSystem.h"
 #include "P_UnixNet.h"
 #include "I_IOBuffer.h"
+#include "SpdyDefs.h"
 
 class SpdySessionAccept: public SessionAccept
 {
 public:
 
-  enum {
-    SPDY_VERSION_2 = 0,
-    SPDY_VERSION_3,
-    SPDY_VERSION_3_1,
-  };
-
-  explicit SpdySessionAccept(unsigned vers);
+  explicit SpdySessionAccept(spdy::SessionVersion vers);
   ~SpdySessionAccept() {}
 
   void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
@@ -49,7 +44,7 @@ private:
   SpdySessionAccept(const SpdySessionAccept &); // disabled
   SpdySessionAccept& operator =(const SpdySessionAccept&); // disabled
 
-  unsigned version;
+  spdy::SessionVersion version;
 };
 
 #endif /* SpdySessionAccept_H_ */


Re: git commit: TS-2838 Add log fields for plugins on HttpSM and use for SPDY.

Posted by James Peach <jp...@apache.org>.
This should have gone through API review. Can you please post a review email?

On May 23, 2014, at 9:30 AM, amc@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master b6383928a -> aadb75226
> 
> 
> TS-2838 Add log fields for plugins on HttpSM and use for SPDY.
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/aadb7522
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/aadb7522
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/aadb7522
> 
> Branch: refs/heads/master
> Commit: aadb752267ba194a3a6f584c0874cda36e6ab859
> Parents: b638392
> Author: Alan M. Carroll <am...@network-geographics.com>
> Authored: Fri May 23 09:30:03 2014 -0700
> Committer: Alan M. Carroll <am...@network-geographics.com>
> Committed: Fri May 23 09:30:03 2014 -0700
> 
> ----------------------------------------------------------------------
> CHANGES                           |  2 ++
> proxy/FetchSM.cc                  | 17 ++++++-------
> proxy/FetchSM.h                   |  4 ++-
> proxy/InkAPI.cc                   |  9 ++++++-
> proxy/Plugin.h                    | 29 ++++++++++++++++++++++
> proxy/PluginVC.cc                 | 12 ++++++++-
> proxy/PluginVC.h                  | 23 ++++++++++++++++-
> proxy/api/ts/ts.h                 | 14 +++++++----
> proxy/http/HttpClientSession.cc   |  8 ++++++
> proxy/http/HttpProxyServerMain.cc |  9 +++----
> proxy/http/HttpSM.cc              |  1 +
> proxy/http/HttpSM.h               |  5 ++++
> proxy/logging/Log.cc              | 30 +++++++++--------------
> proxy/logging/LogAccess.cc        | 45 ++++++----------------------------
> proxy/logging/LogAccess.h         |  3 ++-
> proxy/logging/LogAccessHttp.cc    | 23 ++++++++++++-----
> proxy/logging/LogAccessHttp.h     |  3 ++-
> proxy/spdy/SpdyClientSession.cc   | 36 ++++++++++++++++++++++++---
> proxy/spdy/SpdyClientSession.h    | 12 ++++++---
> proxy/spdy/SpdySessionAccept.cc   | 15 +++---------
> proxy/spdy/SpdySessionAccept.h    | 11 +++------
> 21 files changed, 198 insertions(+), 113 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/CHANGES
> ----------------------------------------------------------------------
> diff --git a/CHANGES b/CHANGES
> index 4f8d8af..fa34b15 100644
> --- a/CHANGES
> +++ b/CHANGES
> @@ -1,6 +1,8 @@
>                                                          -*- coding: utf-8 -*-
> Changes with Apache Traffic Server 5.0.0
> 
> +  *) [TS-2838] Add logging fields for plugins to TS connect API. Use for SPDY.
> +
>   *) [TS-2833] Remove REC_BUILD_STAND_ALONE, REC_BUILD_MGMT and
>    REC_BUILD_STUB. Dead code.
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.cc b/proxy/FetchSM.cc
> index c119aca..a0a4033 100644
> --- a/proxy/FetchSM.cc
> +++ b/proxy/FetchSM.cc
> @@ -61,23 +61,22 @@ FetchSM::cleanUp()
>   client_response_hdr.destroy();
>   ats_free(client_response);
>   cont_mutex.clear();
> -
> -  PluginVC *vc = (PluginVC *) http_vc;
> -
> -  vc->do_io_close();
> +  http_vc->do_io_close();
>   FetchSMAllocator.free(this);
> }
> 
> void
> FetchSM::httpConnect()
> {
> -  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> -  http_vc = TSHttpConnect(&_addr.sa);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(contp);
> +  char const* tag = pi ? pi->getPluginTag() : "fetchSM";
> +  int64_t id = pi ? pi->getPluginId() : 0;
> 
> -  PluginVC *vc = (PluginVC *) http_vc;
> +  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> +  http_vc = reinterpret_cast<PluginVC*>(TSHttpConnectWithPluginId(&_addr.sa, tag, id));
> 
> -  read_vio = vc->do_io_read(this, INT64_MAX, resp_buffer);
> -  write_vio = vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> +  read_vio = http_vc->do_io_read(this, INT64_MAX, resp_buffer);
> +  write_vio = http_vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> }
> 
> char* FetchSM::resp_get(int *length) {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.h b/proxy/FetchSM.h
> index 0de5d96..c1b94bf 100644
> --- a/proxy/FetchSM.h
> +++ b/proxy/FetchSM.h
> @@ -35,6 +35,8 @@
> #include "HttpSM.h"
> #include "HttpTunnel.h"
> 
> +class PluginVC;
> +
> class FetchSM: public Continuation
> {
> public:
> @@ -139,7 +141,7 @@ private:
>   int dechunk_body();
> 
>   int recursion;
> -  TSVConn http_vc;
> +  PluginVC* http_vc;
>   VIO *read_vio;
>   VIO *write_vio;
>   MIOBuffer *req_buffer;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/InkAPI.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
> index 0182ebe..dceea6e 100644
> --- a/proxy/InkAPI.cc
> +++ b/proxy/InkAPI.cc
> @@ -6047,7 +6047,7 @@ extern HttpSessionAccept *plugin_http_accept;
> extern HttpSessionAccept *plugin_http_transparent_accept;
> 
> TSVConn
> -TSHttpConnect(sockaddr const* addr)
> +TSHttpConnectWithPluginId(sockaddr const* addr, char const* tag, int64_t id)
> {
>   sdk_assert(addr);
> 
> @@ -6058,6 +6058,8 @@ TSHttpConnect(sockaddr const* addr)
>     PluginVCCore *new_pvc = PluginVCCore::alloc();
> 
>     new_pvc->set_active_addr(addr);
> +    new_pvc->set_plugin_id(id);
> +    new_pvc->set_plugin_tag(tag);
>     new_pvc->set_accept_cont(plugin_http_accept);
> 
>     PluginVC *return_vc = new_pvc->connect();
> @@ -6076,6 +6078,11 @@ TSHttpConnect(sockaddr const* addr)
>   return NULL;
> }
> 
> +TSVConn
> +TSHttpConnect(sockaddr const* addr)
> +{
> +  return TSHttpConnectWithPluginId(addr, "plugin", 0);
> +}
> 
> TSVConn
> TSHttpConnectTransparent(sockaddr const* client_addr, sockaddr const* server_addr)
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/Plugin.h
> ----------------------------------------------------------------------
> diff --git a/proxy/Plugin.h b/proxy/Plugin.h
> index 4c1ad09..8f70f1a 100644
> --- a/proxy/Plugin.h
> +++ b/proxy/Plugin.h
> @@ -58,4 +58,33 @@ extern PluginRegInfo *plugin_reg_current;
> 
> void plugin_init(void);
> 
> +/** Abstract interface class for plugin based continuations.
> +
> +    The primary intended use of this is for logging so that continuations
> +    that generate logging messages can generate plugin local data in a
> +    generic way.
> +
> +    The core will at appropriate times dynamically cast the continuation
> +    to this class and if successful access the plugin data via these
> +    methods.
> +
> +    Plugins should mix this in to continuations for which it is useful.
> +    The default implementations return empty / invalid responses and should
> +    be overridden by the plugin.
> + */
> +class PluginIdentity
> +{
> + public:
> +  /** Get the plugin tag.
> +      The returned string must have a lifetime at least as long as the plugin.
> +      @return A string identifying the plugin or @c NULL.
> +  */
> +  virtual char const* getPluginTag() const { return NULL; }
> +  /** Get the plugin instance ID.
> +      A plugin can create multiple subsidiary instances. This is used as the
> +      identifier for those to distinguish the instances.
> +  */
> +  virtual int64_t getPluginId() const { return 0; }
> +};
> +
> #endif /* __PLUGIN_H__ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.cc b/proxy/PluginVC.cc
> index 2e4cfa1..16eb5a2 100644
> --- a/proxy/PluginVC.cc
> +++ b/proxy/PluginVC.cc
> @@ -1051,7 +1051,6 @@ void
> PluginVCCore::set_accept_cont(Continuation * c)
> {
>   connect_to = c;
> -
>   // FIX ME - must return action
> }
> 
> @@ -1202,6 +1201,17 @@ PluginVCCore::set_transparent(bool passive_side, bool active_side)
>   active_vc.set_is_transparent(active_side);
> }
> 
> +void
> +PluginVCCore::set_plugin_id(int64_t id)
> +{
> +  passive_vc.plugin_id = active_vc.plugin_id = id;
> +}
> +
> +void
> +PluginVCCore::set_plugin_tag(char const* tag)
> +{
> +  passive_vc.plugin_tag = active_vc.plugin_tag = tag;
> +}
> 
> /*************************************************************
>  *
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.h
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.h b/proxy/PluginVC.h
> index d770de5..6ee3a93 100644
> --- a/proxy/PluginVC.h
> +++ b/proxy/PluginVC.h
> @@ -36,6 +36,7 @@
> #ifndef _PLUGIN_VC_H_
> #define _PLUGIN_VC_H_
> 
> +#include "Plugin.h"
> #include "P_Net.h"
> #include "ink_atomic.h"
> 
> @@ -75,7 +76,7 @@ enum
>   PLUGIN_VC_MAGIC_DEAD = 0xaabbdead
> };
> 
> -class PluginVC:public NetVConnection
> +class PluginVC:public NetVConnection, public PluginIdentity
> {
>   friend class PluginVCCore;
> public:
> @@ -114,6 +115,18 @@ public:
> 
>   virtual PluginVC* get_other_side() { return other_side; }
> 
> +  //@{ @name Plugin identity.
> +  /// Override for @c PluginIdentity.
> +  virtual char const* getPluginTag() const { return plugin_tag; }
> +  /// Override for @c PluginIdentity.
> +  virtual int64_t getPluginId() const { return plugin_id; }
> +
> +  /// Setter for plugin tag.
> +  virtual void setPluginTag(char const* tag) { plugin_tag = tag; }
> +  /// Setter for plugin id.
> +  virtual void setPluginId(int64_t id) { plugin_id = id; }
> +  //@}
> +
>   int main_handler(int event, void *data);
> 
> private:
> @@ -152,6 +165,9 @@ private:
>   ink_hrtime inactive_timeout;
>   ink_hrtime inactive_timeout_at;
>   Event *inactive_event;
> +
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> };
> 
> class PluginVCCore:public Continuation
> @@ -198,6 +214,11 @@ public:
> 
>   void set_transparent(bool passive_side, bool active_side);
> 
> +  /// Set the plugin ID for the internal VCs.
> +  void set_plugin_id(int64_t id);
> +  /// Set the plugin tag for the internal VCs.
> +  void set_plugin_tag(char const* tag);
> +
>   // The active vc is handed to the initiator of
>   //   connection.  The passive vc is handled to
>   //   receiver of the connection
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/api/ts/ts.h
> ----------------------------------------------------------------------
> diff --git a/proxy/api/ts/ts.h b/proxy/api/ts/ts.h
> index 27d0546..917d6b2 100644
> --- a/proxy/api/ts/ts.h
> +++ b/proxy/api/ts/ts.h
> @@ -1588,12 +1588,16 @@ extern "C"
>       than TSNetConnect() to localhost since it avoids the overhead of
>       passing the data through the operating system.
> 
> -      @param log_ip ip address (in network byte order) that connection
> -        will be logged as coming from.
> -      @param log_port port (in network byte order) that connection will
> -        be logged as coming from.
> -      @param vc will be set to point to the new TSVConn on success.
> +      This returns a VConn that connected to the transaction.
> +
> +      @param addr Target address of the origin server.
> +      @param tag A logging tag that can be accessed via the pitag field. May be @c NULL.
> +      @param id A logging id that can be access via the piid field.
> +   */
> +  tsapi TSVConn TSHttpConnectWithPluginId(struct sockaddr const* addr, char const* tag, int64_t id);
> 
> +  /** Backwards compatible version.
> +      This provides a @a tag of "plugin" and an @a id of 0.
>    */
>   tsapi TSVConn TSHttpConnect(struct sockaddr const* addr);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
> index d2246c8..0620b3b 100644
> --- a/proxy/http/HttpClientSession.cc
> +++ b/proxy/http/HttpClientSession.cc
> @@ -36,6 +36,7 @@
> #include "HttpSM.h"
> #include "HttpDebugNames.h"
> #include "HttpServerSession.h"
> +#include "Plugin.h"
> 
> #define DebugSsn(tag, ...) DebugSpecific(debug_on, tag, __VA_ARGS__)
> #define STATE_ENTER(state_name, event, vio) { \
> @@ -137,6 +138,7 @@ void
> HttpClientSession::new_transaction()
> {
>   ink_assert(current_reader == NULL);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(client_vc);
> 
>   read_state = HCS_ACTIVE_READER;
>   current_reader = HttpSM::allocate();
> @@ -145,6 +147,12 @@ HttpClientSession::new_transaction()
>   DebugSsn("http_cs", "[%" PRId64 "] Starting transaction %d using sm [%" PRId64 "]", con_id, transact_count, current_reader->sm_id);
> 
>   current_reader->attach_client_session(this, sm_reader);
> +  if (pi) {
> +    // it's a plugin VC of some sort with identify information.
> +    // copy it to the SM.
> +    current_reader->plugin_tag = pi->getPluginTag();
> +    current_reader->plugin_id = pi->getPluginId();
> +  }
> }
> 
> inline void
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpProxyServerMain.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
> index 2bf7d64..9eb9291 100644
> --- a/proxy/http/HttpProxyServerMain.cc
> +++ b/proxy/http/HttpProxyServerMain.cc
> @@ -180,8 +180,7 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
> 
> #if TS_HAS_SPDY
>   if (port.m_session_protocol_preference.intersects(SPDY_PROTOCOL_SET)) {
> -    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY,
> -                            new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>   }
> #endif
> 
> @@ -207,11 +206,11 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
>     // SPDY
> #if TS_HAS_SPDY
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(spdy::SESSION_VERSION_3));
>     }
> 
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3_1)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>     }
> #endif
> 
> @@ -316,7 +315,7 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
>   opt.localhost_only = true;
>   ha_opt.backdoor = true;
>   opt.backdoor = true;
> -  
> +
>   // The backdoor only binds the loopback interface
>   netProcessor.main_accept(new HttpSessionAccept(ha_opt), NO_FD, opt);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
> index 2eabc1d..02afe4a 100644
> --- a/proxy/http/HttpSM.cc
> +++ b/proxy/http/HttpSM.cc
> @@ -325,6 +325,7 @@ HttpSM::HttpSM()
>     client_response_hdr_bytes(0), client_response_body_bytes(0),
>     cache_response_hdr_bytes(0), cache_response_body_bytes(0),
>     pushed_response_hdr_bytes(0), pushed_response_body_bytes(0),
> +    plugin_tag(0), plugin_id(0),
>     hooks_set(0), cur_hook_id(TS_HTTP_LAST_HOOK), cur_hook(NULL),
>     cur_hooks(0), callout_state(HTTP_API_NO_CALLOUT), terminate_sm(false), kill_this_async_done(false)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
> index 2b5a6fd..13121ec 100644
> --- a/proxy/http/HttpSM.h
> +++ b/proxy/http/HttpSM.h
> @@ -488,6 +488,11 @@ public:
>   int pushed_response_hdr_bytes;
>   int64_t pushed_response_body_bytes;
>   TransactionMilestones milestones;
> +  // The next two enable plugins to tag the state machine for
> +  // the purposes of logging so the instances can be correlated
> +  // with the source plugin.
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> 
>   // hooks_set records whether there are any hooks relevant
>   //  to this transaction.  Used to avoid costly calls
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/Log.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/Log.cc b/proxy/logging/Log.cc
> index 16138b4..9ee618e 100644
> --- a/proxy/logging/Log.cc
> +++ b/proxy/logging/Log.cc
> @@ -365,27 +365,19 @@ Log::init_fields()
>   global_field_list.add (field, false);
>   ink_hash_table_insert (field_symbol_hash, "caun", field);
> 
> -# if 0
> -  Ptr<LogFieldAliasTable> proto_type_map = make_ptr(new LogFieldAliasTable);
> -  proto_type_map->init(7,
> -                       // Transport protocols
> -                       TS_PROTO_UDP, "UDP",
> -                       TS_PROTO_TCP, "TCP",
> -                       TS_PROTO_TLS, "TLS",
> -                       // Application protocols
> -                       TS_PROTO_HTTP, "HTTP",
> -                       TS_PROTO_SPDY, "SPDY",
> -                       TS_PROTO_RTMP, "RTMP",
> -                       TS_PROTO_WS,   "WS");
> -
> -  field = new LogField("client_protocol_stack", "cps",
> +  field = new LogField("plugin_identity_id", "piid",
>                        LogField::sINT,
> -                       &LogAccess::marshal_client_protocol_stack,
> -                       &LogAccess::unmarshal_client_protocol_stack,
> -                       (Ptr<LogFieldAliasMap>) proto_type_map);
> +                       &LogAccess::marshal_plugin_identity_id,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_int_to_str));
>   global_field_list.add(field, false);
> -  ink_hash_table_insert(field_symbol_hash, "cps", field);
> -# endif
> +  ink_hash_table_insert(field_symbol_hash, "piid", field);
> +
> +  field = new LogField("plugin_identity_tag", "pitag",
> +                       LogField::STRING,
> +                       &LogAccess::marshal_plugin_identity_tag,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_str));
> +  global_field_list.add(field, false);
> +  ink_hash_table_insert(field_symbol_hash, "pitag", field);
> 
>   field = new LogField("client_req_timestamp_sec", "cqts",
>                        LogField::sINT,
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.cc b/proxy/logging/LogAccess.cc
> index 968762e..5009113 100644
> --- a/proxy/logging/LogAccess.cc
> +++ b/proxy/logging/LogAccess.cc
> @@ -75,12 +75,18 @@ LogAccess::init()
>   -------------------------------------------------------------------------*/
> 
> int
> -LogAccess::marshal_client_protocol_stack(char *buf)
> +LogAccess::marshal_plugin_identity_id(char *buf)
> {
>   DEFAULT_INT_FIELD;
> }
> 
> int
> +LogAccess::marshal_plugin_identity_tag(char *buf)
> +{
> +  DEFAULT_STR_FIELD;
> +}
> +
> +int
> LogAccess::marshal_client_host_ip(char *buf)
> {
>   DEFAULT_IP_FIELD;
> @@ -1311,43 +1317,6 @@ LogAccess::unmarshal_cache_write_code(char **buf, char *dest, int len, Ptr<LogFi
>   return (LogAccess::unmarshal_with_map(unmarshal_int(buf), dest, len, map, "UNKNOWN_CACHE_WRITE_CODE"));
> }
> 
> -# if 0
> -int
> -LogAccess::unmarshal_client_protocol_stack(char **buf, char *dest, int len, Ptr<LogFieldAliasMap> map)
> -{
> -  ink_assert(buf != NULL);
> -  ink_assert(*buf != NULL);
> -  ink_assert(dest != NULL);
> -
> -  char *p;
> -  size_t nr_chars = 0;
> -  int i, ret, nr_bits, left_len;
> -  TSClientProtoStack proto_stack = (TSClientProtoStack)unmarshal_int(buf);
> -
> -  p = dest;
> -  left_len = len;
> -  nr_bits = 8 * sizeof(TSClientProtoStack);
> -
> -  for (i = 0; i < nr_bits && left_len; i++) {
> -    if ((proto_stack >> i) & 0x1) {
> -      if (p != dest) {
> -        *p++ = '+';
> -        left_len--;
> -      }
> -      ret = map->asString(i, p, left_len, &nr_chars);
> -      if (ret == LogFieldAliasMap::ALL_OK) {
> -        p += nr_chars;
> -        left_len -= nr_chars;
> -      } else if (ret == LogFieldAliasMap::BUFFER_TOO_SMALL) {
> -        break;
> -      }
> -    }
> -  }
> -
> -  return (len - left_len);
> -}
> -# endif
> -
> int
> LogAccess::unmarshal_record(char **buf, char *dest, int len)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.h b/proxy/logging/LogAccess.h
> index 940145d..8fb35b5 100644
> --- a/proxy/logging/LogAccess.h
> +++ b/proxy/logging/LogAccess.h
> @@ -172,7 +172,6 @@ public:
>   inkcoreapi virtual int marshal_client_auth_user_name(char *); // STR
>   int marshal_client_req_timestamp_sec(char *); // INT
> 
> -  inkcoreapi virtual int marshal_client_protocol_stack(char *); // INT
>   inkcoreapi virtual int marshal_client_req_text(char *);       // STR
>   inkcoreapi virtual int marshal_client_req_http_method(char *);        // STR
>   inkcoreapi virtual int marshal_client_req_url(char *);        // STR
> @@ -251,6 +250,8 @@ public:
>   inkcoreapi virtual int marshal_transfer_time_ms(char *);      // INT
>   inkcoreapi virtual int marshal_transfer_time_s(char *);       // INT
>   inkcoreapi virtual int marshal_file_size(char *);     // INT
> +  inkcoreapi virtual int marshal_plugin_identity_id(char *); // INT
> +  inkcoreapi virtual int marshal_plugin_identity_tag(char *); // STR
>   int marshal_entry_type(char *);       // INT
> 
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.cc b/proxy/logging/LogAccessHttp.cc
> index a8dad70..b44e684 100644
> --- a/proxy/logging/LogAccessHttp.cc
> +++ b/proxy/logging/LogAccessHttp.cc
> @@ -206,16 +206,27 @@ LogAccessHttp::set_client_req_url_path(char *buf, int len)
> 
> /*-------------------------------------------------------------------------
>   -------------------------------------------------------------------------*/
> -# if 0
> int
> -LogAccessHttp::marshal_client_protocol_stack(char *buf)
> +LogAccessHttp::marshal_plugin_identity_id(char *buf)
> {
> -  if (buf) {
> -    marshal_int(buf, m_http_sm->proto_stack);
> -  }
> +  if (buf) marshal_int(buf, m_http_sm->plugin_id);
>   return INK_MIN_ALIGN;
> }
> -# endif
> +
> +int
> +LogAccessHttp::marshal_plugin_identity_tag(char *buf)
> +{
> +  int len = INK_MIN_ALIGN;
> +  char const* tag = m_http_sm->plugin_tag;
> +
> +  if (!tag) tag = "*";
> +  else len = LogAccess::strlen(tag);
> +
> +  if (buf) marshal_str(buf, tag, len);
> +
> +  return len;
> +}
> +
> int
> LogAccessHttp::marshal_client_host_ip(char *buf)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.h b/proxy/logging/LogAccessHttp.h
> index 53a0ff5..51ee9e3 100644
> --- a/proxy/logging/LogAccessHttp.h
> +++ b/proxy/logging/LogAccessHttp.h
> @@ -58,7 +58,6 @@ public:
>   virtual int marshal_client_host_ip(char *);   // STR
>   virtual int marshal_client_host_port(char *); // INT
>   virtual int marshal_client_auth_user_name(char *);    // STR
> -  //  virtual int marshal_client_protocol_stack(char *);    // INT
>   virtual int marshal_client_req_text(char *);  // STR
>   virtual int marshal_client_req_http_method(char *);   // INT
>   virtual int marshal_client_req_url(char *);   // STR
> @@ -128,6 +127,8 @@ public:
>   virtual int marshal_transfer_time_ms(char *); // INT
>   virtual int marshal_transfer_time_s(char *);  // INT
>   virtual int marshal_file_size(char *); // INT
> +  virtual int marshal_plugin_identity_id(char *);    // INT
> +  virtual int marshal_plugin_identity_tag(char *);    // STR
> 
>   //
>   // named fields from within a http header
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.cc b/proxy/spdy/SpdyClientSession.cc
> index 2c77ff6..fcb79fb 100644
> --- a/proxy/spdy/SpdyClientSession.cc
> +++ b/proxy/spdy/SpdyClientSession.cc
> @@ -27,6 +27,22 @@
> static ClassAllocator<SpdyClientSession> spdyClientSessionAllocator("spdyClientSessionAllocator");
> ClassAllocator<SpdyRequest> spdyRequestAllocator("spdyRequestAllocator");
> 
> +#if TS_HAS_SPDY
> +#include "SpdyClientSession.h"
> +
> +static const spdylay_proto_version versmap[] = {
> +  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> +  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> +  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> +};
> +
> +static char const* const  npnmap[] = {
> +  TS_NPN_PROTOCOL_SPDY_2,
> +  TS_NPN_PROTOCOL_SPDY_3,
> +  TS_NPN_PROTOCOL_SPDY_3_1
> +};
> +
> +#endif
> static int spdy_process_read(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_write(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_fetch(TSEvent event, SpdyClientSession *sm, void *edata);
> @@ -69,15 +85,16 @@ SpdyRequest::clear()
> }
> 
> void
> -SpdyClientSession::init(NetVConnection * netvc, spdylay_proto_version vers)
> +SpdyClientSession::init(NetVConnection * netvc, spdy::SessionVersion vers)
> {
>   int r;
> 
>   this->mutex = new_ProxyMutex();
>   this->vc = netvc;
>   this->req_map.clear();
> +  this->version = vers;
> 
> -  r = spdylay_session_server_new(&session, vers, &SPDY_CFG.spdy.callbacks, this);
> +  r = spdylay_session_server_new(&session, versmap[vers], &SPDY_CFG.spdy.callbacks, this);
> 
>   // A bit ugly but we need a thread and I don't want to wait until the
>   // session start event in case of a time out generating a decrement
> @@ -157,7 +174,7 @@ SpdyClientSession::clear()
> }
> 
> void
> -spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader)
> +spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader)
> {
>   SpdyClientSession  *sm;
> 
> @@ -242,6 +259,19 @@ out:
>   return EVENT_CONT;
> }
> 
> +int64_t
> +SpdyClientSession::getPluginId() const
> +{
> +  return sm_id;
> +}
> +
> +char const*
> +SpdyClientSession::getPluginTag() const
> +{
> +  return npnmap[this->version];
> +}
> +
> +
> static int
> spdy_process_read(TSEvent /* event ATS_UNUSED */, SpdyClientSession *sm)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.h b/proxy/spdy/SpdyClientSession.h
> index 2f37d35..738ff55 100644
> --- a/proxy/spdy/SpdyClientSession.h
> +++ b/proxy/spdy/SpdyClientSession.h
> @@ -24,9 +24,11 @@
> #ifndef __P_SPDY_SM_H__
> #define __P_SPDY_SM_H__
> 
> +#include "SpdyDefs.h"
> #include "SpdyCommon.h"
> #include "SpdyCallbacks.h"
> #include <openssl/md5.h>
> +#include "Plugin.h"
> 
> class SpdyClientSession;
> typedef int (*SpdyClientSessionHandler) (TSCont contp, TSEvent event, void *data);
> @@ -89,7 +91,7 @@ public:
>   MD5_CTX recv_md5;
> };
> 
> -class SpdyClientSession : public Continuation
> +class SpdyClientSession : public Continuation, public PluginIdentity
> {
> 
> public:
> @@ -101,10 +103,11 @@ public:
>     clear();
>   }
> 
> -  void init(NetVConnection * netvc, spdylay_proto_version vers);
> +  void init(NetVConnection * netvc, spdy::SessionVersion vers);
>   void clear();
> 
>   int64_t sm_id;
> +  spdy::SessionVersion version;
>   uint64_t total_size;
>   TSHRTime start_time;
> 
> @@ -124,12 +127,15 @@ public:
> 
>   map<int32_t, SpdyRequest*> req_map;
> 
> +  virtual char const* getPluginTag() const;
> +  virtual int64_t getPluginId() const;
> +
> private:
>   int state_session_start(int event, void * edata);
>   int state_session_readwrite(int event, void * edata);
> };
> 
> -void spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader);
> +void spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader);
> 
> extern ClassAllocator<SpdyRequest> spdyRequestAllocator;
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
> index e679adc..d7dfa11 100644
> --- a/proxy/spdy/SpdySessionAccept.cc
> +++ b/proxy/spdy/SpdySessionAccept.cc
> @@ -26,20 +26,13 @@
> 
> #if TS_HAS_SPDY
> #include "SpdyClientSession.h"
> -
> -static const spdylay_proto_version versmap[] = {
> -  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> -  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> -  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> -};
> -
> #endif
> 
> -SpdySessionAccept::SpdySessionAccept(unsigned vers)
> +SpdySessionAccept::SpdySessionAccept(spdy::SessionVersion vers)
>     : SessionAccept(new_ProxyMutex()), version(vers)
> {
> #if TS_HAS_SPDY
> -  ink_release_assert(vers < countof(versmap));
> +  ink_release_assert(spdy::SESSION_VERSION_2 <= vers && vers <= spdy::SESSION_VERSION_3_1);
> #endif
>   SET_HANDLER(&SpdySessionAccept::mainEvent);
> }
> @@ -51,7 +44,7 @@ SpdySessionAccept::mainEvent(int event, void * edata)
>     NetVConnection * netvc =static_cast<NetVConnection *>(edata);
> 
> #if TS_HAS_SPDY
> -    spdy_sm_create(netvc, versmap[this->version], NULL, NULL);
> +    spdy_sm_create(netvc, this->version, NULL, NULL);
> #else
>     Error("accepted a SPDY session, but SPDY support is not available");
>     netvc->do_io_close();
> @@ -68,7 +61,7 @@ void
> SpdySessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
> {
> #if TS_HAS_SPDY
> -  spdy_sm_create(netvc, versmap[this->version], iobuf, reader);
> +  spdy_sm_create(netvc, this->version, iobuf, reader);
> #else
>   (void)netvc;
>   (void)iobuf;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.h b/proxy/spdy/SpdySessionAccept.h
> index 9d740e2..af3d81d 100644
> --- a/proxy/spdy/SpdySessionAccept.h
> +++ b/proxy/spdy/SpdySessionAccept.h
> @@ -28,18 +28,13 @@
> #include "P_EventSystem.h"
> #include "P_UnixNet.h"
> #include "I_IOBuffer.h"
> +#include "SpdyDefs.h"
> 
> class SpdySessionAccept: public SessionAccept
> {
> public:
> 
> -  enum {
> -    SPDY_VERSION_2 = 0,
> -    SPDY_VERSION_3,
> -    SPDY_VERSION_3_1,
> -  };
> -
> -  explicit SpdySessionAccept(unsigned vers);
> +  explicit SpdySessionAccept(spdy::SessionVersion vers);
>   ~SpdySessionAccept() {}
> 
>   void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
> @@ -49,7 +44,7 @@ private:
>   SpdySessionAccept(const SpdySessionAccept &); // disabled
>   SpdySessionAccept& operator =(const SpdySessionAccept&); // disabled
> 
> -  unsigned version;
> +  spdy::SessionVersion version;
> };
> 
> #endif /* SpdySessionAccept_H_ */
> 


Re: git commit: TS-2838 Add log fields for plugins on HttpSM and use for SPDY.

Posted by James Peach <jp...@apache.org>.
Please remove the documentation for the 'cps' log field.
Please document the 'piid' log field.
I don't see how this is wired up to SPDY, could you spell that out for me?

On May 23, 2014, at 9:30 AM, amc@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master b6383928a -> aadb75226
> 
> 
> TS-2838 Add log fields for plugins on HttpSM and use for SPDY.
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/aadb7522
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/aadb7522
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/aadb7522
> 
> Branch: refs/heads/master
> Commit: aadb752267ba194a3a6f584c0874cda36e6ab859
> Parents: b638392
> Author: Alan M. Carroll <am...@network-geographics.com>
> Authored: Fri May 23 09:30:03 2014 -0700
> Committer: Alan M. Carroll <am...@network-geographics.com>
> Committed: Fri May 23 09:30:03 2014 -0700
> 
> ----------------------------------------------------------------------
> CHANGES                           |  2 ++
> proxy/FetchSM.cc                  | 17 ++++++-------
> proxy/FetchSM.h                   |  4 ++-
> proxy/InkAPI.cc                   |  9 ++++++-
> proxy/Plugin.h                    | 29 ++++++++++++++++++++++
> proxy/PluginVC.cc                 | 12 ++++++++-
> proxy/PluginVC.h                  | 23 ++++++++++++++++-
> proxy/api/ts/ts.h                 | 14 +++++++----
> proxy/http/HttpClientSession.cc   |  8 ++++++
> proxy/http/HttpProxyServerMain.cc |  9 +++----
> proxy/http/HttpSM.cc              |  1 +
> proxy/http/HttpSM.h               |  5 ++++
> proxy/logging/Log.cc              | 30 +++++++++--------------
> proxy/logging/LogAccess.cc        | 45 ++++++----------------------------
> proxy/logging/LogAccess.h         |  3 ++-
> proxy/logging/LogAccessHttp.cc    | 23 ++++++++++++-----
> proxy/logging/LogAccessHttp.h     |  3 ++-
> proxy/spdy/SpdyClientSession.cc   | 36 ++++++++++++++++++++++++---
> proxy/spdy/SpdyClientSession.h    | 12 ++++++---
> proxy/spdy/SpdySessionAccept.cc   | 15 +++---------
> proxy/spdy/SpdySessionAccept.h    | 11 +++------
> 21 files changed, 198 insertions(+), 113 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/CHANGES
> ----------------------------------------------------------------------
> diff --git a/CHANGES b/CHANGES
> index 4f8d8af..fa34b15 100644
> --- a/CHANGES
> +++ b/CHANGES
> @@ -1,6 +1,8 @@
>                                                          -*- coding: utf-8 -*-
> Changes with Apache Traffic Server 5.0.0
> 
> +  *) [TS-2838] Add logging fields for plugins to TS connect API. Use for SPDY.
> +
>   *) [TS-2833] Remove REC_BUILD_STAND_ALONE, REC_BUILD_MGMT and
>    REC_BUILD_STUB. Dead code.
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.cc b/proxy/FetchSM.cc
> index c119aca..a0a4033 100644
> --- a/proxy/FetchSM.cc
> +++ b/proxy/FetchSM.cc
> @@ -61,23 +61,22 @@ FetchSM::cleanUp()
>   client_response_hdr.destroy();
>   ats_free(client_response);
>   cont_mutex.clear();
> -
> -  PluginVC *vc = (PluginVC *) http_vc;
> -
> -  vc->do_io_close();
> +  http_vc->do_io_close();
>   FetchSMAllocator.free(this);
> }
> 
> void
> FetchSM::httpConnect()
> {
> -  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> -  http_vc = TSHttpConnect(&_addr.sa);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(contp);
> +  char const* tag = pi ? pi->getPluginTag() : "fetchSM";
> +  int64_t id = pi ? pi->getPluginId() : 0;
> 
> -  PluginVC *vc = (PluginVC *) http_vc;
> +  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> +  http_vc = reinterpret_cast<PluginVC*>(TSHttpConnectWithPluginId(&_addr.sa, tag, id));
> 
> -  read_vio = vc->do_io_read(this, INT64_MAX, resp_buffer);
> -  write_vio = vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> +  read_vio = http_vc->do_io_read(this, INT64_MAX, resp_buffer);
> +  write_vio = http_vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> }
> 
> char* FetchSM::resp_get(int *length) {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.h b/proxy/FetchSM.h
> index 0de5d96..c1b94bf 100644
> --- a/proxy/FetchSM.h
> +++ b/proxy/FetchSM.h
> @@ -35,6 +35,8 @@
> #include "HttpSM.h"
> #include "HttpTunnel.h"
> 
> +class PluginVC;
> +
> class FetchSM: public Continuation
> {
> public:
> @@ -139,7 +141,7 @@ private:
>   int dechunk_body();
> 
>   int recursion;
> -  TSVConn http_vc;
> +  PluginVC* http_vc;
>   VIO *read_vio;
>   VIO *write_vio;
>   MIOBuffer *req_buffer;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/InkAPI.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
> index 0182ebe..dceea6e 100644
> --- a/proxy/InkAPI.cc
> +++ b/proxy/InkAPI.cc
> @@ -6047,7 +6047,7 @@ extern HttpSessionAccept *plugin_http_accept;
> extern HttpSessionAccept *plugin_http_transparent_accept;
> 
> TSVConn
> -TSHttpConnect(sockaddr const* addr)
> +TSHttpConnectWithPluginId(sockaddr const* addr, char const* tag, int64_t id)
> {
>   sdk_assert(addr);
> 
> @@ -6058,6 +6058,8 @@ TSHttpConnect(sockaddr const* addr)
>     PluginVCCore *new_pvc = PluginVCCore::alloc();
> 
>     new_pvc->set_active_addr(addr);
> +    new_pvc->set_plugin_id(id);
> +    new_pvc->set_plugin_tag(tag);
>     new_pvc->set_accept_cont(plugin_http_accept);
> 
>     PluginVC *return_vc = new_pvc->connect();
> @@ -6076,6 +6078,11 @@ TSHttpConnect(sockaddr const* addr)
>   return NULL;
> }
> 
> +TSVConn
> +TSHttpConnect(sockaddr const* addr)
> +{
> +  return TSHttpConnectWithPluginId(addr, "plugin", 0);
> +}
> 
> TSVConn
> TSHttpConnectTransparent(sockaddr const* client_addr, sockaddr const* server_addr)
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/Plugin.h
> ----------------------------------------------------------------------
> diff --git a/proxy/Plugin.h b/proxy/Plugin.h
> index 4c1ad09..8f70f1a 100644
> --- a/proxy/Plugin.h
> +++ b/proxy/Plugin.h
> @@ -58,4 +58,33 @@ extern PluginRegInfo *plugin_reg_current;
> 
> void plugin_init(void);
> 
> +/** Abstract interface class for plugin based continuations.
> +
> +    The primary intended use of this is for logging so that continuations
> +    that generate logging messages can generate plugin local data in a
> +    generic way.
> +
> +    The core will at appropriate times dynamically cast the continuation
> +    to this class and if successful access the plugin data via these
> +    methods.
> +
> +    Plugins should mix this in to continuations for which it is useful.
> +    The default implementations return empty / invalid responses and should
> +    be overridden by the plugin.
> + */
> +class PluginIdentity
> +{
> + public:
> +  /** Get the plugin tag.
> +      The returned string must have a lifetime at least as long as the plugin.
> +      @return A string identifying the plugin or @c NULL.
> +  */
> +  virtual char const* getPluginTag() const { return NULL; }
> +  /** Get the plugin instance ID.
> +      A plugin can create multiple subsidiary instances. This is used as the
> +      identifier for those to distinguish the instances.
> +  */
> +  virtual int64_t getPluginId() const { return 0; }
> +};
> +
> #endif /* __PLUGIN_H__ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.cc b/proxy/PluginVC.cc
> index 2e4cfa1..16eb5a2 100644
> --- a/proxy/PluginVC.cc
> +++ b/proxy/PluginVC.cc
> @@ -1051,7 +1051,6 @@ void
> PluginVCCore::set_accept_cont(Continuation * c)
> {
>   connect_to = c;
> -
>   // FIX ME - must return action
> }
> 
> @@ -1202,6 +1201,17 @@ PluginVCCore::set_transparent(bool passive_side, bool active_side)
>   active_vc.set_is_transparent(active_side);
> }
> 
> +void
> +PluginVCCore::set_plugin_id(int64_t id)
> +{
> +  passive_vc.plugin_id = active_vc.plugin_id = id;
> +}
> +
> +void
> +PluginVCCore::set_plugin_tag(char const* tag)
> +{
> +  passive_vc.plugin_tag = active_vc.plugin_tag = tag;
> +}
> 
> /*************************************************************
>  *
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.h
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.h b/proxy/PluginVC.h
> index d770de5..6ee3a93 100644
> --- a/proxy/PluginVC.h
> +++ b/proxy/PluginVC.h
> @@ -36,6 +36,7 @@
> #ifndef _PLUGIN_VC_H_
> #define _PLUGIN_VC_H_
> 
> +#include "Plugin.h"
> #include "P_Net.h"
> #include "ink_atomic.h"
> 
> @@ -75,7 +76,7 @@ enum
>   PLUGIN_VC_MAGIC_DEAD = 0xaabbdead
> };
> 
> -class PluginVC:public NetVConnection
> +class PluginVC:public NetVConnection, public PluginIdentity
> {
>   friend class PluginVCCore;
> public:
> @@ -114,6 +115,18 @@ public:
> 
>   virtual PluginVC* get_other_side() { return other_side; }
> 
> +  //@{ @name Plugin identity.
> +  /// Override for @c PluginIdentity.
> +  virtual char const* getPluginTag() const { return plugin_tag; }
> +  /// Override for @c PluginIdentity.
> +  virtual int64_t getPluginId() const { return plugin_id; }
> +
> +  /// Setter for plugin tag.
> +  virtual void setPluginTag(char const* tag) { plugin_tag = tag; }
> +  /// Setter for plugin id.
> +  virtual void setPluginId(int64_t id) { plugin_id = id; }
> +  //@}
> +
>   int main_handler(int event, void *data);
> 
> private:
> @@ -152,6 +165,9 @@ private:
>   ink_hrtime inactive_timeout;
>   ink_hrtime inactive_timeout_at;
>   Event *inactive_event;
> +
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> };
> 
> class PluginVCCore:public Continuation
> @@ -198,6 +214,11 @@ public:
> 
>   void set_transparent(bool passive_side, bool active_side);
> 
> +  /// Set the plugin ID for the internal VCs.
> +  void set_plugin_id(int64_t id);
> +  /// Set the plugin tag for the internal VCs.
> +  void set_plugin_tag(char const* tag);
> +
>   // The active vc is handed to the initiator of
>   //   connection.  The passive vc is handled to
>   //   receiver of the connection
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/api/ts/ts.h
> ----------------------------------------------------------------------
> diff --git a/proxy/api/ts/ts.h b/proxy/api/ts/ts.h
> index 27d0546..917d6b2 100644
> --- a/proxy/api/ts/ts.h
> +++ b/proxy/api/ts/ts.h
> @@ -1588,12 +1588,16 @@ extern "C"
>       than TSNetConnect() to localhost since it avoids the overhead of
>       passing the data through the operating system.
> 
> -      @param log_ip ip address (in network byte order) that connection
> -        will be logged as coming from.
> -      @param log_port port (in network byte order) that connection will
> -        be logged as coming from.
> -      @param vc will be set to point to the new TSVConn on success.
> +      This returns a VConn that connected to the transaction.
> +
> +      @param addr Target address of the origin server.
> +      @param tag A logging tag that can be accessed via the pitag field. May be @c NULL.
> +      @param id A logging id that can be access via the piid field.
> +   */
> +  tsapi TSVConn TSHttpConnectWithPluginId(struct sockaddr const* addr, char const* tag, int64_t id);
> 
> +  /** Backwards compatible version.
> +      This provides a @a tag of "plugin" and an @a id of 0.
>    */
>   tsapi TSVConn TSHttpConnect(struct sockaddr const* addr);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
> index d2246c8..0620b3b 100644
> --- a/proxy/http/HttpClientSession.cc
> +++ b/proxy/http/HttpClientSession.cc
> @@ -36,6 +36,7 @@
> #include "HttpSM.h"
> #include "HttpDebugNames.h"
> #include "HttpServerSession.h"
> +#include "Plugin.h"
> 
> #define DebugSsn(tag, ...) DebugSpecific(debug_on, tag, __VA_ARGS__)
> #define STATE_ENTER(state_name, event, vio) { \
> @@ -137,6 +138,7 @@ void
> HttpClientSession::new_transaction()
> {
>   ink_assert(current_reader == NULL);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(client_vc);
> 
>   read_state = HCS_ACTIVE_READER;
>   current_reader = HttpSM::allocate();
> @@ -145,6 +147,12 @@ HttpClientSession::new_transaction()
>   DebugSsn("http_cs", "[%" PRId64 "] Starting transaction %d using sm [%" PRId64 "]", con_id, transact_count, current_reader->sm_id);
> 
>   current_reader->attach_client_session(this, sm_reader);
> +  if (pi) {
> +    // it's a plugin VC of some sort with identify information.
> +    // copy it to the SM.
> +    current_reader->plugin_tag = pi->getPluginTag();
> +    current_reader->plugin_id = pi->getPluginId();
> +  }
> }
> 
> inline void
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpProxyServerMain.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
> index 2bf7d64..9eb9291 100644
> --- a/proxy/http/HttpProxyServerMain.cc
> +++ b/proxy/http/HttpProxyServerMain.cc
> @@ -180,8 +180,7 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
> 
> #if TS_HAS_SPDY
>   if (port.m_session_protocol_preference.intersects(SPDY_PROTOCOL_SET)) {
> -    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY,
> -                            new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>   }
> #endif
> 
> @@ -207,11 +206,11 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
>     // SPDY
> #if TS_HAS_SPDY
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(spdy::SESSION_VERSION_3));
>     }
> 
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3_1)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>     }
> #endif
> 
> @@ -316,7 +315,7 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
>   opt.localhost_only = true;
>   ha_opt.backdoor = true;
>   opt.backdoor = true;
> -  
> +
>   // The backdoor only binds the loopback interface
>   netProcessor.main_accept(new HttpSessionAccept(ha_opt), NO_FD, opt);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
> index 2eabc1d..02afe4a 100644
> --- a/proxy/http/HttpSM.cc
> +++ b/proxy/http/HttpSM.cc
> @@ -325,6 +325,7 @@ HttpSM::HttpSM()
>     client_response_hdr_bytes(0), client_response_body_bytes(0),
>     cache_response_hdr_bytes(0), cache_response_body_bytes(0),
>     pushed_response_hdr_bytes(0), pushed_response_body_bytes(0),
> +    plugin_tag(0), plugin_id(0),
>     hooks_set(0), cur_hook_id(TS_HTTP_LAST_HOOK), cur_hook(NULL),
>     cur_hooks(0), callout_state(HTTP_API_NO_CALLOUT), terminate_sm(false), kill_this_async_done(false)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
> index 2b5a6fd..13121ec 100644
> --- a/proxy/http/HttpSM.h
> +++ b/proxy/http/HttpSM.h
> @@ -488,6 +488,11 @@ public:
>   int pushed_response_hdr_bytes;
>   int64_t pushed_response_body_bytes;
>   TransactionMilestones milestones;
> +  // The next two enable plugins to tag the state machine for
> +  // the purposes of logging so the instances can be correlated
> +  // with the source plugin.
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> 
>   // hooks_set records whether there are any hooks relevant
>   //  to this transaction.  Used to avoid costly calls
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/Log.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/Log.cc b/proxy/logging/Log.cc
> index 16138b4..9ee618e 100644
> --- a/proxy/logging/Log.cc
> +++ b/proxy/logging/Log.cc
> @@ -365,27 +365,19 @@ Log::init_fields()
>   global_field_list.add (field, false);
>   ink_hash_table_insert (field_symbol_hash, "caun", field);
> 
> -# if 0
> -  Ptr<LogFieldAliasTable> proto_type_map = make_ptr(new LogFieldAliasTable);
> -  proto_type_map->init(7,
> -                       // Transport protocols
> -                       TS_PROTO_UDP, "UDP",
> -                       TS_PROTO_TCP, "TCP",
> -                       TS_PROTO_TLS, "TLS",
> -                       // Application protocols
> -                       TS_PROTO_HTTP, "HTTP",
> -                       TS_PROTO_SPDY, "SPDY",
> -                       TS_PROTO_RTMP, "RTMP",
> -                       TS_PROTO_WS,   "WS");
> -
> -  field = new LogField("client_protocol_stack", "cps",
> +  field = new LogField("plugin_identity_id", "piid",
>                        LogField::sINT,
> -                       &LogAccess::marshal_client_protocol_stack,
> -                       &LogAccess::unmarshal_client_protocol_stack,
> -                       (Ptr<LogFieldAliasMap>) proto_type_map);
> +                       &LogAccess::marshal_plugin_identity_id,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_int_to_str));
>   global_field_list.add(field, false);
> -  ink_hash_table_insert(field_symbol_hash, "cps", field);
> -# endif
> +  ink_hash_table_insert(field_symbol_hash, "piid", field);
> +
> +  field = new LogField("plugin_identity_tag", "pitag",
> +                       LogField::STRING,
> +                       &LogAccess::marshal_plugin_identity_tag,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_str));
> +  global_field_list.add(field, false);
> +  ink_hash_table_insert(field_symbol_hash, "pitag", field);
> 
>   field = new LogField("client_req_timestamp_sec", "cqts",
>                        LogField::sINT,
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.cc b/proxy/logging/LogAccess.cc
> index 968762e..5009113 100644
> --- a/proxy/logging/LogAccess.cc
> +++ b/proxy/logging/LogAccess.cc
> @@ -75,12 +75,18 @@ LogAccess::init()
>   -------------------------------------------------------------------------*/
> 
> int
> -LogAccess::marshal_client_protocol_stack(char *buf)
> +LogAccess::marshal_plugin_identity_id(char *buf)
> {
>   DEFAULT_INT_FIELD;
> }
> 
> int
> +LogAccess::marshal_plugin_identity_tag(char *buf)
> +{
> +  DEFAULT_STR_FIELD;
> +}
> +
> +int
> LogAccess::marshal_client_host_ip(char *buf)
> {
>   DEFAULT_IP_FIELD;
> @@ -1311,43 +1317,6 @@ LogAccess::unmarshal_cache_write_code(char **buf, char *dest, int len, Ptr<LogFi
>   return (LogAccess::unmarshal_with_map(unmarshal_int(buf), dest, len, map, "UNKNOWN_CACHE_WRITE_CODE"));
> }
> 
> -# if 0
> -int
> -LogAccess::unmarshal_client_protocol_stack(char **buf, char *dest, int len, Ptr<LogFieldAliasMap> map)
> -{
> -  ink_assert(buf != NULL);
> -  ink_assert(*buf != NULL);
> -  ink_assert(dest != NULL);
> -
> -  char *p;
> -  size_t nr_chars = 0;
> -  int i, ret, nr_bits, left_len;
> -  TSClientProtoStack proto_stack = (TSClientProtoStack)unmarshal_int(buf);
> -
> -  p = dest;
> -  left_len = len;
> -  nr_bits = 8 * sizeof(TSClientProtoStack);
> -
> -  for (i = 0; i < nr_bits && left_len; i++) {
> -    if ((proto_stack >> i) & 0x1) {
> -      if (p != dest) {
> -        *p++ = '+';
> -        left_len--;
> -      }
> -      ret = map->asString(i, p, left_len, &nr_chars);
> -      if (ret == LogFieldAliasMap::ALL_OK) {
> -        p += nr_chars;
> -        left_len -= nr_chars;
> -      } else if (ret == LogFieldAliasMap::BUFFER_TOO_SMALL) {
> -        break;
> -      }
> -    }
> -  }
> -
> -  return (len - left_len);
> -}
> -# endif
> -
> int
> LogAccess::unmarshal_record(char **buf, char *dest, int len)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.h b/proxy/logging/LogAccess.h
> index 940145d..8fb35b5 100644
> --- a/proxy/logging/LogAccess.h
> +++ b/proxy/logging/LogAccess.h
> @@ -172,7 +172,6 @@ public:
>   inkcoreapi virtual int marshal_client_auth_user_name(char *); // STR
>   int marshal_client_req_timestamp_sec(char *); // INT
> 
> -  inkcoreapi virtual int marshal_client_protocol_stack(char *); // INT
>   inkcoreapi virtual int marshal_client_req_text(char *);       // STR
>   inkcoreapi virtual int marshal_client_req_http_method(char *);        // STR
>   inkcoreapi virtual int marshal_client_req_url(char *);        // STR
> @@ -251,6 +250,8 @@ public:
>   inkcoreapi virtual int marshal_transfer_time_ms(char *);      // INT
>   inkcoreapi virtual int marshal_transfer_time_s(char *);       // INT
>   inkcoreapi virtual int marshal_file_size(char *);     // INT
> +  inkcoreapi virtual int marshal_plugin_identity_id(char *); // INT
> +  inkcoreapi virtual int marshal_plugin_identity_tag(char *); // STR
>   int marshal_entry_type(char *);       // INT
> 
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.cc b/proxy/logging/LogAccessHttp.cc
> index a8dad70..b44e684 100644
> --- a/proxy/logging/LogAccessHttp.cc
> +++ b/proxy/logging/LogAccessHttp.cc
> @@ -206,16 +206,27 @@ LogAccessHttp::set_client_req_url_path(char *buf, int len)
> 
> /*-------------------------------------------------------------------------
>   -------------------------------------------------------------------------*/
> -# if 0
> int
> -LogAccessHttp::marshal_client_protocol_stack(char *buf)
> +LogAccessHttp::marshal_plugin_identity_id(char *buf)
> {
> -  if (buf) {
> -    marshal_int(buf, m_http_sm->proto_stack);
> -  }
> +  if (buf) marshal_int(buf, m_http_sm->plugin_id);
>   return INK_MIN_ALIGN;
> }
> -# endif
> +
> +int
> +LogAccessHttp::marshal_plugin_identity_tag(char *buf)
> +{
> +  int len = INK_MIN_ALIGN;
> +  char const* tag = m_http_sm->plugin_tag;
> +
> +  if (!tag) tag = "*";
> +  else len = LogAccess::strlen(tag);
> +
> +  if (buf) marshal_str(buf, tag, len);
> +
> +  return len;
> +}
> +
> int
> LogAccessHttp::marshal_client_host_ip(char *buf)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.h b/proxy/logging/LogAccessHttp.h
> index 53a0ff5..51ee9e3 100644
> --- a/proxy/logging/LogAccessHttp.h
> +++ b/proxy/logging/LogAccessHttp.h
> @@ -58,7 +58,6 @@ public:
>   virtual int marshal_client_host_ip(char *);   // STR
>   virtual int marshal_client_host_port(char *); // INT
>   virtual int marshal_client_auth_user_name(char *);    // STR
> -  //  virtual int marshal_client_protocol_stack(char *);    // INT
>   virtual int marshal_client_req_text(char *);  // STR
>   virtual int marshal_client_req_http_method(char *);   // INT
>   virtual int marshal_client_req_url(char *);   // STR
> @@ -128,6 +127,8 @@ public:
>   virtual int marshal_transfer_time_ms(char *); // INT
>   virtual int marshal_transfer_time_s(char *);  // INT
>   virtual int marshal_file_size(char *); // INT
> +  virtual int marshal_plugin_identity_id(char *);    // INT
> +  virtual int marshal_plugin_identity_tag(char *);    // STR
> 
>   //
>   // named fields from within a http header
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.cc b/proxy/spdy/SpdyClientSession.cc
> index 2c77ff6..fcb79fb 100644
> --- a/proxy/spdy/SpdyClientSession.cc
> +++ b/proxy/spdy/SpdyClientSession.cc
> @@ -27,6 +27,22 @@
> static ClassAllocator<SpdyClientSession> spdyClientSessionAllocator("spdyClientSessionAllocator");
> ClassAllocator<SpdyRequest> spdyRequestAllocator("spdyRequestAllocator");
> 
> +#if TS_HAS_SPDY
> +#include "SpdyClientSession.h"
> +
> +static const spdylay_proto_version versmap[] = {
> +  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> +  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> +  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> +};
> +
> +static char const* const  npnmap[] = {
> +  TS_NPN_PROTOCOL_SPDY_2,
> +  TS_NPN_PROTOCOL_SPDY_3,
> +  TS_NPN_PROTOCOL_SPDY_3_1
> +};
> +
> +#endif
> static int spdy_process_read(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_write(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_fetch(TSEvent event, SpdyClientSession *sm, void *edata);
> @@ -69,15 +85,16 @@ SpdyRequest::clear()
> }
> 
> void
> -SpdyClientSession::init(NetVConnection * netvc, spdylay_proto_version vers)
> +SpdyClientSession::init(NetVConnection * netvc, spdy::SessionVersion vers)
> {
>   int r;
> 
>   this->mutex = new_ProxyMutex();
>   this->vc = netvc;
>   this->req_map.clear();
> +  this->version = vers;
> 
> -  r = spdylay_session_server_new(&session, vers, &SPDY_CFG.spdy.callbacks, this);
> +  r = spdylay_session_server_new(&session, versmap[vers], &SPDY_CFG.spdy.callbacks, this);
> 
>   // A bit ugly but we need a thread and I don't want to wait until the
>   // session start event in case of a time out generating a decrement
> @@ -157,7 +174,7 @@ SpdyClientSession::clear()
> }
> 
> void
> -spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader)
> +spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader)
> {
>   SpdyClientSession  *sm;
> 
> @@ -242,6 +259,19 @@ out:
>   return EVENT_CONT;
> }
> 
> +int64_t
> +SpdyClientSession::getPluginId() const
> +{
> +  return sm_id;
> +}
> +
> +char const*
> +SpdyClientSession::getPluginTag() const
> +{
> +  return npnmap[this->version];
> +}
> +
> +
> static int
> spdy_process_read(TSEvent /* event ATS_UNUSED */, SpdyClientSession *sm)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.h b/proxy/spdy/SpdyClientSession.h
> index 2f37d35..738ff55 100644
> --- a/proxy/spdy/SpdyClientSession.h
> +++ b/proxy/spdy/SpdyClientSession.h
> @@ -24,9 +24,11 @@
> #ifndef __P_SPDY_SM_H__
> #define __P_SPDY_SM_H__
> 
> +#include "SpdyDefs.h"
> #include "SpdyCommon.h"
> #include "SpdyCallbacks.h"
> #include <openssl/md5.h>
> +#include "Plugin.h"
> 
> class SpdyClientSession;
> typedef int (*SpdyClientSessionHandler) (TSCont contp, TSEvent event, void *data);
> @@ -89,7 +91,7 @@ public:
>   MD5_CTX recv_md5;
> };
> 
> -class SpdyClientSession : public Continuation
> +class SpdyClientSession : public Continuation, public PluginIdentity
> {
> 
> public:
> @@ -101,10 +103,11 @@ public:
>     clear();
>   }
> 
> -  void init(NetVConnection * netvc, spdylay_proto_version vers);
> +  void init(NetVConnection * netvc, spdy::SessionVersion vers);
>   void clear();
> 
>   int64_t sm_id;
> +  spdy::SessionVersion version;
>   uint64_t total_size;
>   TSHRTime start_time;
> 
> @@ -124,12 +127,15 @@ public:
> 
>   map<int32_t, SpdyRequest*> req_map;
> 
> +  virtual char const* getPluginTag() const;
> +  virtual int64_t getPluginId() const;
> +
> private:
>   int state_session_start(int event, void * edata);
>   int state_session_readwrite(int event, void * edata);
> };
> 
> -void spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader);
> +void spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader);
> 
> extern ClassAllocator<SpdyRequest> spdyRequestAllocator;
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
> index e679adc..d7dfa11 100644
> --- a/proxy/spdy/SpdySessionAccept.cc
> +++ b/proxy/spdy/SpdySessionAccept.cc
> @@ -26,20 +26,13 @@
> 
> #if TS_HAS_SPDY
> #include "SpdyClientSession.h"
> -
> -static const spdylay_proto_version versmap[] = {
> -  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> -  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> -  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> -};
> -
> #endif
> 
> -SpdySessionAccept::SpdySessionAccept(unsigned vers)
> +SpdySessionAccept::SpdySessionAccept(spdy::SessionVersion vers)
>     : SessionAccept(new_ProxyMutex()), version(vers)
> {
> #if TS_HAS_SPDY
> -  ink_release_assert(vers < countof(versmap));
> +  ink_release_assert(spdy::SESSION_VERSION_2 <= vers && vers <= spdy::SESSION_VERSION_3_1);
> #endif
>   SET_HANDLER(&SpdySessionAccept::mainEvent);
> }
> @@ -51,7 +44,7 @@ SpdySessionAccept::mainEvent(int event, void * edata)
>     NetVConnection * netvc =static_cast<NetVConnection *>(edata);
> 
> #if TS_HAS_SPDY
> -    spdy_sm_create(netvc, versmap[this->version], NULL, NULL);
> +    spdy_sm_create(netvc, this->version, NULL, NULL);
> #else
>     Error("accepted a SPDY session, but SPDY support is not available");
>     netvc->do_io_close();
> @@ -68,7 +61,7 @@ void
> SpdySessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
> {
> #if TS_HAS_SPDY
> -  spdy_sm_create(netvc, versmap[this->version], iobuf, reader);
> +  spdy_sm_create(netvc, this->version, iobuf, reader);
> #else
>   (void)netvc;
>   (void)iobuf;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.h b/proxy/spdy/SpdySessionAccept.h
> index 9d740e2..af3d81d 100644
> --- a/proxy/spdy/SpdySessionAccept.h
> +++ b/proxy/spdy/SpdySessionAccept.h
> @@ -28,18 +28,13 @@
> #include "P_EventSystem.h"
> #include "P_UnixNet.h"
> #include "I_IOBuffer.h"
> +#include "SpdyDefs.h"
> 
> class SpdySessionAccept: public SessionAccept
> {
> public:
> 
> -  enum {
> -    SPDY_VERSION_2 = 0,
> -    SPDY_VERSION_3,
> -    SPDY_VERSION_3_1,
> -  };
> -
> -  explicit SpdySessionAccept(unsigned vers);
> +  explicit SpdySessionAccept(spdy::SessionVersion vers);
>   ~SpdySessionAccept() {}
> 
>   void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
> @@ -49,7 +44,7 @@ private:
>   SpdySessionAccept(const SpdySessionAccept &); // disabled
>   SpdySessionAccept& operator =(const SpdySessionAccept&); // disabled
> 
> -  unsigned version;
> +  spdy::SessionVersion version;
> };
> 
> #endif /* SpdySessionAccept_H_ */
> 


Re: git commit: TS-2838 Add log fields for plugins on HttpSM and use for SPDY.

Posted by James Peach <jp...@apache.org>.
Please remove the documentation for the 'cps' log field.
Please document the 'piid' log field.
I don't see how this is wired up to SPDY, could you spell that out for me?

On May 23, 2014, at 9:30 AM, amc@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master b6383928a -> aadb75226
> 
> 
> TS-2838 Add log fields for plugins on HttpSM and use for SPDY.
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/aadb7522
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/aadb7522
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/aadb7522
> 
> Branch: refs/heads/master
> Commit: aadb752267ba194a3a6f584c0874cda36e6ab859
> Parents: b638392
> Author: Alan M. Carroll <am...@network-geographics.com>
> Authored: Fri May 23 09:30:03 2014 -0700
> Committer: Alan M. Carroll <am...@network-geographics.com>
> Committed: Fri May 23 09:30:03 2014 -0700
> 
> ----------------------------------------------------------------------
> CHANGES                           |  2 ++
> proxy/FetchSM.cc                  | 17 ++++++-------
> proxy/FetchSM.h                   |  4 ++-
> proxy/InkAPI.cc                   |  9 ++++++-
> proxy/Plugin.h                    | 29 ++++++++++++++++++++++
> proxy/PluginVC.cc                 | 12 ++++++++-
> proxy/PluginVC.h                  | 23 ++++++++++++++++-
> proxy/api/ts/ts.h                 | 14 +++++++----
> proxy/http/HttpClientSession.cc   |  8 ++++++
> proxy/http/HttpProxyServerMain.cc |  9 +++----
> proxy/http/HttpSM.cc              |  1 +
> proxy/http/HttpSM.h               |  5 ++++
> proxy/logging/Log.cc              | 30 +++++++++--------------
> proxy/logging/LogAccess.cc        | 45 ++++++----------------------------
> proxy/logging/LogAccess.h         |  3 ++-
> proxy/logging/LogAccessHttp.cc    | 23 ++++++++++++-----
> proxy/logging/LogAccessHttp.h     |  3 ++-
> proxy/spdy/SpdyClientSession.cc   | 36 ++++++++++++++++++++++++---
> proxy/spdy/SpdyClientSession.h    | 12 ++++++---
> proxy/spdy/SpdySessionAccept.cc   | 15 +++---------
> proxy/spdy/SpdySessionAccept.h    | 11 +++------
> 21 files changed, 198 insertions(+), 113 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/CHANGES
> ----------------------------------------------------------------------
> diff --git a/CHANGES b/CHANGES
> index 4f8d8af..fa34b15 100644
> --- a/CHANGES
> +++ b/CHANGES
> @@ -1,6 +1,8 @@
>                                                          -*- coding: utf-8 -*-
> Changes with Apache Traffic Server 5.0.0
> 
> +  *) [TS-2838] Add logging fields for plugins to TS connect API. Use for SPDY.
> +
>   *) [TS-2833] Remove REC_BUILD_STAND_ALONE, REC_BUILD_MGMT and
>    REC_BUILD_STUB. Dead code.
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.cc b/proxy/FetchSM.cc
> index c119aca..a0a4033 100644
> --- a/proxy/FetchSM.cc
> +++ b/proxy/FetchSM.cc
> @@ -61,23 +61,22 @@ FetchSM::cleanUp()
>   client_response_hdr.destroy();
>   ats_free(client_response);
>   cont_mutex.clear();
> -
> -  PluginVC *vc = (PluginVC *) http_vc;
> -
> -  vc->do_io_close();
> +  http_vc->do_io_close();
>   FetchSMAllocator.free(this);
> }
> 
> void
> FetchSM::httpConnect()
> {
> -  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> -  http_vc = TSHttpConnect(&_addr.sa);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(contp);
> +  char const* tag = pi ? pi->getPluginTag() : "fetchSM";
> +  int64_t id = pi ? pi->getPluginId() : 0;
> 
> -  PluginVC *vc = (PluginVC *) http_vc;
> +  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> +  http_vc = reinterpret_cast<PluginVC*>(TSHttpConnectWithPluginId(&_addr.sa, tag, id));
> 
> -  read_vio = vc->do_io_read(this, INT64_MAX, resp_buffer);
> -  write_vio = vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> +  read_vio = http_vc->do_io_read(this, INT64_MAX, resp_buffer);
> +  write_vio = http_vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> }
> 
> char* FetchSM::resp_get(int *length) {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.h b/proxy/FetchSM.h
> index 0de5d96..c1b94bf 100644
> --- a/proxy/FetchSM.h
> +++ b/proxy/FetchSM.h
> @@ -35,6 +35,8 @@
> #include "HttpSM.h"
> #include "HttpTunnel.h"
> 
> +class PluginVC;
> +
> class FetchSM: public Continuation
> {
> public:
> @@ -139,7 +141,7 @@ private:
>   int dechunk_body();
> 
>   int recursion;
> -  TSVConn http_vc;
> +  PluginVC* http_vc;
>   VIO *read_vio;
>   VIO *write_vio;
>   MIOBuffer *req_buffer;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/InkAPI.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
> index 0182ebe..dceea6e 100644
> --- a/proxy/InkAPI.cc
> +++ b/proxy/InkAPI.cc
> @@ -6047,7 +6047,7 @@ extern HttpSessionAccept *plugin_http_accept;
> extern HttpSessionAccept *plugin_http_transparent_accept;
> 
> TSVConn
> -TSHttpConnect(sockaddr const* addr)
> +TSHttpConnectWithPluginId(sockaddr const* addr, char const* tag, int64_t id)
> {
>   sdk_assert(addr);
> 
> @@ -6058,6 +6058,8 @@ TSHttpConnect(sockaddr const* addr)
>     PluginVCCore *new_pvc = PluginVCCore::alloc();
> 
>     new_pvc->set_active_addr(addr);
> +    new_pvc->set_plugin_id(id);
> +    new_pvc->set_plugin_tag(tag);
>     new_pvc->set_accept_cont(plugin_http_accept);
> 
>     PluginVC *return_vc = new_pvc->connect();
> @@ -6076,6 +6078,11 @@ TSHttpConnect(sockaddr const* addr)
>   return NULL;
> }
> 
> +TSVConn
> +TSHttpConnect(sockaddr const* addr)
> +{
> +  return TSHttpConnectWithPluginId(addr, "plugin", 0);
> +}
> 
> TSVConn
> TSHttpConnectTransparent(sockaddr const* client_addr, sockaddr const* server_addr)
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/Plugin.h
> ----------------------------------------------------------------------
> diff --git a/proxy/Plugin.h b/proxy/Plugin.h
> index 4c1ad09..8f70f1a 100644
> --- a/proxy/Plugin.h
> +++ b/proxy/Plugin.h
> @@ -58,4 +58,33 @@ extern PluginRegInfo *plugin_reg_current;
> 
> void plugin_init(void);
> 
> +/** Abstract interface class for plugin based continuations.
> +
> +    The primary intended use of this is for logging so that continuations
> +    that generate logging messages can generate plugin local data in a
> +    generic way.
> +
> +    The core will at appropriate times dynamically cast the continuation
> +    to this class and if successful access the plugin data via these
> +    methods.
> +
> +    Plugins should mix this in to continuations for which it is useful.
> +    The default implementations return empty / invalid responses and should
> +    be overridden by the plugin.
> + */
> +class PluginIdentity
> +{
> + public:
> +  /** Get the plugin tag.
> +      The returned string must have a lifetime at least as long as the plugin.
> +      @return A string identifying the plugin or @c NULL.
> +  */
> +  virtual char const* getPluginTag() const { return NULL; }
> +  /** Get the plugin instance ID.
> +      A plugin can create multiple subsidiary instances. This is used as the
> +      identifier for those to distinguish the instances.
> +  */
> +  virtual int64_t getPluginId() const { return 0; }
> +};
> +
> #endif /* __PLUGIN_H__ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.cc b/proxy/PluginVC.cc
> index 2e4cfa1..16eb5a2 100644
> --- a/proxy/PluginVC.cc
> +++ b/proxy/PluginVC.cc
> @@ -1051,7 +1051,6 @@ void
> PluginVCCore::set_accept_cont(Continuation * c)
> {
>   connect_to = c;
> -
>   // FIX ME - must return action
> }
> 
> @@ -1202,6 +1201,17 @@ PluginVCCore::set_transparent(bool passive_side, bool active_side)
>   active_vc.set_is_transparent(active_side);
> }
> 
> +void
> +PluginVCCore::set_plugin_id(int64_t id)
> +{
> +  passive_vc.plugin_id = active_vc.plugin_id = id;
> +}
> +
> +void
> +PluginVCCore::set_plugin_tag(char const* tag)
> +{
> +  passive_vc.plugin_tag = active_vc.plugin_tag = tag;
> +}
> 
> /*************************************************************
>  *
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.h
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.h b/proxy/PluginVC.h
> index d770de5..6ee3a93 100644
> --- a/proxy/PluginVC.h
> +++ b/proxy/PluginVC.h
> @@ -36,6 +36,7 @@
> #ifndef _PLUGIN_VC_H_
> #define _PLUGIN_VC_H_
> 
> +#include "Plugin.h"
> #include "P_Net.h"
> #include "ink_atomic.h"
> 
> @@ -75,7 +76,7 @@ enum
>   PLUGIN_VC_MAGIC_DEAD = 0xaabbdead
> };
> 
> -class PluginVC:public NetVConnection
> +class PluginVC:public NetVConnection, public PluginIdentity
> {
>   friend class PluginVCCore;
> public:
> @@ -114,6 +115,18 @@ public:
> 
>   virtual PluginVC* get_other_side() { return other_side; }
> 
> +  //@{ @name Plugin identity.
> +  /// Override for @c PluginIdentity.
> +  virtual char const* getPluginTag() const { return plugin_tag; }
> +  /// Override for @c PluginIdentity.
> +  virtual int64_t getPluginId() const { return plugin_id; }
> +
> +  /// Setter for plugin tag.
> +  virtual void setPluginTag(char const* tag) { plugin_tag = tag; }
> +  /// Setter for plugin id.
> +  virtual void setPluginId(int64_t id) { plugin_id = id; }
> +  //@}
> +
>   int main_handler(int event, void *data);
> 
> private:
> @@ -152,6 +165,9 @@ private:
>   ink_hrtime inactive_timeout;
>   ink_hrtime inactive_timeout_at;
>   Event *inactive_event;
> +
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> };
> 
> class PluginVCCore:public Continuation
> @@ -198,6 +214,11 @@ public:
> 
>   void set_transparent(bool passive_side, bool active_side);
> 
> +  /// Set the plugin ID for the internal VCs.
> +  void set_plugin_id(int64_t id);
> +  /// Set the plugin tag for the internal VCs.
> +  void set_plugin_tag(char const* tag);
> +
>   // The active vc is handed to the initiator of
>   //   connection.  The passive vc is handled to
>   //   receiver of the connection
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/api/ts/ts.h
> ----------------------------------------------------------------------
> diff --git a/proxy/api/ts/ts.h b/proxy/api/ts/ts.h
> index 27d0546..917d6b2 100644
> --- a/proxy/api/ts/ts.h
> +++ b/proxy/api/ts/ts.h
> @@ -1588,12 +1588,16 @@ extern "C"
>       than TSNetConnect() to localhost since it avoids the overhead of
>       passing the data through the operating system.
> 
> -      @param log_ip ip address (in network byte order) that connection
> -        will be logged as coming from.
> -      @param log_port port (in network byte order) that connection will
> -        be logged as coming from.
> -      @param vc will be set to point to the new TSVConn on success.
> +      This returns a VConn that connected to the transaction.
> +
> +      @param addr Target address of the origin server.
> +      @param tag A logging tag that can be accessed via the pitag field. May be @c NULL.
> +      @param id A logging id that can be access via the piid field.
> +   */
> +  tsapi TSVConn TSHttpConnectWithPluginId(struct sockaddr const* addr, char const* tag, int64_t id);
> 
> +  /** Backwards compatible version.
> +      This provides a @a tag of "plugin" and an @a id of 0.
>    */
>   tsapi TSVConn TSHttpConnect(struct sockaddr const* addr);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
> index d2246c8..0620b3b 100644
> --- a/proxy/http/HttpClientSession.cc
> +++ b/proxy/http/HttpClientSession.cc
> @@ -36,6 +36,7 @@
> #include "HttpSM.h"
> #include "HttpDebugNames.h"
> #include "HttpServerSession.h"
> +#include "Plugin.h"
> 
> #define DebugSsn(tag, ...) DebugSpecific(debug_on, tag, __VA_ARGS__)
> #define STATE_ENTER(state_name, event, vio) { \
> @@ -137,6 +138,7 @@ void
> HttpClientSession::new_transaction()
> {
>   ink_assert(current_reader == NULL);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(client_vc);
> 
>   read_state = HCS_ACTIVE_READER;
>   current_reader = HttpSM::allocate();
> @@ -145,6 +147,12 @@ HttpClientSession::new_transaction()
>   DebugSsn("http_cs", "[%" PRId64 "] Starting transaction %d using sm [%" PRId64 "]", con_id, transact_count, current_reader->sm_id);
> 
>   current_reader->attach_client_session(this, sm_reader);
> +  if (pi) {
> +    // it's a plugin VC of some sort with identify information.
> +    // copy it to the SM.
> +    current_reader->plugin_tag = pi->getPluginTag();
> +    current_reader->plugin_id = pi->getPluginId();
> +  }
> }
> 
> inline void
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpProxyServerMain.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
> index 2bf7d64..9eb9291 100644
> --- a/proxy/http/HttpProxyServerMain.cc
> +++ b/proxy/http/HttpProxyServerMain.cc
> @@ -180,8 +180,7 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
> 
> #if TS_HAS_SPDY
>   if (port.m_session_protocol_preference.intersects(SPDY_PROTOCOL_SET)) {
> -    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY,
> -                            new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>   }
> #endif
> 
> @@ -207,11 +206,11 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
>     // SPDY
> #if TS_HAS_SPDY
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(spdy::SESSION_VERSION_3));
>     }
> 
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3_1)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>     }
> #endif
> 
> @@ -316,7 +315,7 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
>   opt.localhost_only = true;
>   ha_opt.backdoor = true;
>   opt.backdoor = true;
> -  
> +
>   // The backdoor only binds the loopback interface
>   netProcessor.main_accept(new HttpSessionAccept(ha_opt), NO_FD, opt);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
> index 2eabc1d..02afe4a 100644
> --- a/proxy/http/HttpSM.cc
> +++ b/proxy/http/HttpSM.cc
> @@ -325,6 +325,7 @@ HttpSM::HttpSM()
>     client_response_hdr_bytes(0), client_response_body_bytes(0),
>     cache_response_hdr_bytes(0), cache_response_body_bytes(0),
>     pushed_response_hdr_bytes(0), pushed_response_body_bytes(0),
> +    plugin_tag(0), plugin_id(0),
>     hooks_set(0), cur_hook_id(TS_HTTP_LAST_HOOK), cur_hook(NULL),
>     cur_hooks(0), callout_state(HTTP_API_NO_CALLOUT), terminate_sm(false), kill_this_async_done(false)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
> index 2b5a6fd..13121ec 100644
> --- a/proxy/http/HttpSM.h
> +++ b/proxy/http/HttpSM.h
> @@ -488,6 +488,11 @@ public:
>   int pushed_response_hdr_bytes;
>   int64_t pushed_response_body_bytes;
>   TransactionMilestones milestones;
> +  // The next two enable plugins to tag the state machine for
> +  // the purposes of logging so the instances can be correlated
> +  // with the source plugin.
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> 
>   // hooks_set records whether there are any hooks relevant
>   //  to this transaction.  Used to avoid costly calls
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/Log.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/Log.cc b/proxy/logging/Log.cc
> index 16138b4..9ee618e 100644
> --- a/proxy/logging/Log.cc
> +++ b/proxy/logging/Log.cc
> @@ -365,27 +365,19 @@ Log::init_fields()
>   global_field_list.add (field, false);
>   ink_hash_table_insert (field_symbol_hash, "caun", field);
> 
> -# if 0
> -  Ptr<LogFieldAliasTable> proto_type_map = make_ptr(new LogFieldAliasTable);
> -  proto_type_map->init(7,
> -                       // Transport protocols
> -                       TS_PROTO_UDP, "UDP",
> -                       TS_PROTO_TCP, "TCP",
> -                       TS_PROTO_TLS, "TLS",
> -                       // Application protocols
> -                       TS_PROTO_HTTP, "HTTP",
> -                       TS_PROTO_SPDY, "SPDY",
> -                       TS_PROTO_RTMP, "RTMP",
> -                       TS_PROTO_WS,   "WS");
> -
> -  field = new LogField("client_protocol_stack", "cps",
> +  field = new LogField("plugin_identity_id", "piid",
>                        LogField::sINT,
> -                       &LogAccess::marshal_client_protocol_stack,
> -                       &LogAccess::unmarshal_client_protocol_stack,
> -                       (Ptr<LogFieldAliasMap>) proto_type_map);
> +                       &LogAccess::marshal_plugin_identity_id,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_int_to_str));
>   global_field_list.add(field, false);
> -  ink_hash_table_insert(field_symbol_hash, "cps", field);
> -# endif
> +  ink_hash_table_insert(field_symbol_hash, "piid", field);
> +
> +  field = new LogField("plugin_identity_tag", "pitag",
> +                       LogField::STRING,
> +                       &LogAccess::marshal_plugin_identity_tag,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_str));
> +  global_field_list.add(field, false);
> +  ink_hash_table_insert(field_symbol_hash, "pitag", field);
> 
>   field = new LogField("client_req_timestamp_sec", "cqts",
>                        LogField::sINT,
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.cc b/proxy/logging/LogAccess.cc
> index 968762e..5009113 100644
> --- a/proxy/logging/LogAccess.cc
> +++ b/proxy/logging/LogAccess.cc
> @@ -75,12 +75,18 @@ LogAccess::init()
>   -------------------------------------------------------------------------*/
> 
> int
> -LogAccess::marshal_client_protocol_stack(char *buf)
> +LogAccess::marshal_plugin_identity_id(char *buf)
> {
>   DEFAULT_INT_FIELD;
> }
> 
> int
> +LogAccess::marshal_plugin_identity_tag(char *buf)
> +{
> +  DEFAULT_STR_FIELD;
> +}
> +
> +int
> LogAccess::marshal_client_host_ip(char *buf)
> {
>   DEFAULT_IP_FIELD;
> @@ -1311,43 +1317,6 @@ LogAccess::unmarshal_cache_write_code(char **buf, char *dest, int len, Ptr<LogFi
>   return (LogAccess::unmarshal_with_map(unmarshal_int(buf), dest, len, map, "UNKNOWN_CACHE_WRITE_CODE"));
> }
> 
> -# if 0
> -int
> -LogAccess::unmarshal_client_protocol_stack(char **buf, char *dest, int len, Ptr<LogFieldAliasMap> map)
> -{
> -  ink_assert(buf != NULL);
> -  ink_assert(*buf != NULL);
> -  ink_assert(dest != NULL);
> -
> -  char *p;
> -  size_t nr_chars = 0;
> -  int i, ret, nr_bits, left_len;
> -  TSClientProtoStack proto_stack = (TSClientProtoStack)unmarshal_int(buf);
> -
> -  p = dest;
> -  left_len = len;
> -  nr_bits = 8 * sizeof(TSClientProtoStack);
> -
> -  for (i = 0; i < nr_bits && left_len; i++) {
> -    if ((proto_stack >> i) & 0x1) {
> -      if (p != dest) {
> -        *p++ = '+';
> -        left_len--;
> -      }
> -      ret = map->asString(i, p, left_len, &nr_chars);
> -      if (ret == LogFieldAliasMap::ALL_OK) {
> -        p += nr_chars;
> -        left_len -= nr_chars;
> -      } else if (ret == LogFieldAliasMap::BUFFER_TOO_SMALL) {
> -        break;
> -      }
> -    }
> -  }
> -
> -  return (len - left_len);
> -}
> -# endif
> -
> int
> LogAccess::unmarshal_record(char **buf, char *dest, int len)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.h b/proxy/logging/LogAccess.h
> index 940145d..8fb35b5 100644
> --- a/proxy/logging/LogAccess.h
> +++ b/proxy/logging/LogAccess.h
> @@ -172,7 +172,6 @@ public:
>   inkcoreapi virtual int marshal_client_auth_user_name(char *); // STR
>   int marshal_client_req_timestamp_sec(char *); // INT
> 
> -  inkcoreapi virtual int marshal_client_protocol_stack(char *); // INT
>   inkcoreapi virtual int marshal_client_req_text(char *);       // STR
>   inkcoreapi virtual int marshal_client_req_http_method(char *);        // STR
>   inkcoreapi virtual int marshal_client_req_url(char *);        // STR
> @@ -251,6 +250,8 @@ public:
>   inkcoreapi virtual int marshal_transfer_time_ms(char *);      // INT
>   inkcoreapi virtual int marshal_transfer_time_s(char *);       // INT
>   inkcoreapi virtual int marshal_file_size(char *);     // INT
> +  inkcoreapi virtual int marshal_plugin_identity_id(char *); // INT
> +  inkcoreapi virtual int marshal_plugin_identity_tag(char *); // STR
>   int marshal_entry_type(char *);       // INT
> 
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.cc b/proxy/logging/LogAccessHttp.cc
> index a8dad70..b44e684 100644
> --- a/proxy/logging/LogAccessHttp.cc
> +++ b/proxy/logging/LogAccessHttp.cc
> @@ -206,16 +206,27 @@ LogAccessHttp::set_client_req_url_path(char *buf, int len)
> 
> /*-------------------------------------------------------------------------
>   -------------------------------------------------------------------------*/
> -# if 0
> int
> -LogAccessHttp::marshal_client_protocol_stack(char *buf)
> +LogAccessHttp::marshal_plugin_identity_id(char *buf)
> {
> -  if (buf) {
> -    marshal_int(buf, m_http_sm->proto_stack);
> -  }
> +  if (buf) marshal_int(buf, m_http_sm->plugin_id);
>   return INK_MIN_ALIGN;
> }
> -# endif
> +
> +int
> +LogAccessHttp::marshal_plugin_identity_tag(char *buf)
> +{
> +  int len = INK_MIN_ALIGN;
> +  char const* tag = m_http_sm->plugin_tag;
> +
> +  if (!tag) tag = "*";
> +  else len = LogAccess::strlen(tag);
> +
> +  if (buf) marshal_str(buf, tag, len);
> +
> +  return len;
> +}
> +
> int
> LogAccessHttp::marshal_client_host_ip(char *buf)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.h b/proxy/logging/LogAccessHttp.h
> index 53a0ff5..51ee9e3 100644
> --- a/proxy/logging/LogAccessHttp.h
> +++ b/proxy/logging/LogAccessHttp.h
> @@ -58,7 +58,6 @@ public:
>   virtual int marshal_client_host_ip(char *);   // STR
>   virtual int marshal_client_host_port(char *); // INT
>   virtual int marshal_client_auth_user_name(char *);    // STR
> -  //  virtual int marshal_client_protocol_stack(char *);    // INT
>   virtual int marshal_client_req_text(char *);  // STR
>   virtual int marshal_client_req_http_method(char *);   // INT
>   virtual int marshal_client_req_url(char *);   // STR
> @@ -128,6 +127,8 @@ public:
>   virtual int marshal_transfer_time_ms(char *); // INT
>   virtual int marshal_transfer_time_s(char *);  // INT
>   virtual int marshal_file_size(char *); // INT
> +  virtual int marshal_plugin_identity_id(char *);    // INT
> +  virtual int marshal_plugin_identity_tag(char *);    // STR
> 
>   //
>   // named fields from within a http header
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.cc b/proxy/spdy/SpdyClientSession.cc
> index 2c77ff6..fcb79fb 100644
> --- a/proxy/spdy/SpdyClientSession.cc
> +++ b/proxy/spdy/SpdyClientSession.cc
> @@ -27,6 +27,22 @@
> static ClassAllocator<SpdyClientSession> spdyClientSessionAllocator("spdyClientSessionAllocator");
> ClassAllocator<SpdyRequest> spdyRequestAllocator("spdyRequestAllocator");
> 
> +#if TS_HAS_SPDY
> +#include "SpdyClientSession.h"
> +
> +static const spdylay_proto_version versmap[] = {
> +  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> +  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> +  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> +};
> +
> +static char const* const  npnmap[] = {
> +  TS_NPN_PROTOCOL_SPDY_2,
> +  TS_NPN_PROTOCOL_SPDY_3,
> +  TS_NPN_PROTOCOL_SPDY_3_1
> +};
> +
> +#endif
> static int spdy_process_read(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_write(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_fetch(TSEvent event, SpdyClientSession *sm, void *edata);
> @@ -69,15 +85,16 @@ SpdyRequest::clear()
> }
> 
> void
> -SpdyClientSession::init(NetVConnection * netvc, spdylay_proto_version vers)
> +SpdyClientSession::init(NetVConnection * netvc, spdy::SessionVersion vers)
> {
>   int r;
> 
>   this->mutex = new_ProxyMutex();
>   this->vc = netvc;
>   this->req_map.clear();
> +  this->version = vers;
> 
> -  r = spdylay_session_server_new(&session, vers, &SPDY_CFG.spdy.callbacks, this);
> +  r = spdylay_session_server_new(&session, versmap[vers], &SPDY_CFG.spdy.callbacks, this);
> 
>   // A bit ugly but we need a thread and I don't want to wait until the
>   // session start event in case of a time out generating a decrement
> @@ -157,7 +174,7 @@ SpdyClientSession::clear()
> }
> 
> void
> -spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader)
> +spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader)
> {
>   SpdyClientSession  *sm;
> 
> @@ -242,6 +259,19 @@ out:
>   return EVENT_CONT;
> }
> 
> +int64_t
> +SpdyClientSession::getPluginId() const
> +{
> +  return sm_id;
> +}
> +
> +char const*
> +SpdyClientSession::getPluginTag() const
> +{
> +  return npnmap[this->version];
> +}
> +
> +
> static int
> spdy_process_read(TSEvent /* event ATS_UNUSED */, SpdyClientSession *sm)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.h b/proxy/spdy/SpdyClientSession.h
> index 2f37d35..738ff55 100644
> --- a/proxy/spdy/SpdyClientSession.h
> +++ b/proxy/spdy/SpdyClientSession.h
> @@ -24,9 +24,11 @@
> #ifndef __P_SPDY_SM_H__
> #define __P_SPDY_SM_H__
> 
> +#include "SpdyDefs.h"
> #include "SpdyCommon.h"
> #include "SpdyCallbacks.h"
> #include <openssl/md5.h>
> +#include "Plugin.h"
> 
> class SpdyClientSession;
> typedef int (*SpdyClientSessionHandler) (TSCont contp, TSEvent event, void *data);
> @@ -89,7 +91,7 @@ public:
>   MD5_CTX recv_md5;
> };
> 
> -class SpdyClientSession : public Continuation
> +class SpdyClientSession : public Continuation, public PluginIdentity
> {
> 
> public:
> @@ -101,10 +103,11 @@ public:
>     clear();
>   }
> 
> -  void init(NetVConnection * netvc, spdylay_proto_version vers);
> +  void init(NetVConnection * netvc, spdy::SessionVersion vers);
>   void clear();
> 
>   int64_t sm_id;
> +  spdy::SessionVersion version;
>   uint64_t total_size;
>   TSHRTime start_time;
> 
> @@ -124,12 +127,15 @@ public:
> 
>   map<int32_t, SpdyRequest*> req_map;
> 
> +  virtual char const* getPluginTag() const;
> +  virtual int64_t getPluginId() const;
> +
> private:
>   int state_session_start(int event, void * edata);
>   int state_session_readwrite(int event, void * edata);
> };
> 
> -void spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader);
> +void spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader);
> 
> extern ClassAllocator<SpdyRequest> spdyRequestAllocator;
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
> index e679adc..d7dfa11 100644
> --- a/proxy/spdy/SpdySessionAccept.cc
> +++ b/proxy/spdy/SpdySessionAccept.cc
> @@ -26,20 +26,13 @@
> 
> #if TS_HAS_SPDY
> #include "SpdyClientSession.h"
> -
> -static const spdylay_proto_version versmap[] = {
> -  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> -  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> -  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> -};
> -
> #endif
> 
> -SpdySessionAccept::SpdySessionAccept(unsigned vers)
> +SpdySessionAccept::SpdySessionAccept(spdy::SessionVersion vers)
>     : SessionAccept(new_ProxyMutex()), version(vers)
> {
> #if TS_HAS_SPDY
> -  ink_release_assert(vers < countof(versmap));
> +  ink_release_assert(spdy::SESSION_VERSION_2 <= vers && vers <= spdy::SESSION_VERSION_3_1);
> #endif
>   SET_HANDLER(&SpdySessionAccept::mainEvent);
> }
> @@ -51,7 +44,7 @@ SpdySessionAccept::mainEvent(int event, void * edata)
>     NetVConnection * netvc =static_cast<NetVConnection *>(edata);
> 
> #if TS_HAS_SPDY
> -    spdy_sm_create(netvc, versmap[this->version], NULL, NULL);
> +    spdy_sm_create(netvc, this->version, NULL, NULL);
> #else
>     Error("accepted a SPDY session, but SPDY support is not available");
>     netvc->do_io_close();
> @@ -68,7 +61,7 @@ void
> SpdySessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
> {
> #if TS_HAS_SPDY
> -  spdy_sm_create(netvc, versmap[this->version], iobuf, reader);
> +  spdy_sm_create(netvc, this->version, iobuf, reader);
> #else
>   (void)netvc;
>   (void)iobuf;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.h b/proxy/spdy/SpdySessionAccept.h
> index 9d740e2..af3d81d 100644
> --- a/proxy/spdy/SpdySessionAccept.h
> +++ b/proxy/spdy/SpdySessionAccept.h
> @@ -28,18 +28,13 @@
> #include "P_EventSystem.h"
> #include "P_UnixNet.h"
> #include "I_IOBuffer.h"
> +#include "SpdyDefs.h"
> 
> class SpdySessionAccept: public SessionAccept
> {
> public:
> 
> -  enum {
> -    SPDY_VERSION_2 = 0,
> -    SPDY_VERSION_3,
> -    SPDY_VERSION_3_1,
> -  };
> -
> -  explicit SpdySessionAccept(unsigned vers);
> +  explicit SpdySessionAccept(spdy::SessionVersion vers);
>   ~SpdySessionAccept() {}
> 
>   void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
> @@ -49,7 +44,7 @@ private:
>   SpdySessionAccept(const SpdySessionAccept &); // disabled
>   SpdySessionAccept& operator =(const SpdySessionAccept&); // disabled
> 
> -  unsigned version;
> +  spdy::SessionVersion version;
> };
> 
> #endif /* SpdySessionAccept_H_ */
> 


Re: git commit: TS-2838 Add log fields for plugins on HttpSM and use for SPDY.

Posted by James Peach <jp...@apache.org>.
This should have gone through API review. Can you please post a review email?

On May 23, 2014, at 9:30 AM, amc@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master b6383928a -> aadb75226
> 
> 
> TS-2838 Add log fields for plugins on HttpSM and use for SPDY.
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/aadb7522
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/aadb7522
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/aadb7522
> 
> Branch: refs/heads/master
> Commit: aadb752267ba194a3a6f584c0874cda36e6ab859
> Parents: b638392
> Author: Alan M. Carroll <am...@network-geographics.com>
> Authored: Fri May 23 09:30:03 2014 -0700
> Committer: Alan M. Carroll <am...@network-geographics.com>
> Committed: Fri May 23 09:30:03 2014 -0700
> 
> ----------------------------------------------------------------------
> CHANGES                           |  2 ++
> proxy/FetchSM.cc                  | 17 ++++++-------
> proxy/FetchSM.h                   |  4 ++-
> proxy/InkAPI.cc                   |  9 ++++++-
> proxy/Plugin.h                    | 29 ++++++++++++++++++++++
> proxy/PluginVC.cc                 | 12 ++++++++-
> proxy/PluginVC.h                  | 23 ++++++++++++++++-
> proxy/api/ts/ts.h                 | 14 +++++++----
> proxy/http/HttpClientSession.cc   |  8 ++++++
> proxy/http/HttpProxyServerMain.cc |  9 +++----
> proxy/http/HttpSM.cc              |  1 +
> proxy/http/HttpSM.h               |  5 ++++
> proxy/logging/Log.cc              | 30 +++++++++--------------
> proxy/logging/LogAccess.cc        | 45 ++++++----------------------------
> proxy/logging/LogAccess.h         |  3 ++-
> proxy/logging/LogAccessHttp.cc    | 23 ++++++++++++-----
> proxy/logging/LogAccessHttp.h     |  3 ++-
> proxy/spdy/SpdyClientSession.cc   | 36 ++++++++++++++++++++++++---
> proxy/spdy/SpdyClientSession.h    | 12 ++++++---
> proxy/spdy/SpdySessionAccept.cc   | 15 +++---------
> proxy/spdy/SpdySessionAccept.h    | 11 +++------
> 21 files changed, 198 insertions(+), 113 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/CHANGES
> ----------------------------------------------------------------------
> diff --git a/CHANGES b/CHANGES
> index 4f8d8af..fa34b15 100644
> --- a/CHANGES
> +++ b/CHANGES
> @@ -1,6 +1,8 @@
>                                                          -*- coding: utf-8 -*-
> Changes with Apache Traffic Server 5.0.0
> 
> +  *) [TS-2838] Add logging fields for plugins to TS connect API. Use for SPDY.
> +
>   *) [TS-2833] Remove REC_BUILD_STAND_ALONE, REC_BUILD_MGMT and
>    REC_BUILD_STUB. Dead code.
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.cc b/proxy/FetchSM.cc
> index c119aca..a0a4033 100644
> --- a/proxy/FetchSM.cc
> +++ b/proxy/FetchSM.cc
> @@ -61,23 +61,22 @@ FetchSM::cleanUp()
>   client_response_hdr.destroy();
>   ats_free(client_response);
>   cont_mutex.clear();
> -
> -  PluginVC *vc = (PluginVC *) http_vc;
> -
> -  vc->do_io_close();
> +  http_vc->do_io_close();
>   FetchSMAllocator.free(this);
> }
> 
> void
> FetchSM::httpConnect()
> {
> -  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> -  http_vc = TSHttpConnect(&_addr.sa);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(contp);
> +  char const* tag = pi ? pi->getPluginTag() : "fetchSM";
> +  int64_t id = pi ? pi->getPluginId() : 0;
> 
> -  PluginVC *vc = (PluginVC *) http_vc;
> +  Debug(DEBUG_TAG, "[%s] calling httpconnect write", __FUNCTION__);
> +  http_vc = reinterpret_cast<PluginVC*>(TSHttpConnectWithPluginId(&_addr.sa, tag, id));
> 
> -  read_vio = vc->do_io_read(this, INT64_MAX, resp_buffer);
> -  write_vio = vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> +  read_vio = http_vc->do_io_read(this, INT64_MAX, resp_buffer);
> +  write_vio = http_vc->do_io_write(this, getReqLen() + req_content_length, req_reader);
> }
> 
> char* FetchSM::resp_get(int *length) {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/FetchSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/FetchSM.h b/proxy/FetchSM.h
> index 0de5d96..c1b94bf 100644
> --- a/proxy/FetchSM.h
> +++ b/proxy/FetchSM.h
> @@ -35,6 +35,8 @@
> #include "HttpSM.h"
> #include "HttpTunnel.h"
> 
> +class PluginVC;
> +
> class FetchSM: public Continuation
> {
> public:
> @@ -139,7 +141,7 @@ private:
>   int dechunk_body();
> 
>   int recursion;
> -  TSVConn http_vc;
> +  PluginVC* http_vc;
>   VIO *read_vio;
>   VIO *write_vio;
>   MIOBuffer *req_buffer;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/InkAPI.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
> index 0182ebe..dceea6e 100644
> --- a/proxy/InkAPI.cc
> +++ b/proxy/InkAPI.cc
> @@ -6047,7 +6047,7 @@ extern HttpSessionAccept *plugin_http_accept;
> extern HttpSessionAccept *plugin_http_transparent_accept;
> 
> TSVConn
> -TSHttpConnect(sockaddr const* addr)
> +TSHttpConnectWithPluginId(sockaddr const* addr, char const* tag, int64_t id)
> {
>   sdk_assert(addr);
> 
> @@ -6058,6 +6058,8 @@ TSHttpConnect(sockaddr const* addr)
>     PluginVCCore *new_pvc = PluginVCCore::alloc();
> 
>     new_pvc->set_active_addr(addr);
> +    new_pvc->set_plugin_id(id);
> +    new_pvc->set_plugin_tag(tag);
>     new_pvc->set_accept_cont(plugin_http_accept);
> 
>     PluginVC *return_vc = new_pvc->connect();
> @@ -6076,6 +6078,11 @@ TSHttpConnect(sockaddr const* addr)
>   return NULL;
> }
> 
> +TSVConn
> +TSHttpConnect(sockaddr const* addr)
> +{
> +  return TSHttpConnectWithPluginId(addr, "plugin", 0);
> +}
> 
> TSVConn
> TSHttpConnectTransparent(sockaddr const* client_addr, sockaddr const* server_addr)
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/Plugin.h
> ----------------------------------------------------------------------
> diff --git a/proxy/Plugin.h b/proxy/Plugin.h
> index 4c1ad09..8f70f1a 100644
> --- a/proxy/Plugin.h
> +++ b/proxy/Plugin.h
> @@ -58,4 +58,33 @@ extern PluginRegInfo *plugin_reg_current;
> 
> void plugin_init(void);
> 
> +/** Abstract interface class for plugin based continuations.
> +
> +    The primary intended use of this is for logging so that continuations
> +    that generate logging messages can generate plugin local data in a
> +    generic way.
> +
> +    The core will at appropriate times dynamically cast the continuation
> +    to this class and if successful access the plugin data via these
> +    methods.
> +
> +    Plugins should mix this in to continuations for which it is useful.
> +    The default implementations return empty / invalid responses and should
> +    be overridden by the plugin.
> + */
> +class PluginIdentity
> +{
> + public:
> +  /** Get the plugin tag.
> +      The returned string must have a lifetime at least as long as the plugin.
> +      @return A string identifying the plugin or @c NULL.
> +  */
> +  virtual char const* getPluginTag() const { return NULL; }
> +  /** Get the plugin instance ID.
> +      A plugin can create multiple subsidiary instances. This is used as the
> +      identifier for those to distinguish the instances.
> +  */
> +  virtual int64_t getPluginId() const { return 0; }
> +};
> +
> #endif /* __PLUGIN_H__ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.cc b/proxy/PluginVC.cc
> index 2e4cfa1..16eb5a2 100644
> --- a/proxy/PluginVC.cc
> +++ b/proxy/PluginVC.cc
> @@ -1051,7 +1051,6 @@ void
> PluginVCCore::set_accept_cont(Continuation * c)
> {
>   connect_to = c;
> -
>   // FIX ME - must return action
> }
> 
> @@ -1202,6 +1201,17 @@ PluginVCCore::set_transparent(bool passive_side, bool active_side)
>   active_vc.set_is_transparent(active_side);
> }
> 
> +void
> +PluginVCCore::set_plugin_id(int64_t id)
> +{
> +  passive_vc.plugin_id = active_vc.plugin_id = id;
> +}
> +
> +void
> +PluginVCCore::set_plugin_tag(char const* tag)
> +{
> +  passive_vc.plugin_tag = active_vc.plugin_tag = tag;
> +}
> 
> /*************************************************************
>  *
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/PluginVC.h
> ----------------------------------------------------------------------
> diff --git a/proxy/PluginVC.h b/proxy/PluginVC.h
> index d770de5..6ee3a93 100644
> --- a/proxy/PluginVC.h
> +++ b/proxy/PluginVC.h
> @@ -36,6 +36,7 @@
> #ifndef _PLUGIN_VC_H_
> #define _PLUGIN_VC_H_
> 
> +#include "Plugin.h"
> #include "P_Net.h"
> #include "ink_atomic.h"
> 
> @@ -75,7 +76,7 @@ enum
>   PLUGIN_VC_MAGIC_DEAD = 0xaabbdead
> };
> 
> -class PluginVC:public NetVConnection
> +class PluginVC:public NetVConnection, public PluginIdentity
> {
>   friend class PluginVCCore;
> public:
> @@ -114,6 +115,18 @@ public:
> 
>   virtual PluginVC* get_other_side() { return other_side; }
> 
> +  //@{ @name Plugin identity.
> +  /// Override for @c PluginIdentity.
> +  virtual char const* getPluginTag() const { return plugin_tag; }
> +  /// Override for @c PluginIdentity.
> +  virtual int64_t getPluginId() const { return plugin_id; }
> +
> +  /// Setter for plugin tag.
> +  virtual void setPluginTag(char const* tag) { plugin_tag = tag; }
> +  /// Setter for plugin id.
> +  virtual void setPluginId(int64_t id) { plugin_id = id; }
> +  //@}
> +
>   int main_handler(int event, void *data);
> 
> private:
> @@ -152,6 +165,9 @@ private:
>   ink_hrtime inactive_timeout;
>   ink_hrtime inactive_timeout_at;
>   Event *inactive_event;
> +
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> };
> 
> class PluginVCCore:public Continuation
> @@ -198,6 +214,11 @@ public:
> 
>   void set_transparent(bool passive_side, bool active_side);
> 
> +  /// Set the plugin ID for the internal VCs.
> +  void set_plugin_id(int64_t id);
> +  /// Set the plugin tag for the internal VCs.
> +  void set_plugin_tag(char const* tag);
> +
>   // The active vc is handed to the initiator of
>   //   connection.  The passive vc is handled to
>   //   receiver of the connection
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/api/ts/ts.h
> ----------------------------------------------------------------------
> diff --git a/proxy/api/ts/ts.h b/proxy/api/ts/ts.h
> index 27d0546..917d6b2 100644
> --- a/proxy/api/ts/ts.h
> +++ b/proxy/api/ts/ts.h
> @@ -1588,12 +1588,16 @@ extern "C"
>       than TSNetConnect() to localhost since it avoids the overhead of
>       passing the data through the operating system.
> 
> -      @param log_ip ip address (in network byte order) that connection
> -        will be logged as coming from.
> -      @param log_port port (in network byte order) that connection will
> -        be logged as coming from.
> -      @param vc will be set to point to the new TSVConn on success.
> +      This returns a VConn that connected to the transaction.
> +
> +      @param addr Target address of the origin server.
> +      @param tag A logging tag that can be accessed via the pitag field. May be @c NULL.
> +      @param id A logging id that can be access via the piid field.
> +   */
> +  tsapi TSVConn TSHttpConnectWithPluginId(struct sockaddr const* addr, char const* tag, int64_t id);
> 
> +  /** Backwards compatible version.
> +      This provides a @a tag of "plugin" and an @a id of 0.
>    */
>   tsapi TSVConn TSHttpConnect(struct sockaddr const* addr);
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
> index d2246c8..0620b3b 100644
> --- a/proxy/http/HttpClientSession.cc
> +++ b/proxy/http/HttpClientSession.cc
> @@ -36,6 +36,7 @@
> #include "HttpSM.h"
> #include "HttpDebugNames.h"
> #include "HttpServerSession.h"
> +#include "Plugin.h"
> 
> #define DebugSsn(tag, ...) DebugSpecific(debug_on, tag, __VA_ARGS__)
> #define STATE_ENTER(state_name, event, vio) { \
> @@ -137,6 +138,7 @@ void
> HttpClientSession::new_transaction()
> {
>   ink_assert(current_reader == NULL);
> +  PluginIdentity* pi = dynamic_cast<PluginIdentity*>(client_vc);
> 
>   read_state = HCS_ACTIVE_READER;
>   current_reader = HttpSM::allocate();
> @@ -145,6 +147,12 @@ HttpClientSession::new_transaction()
>   DebugSsn("http_cs", "[%" PRId64 "] Starting transaction %d using sm [%" PRId64 "]", con_id, transact_count, current_reader->sm_id);
> 
>   current_reader->attach_client_session(this, sm_reader);
> +  if (pi) {
> +    // it's a plugin VC of some sort with identify information.
> +    // copy it to the SM.
> +    current_reader->plugin_tag = pi->getPluginTag();
> +    current_reader->plugin_id = pi->getPluginId();
> +  }
> }
> 
> inline void
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpProxyServerMain.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
> index 2bf7d64..9eb9291 100644
> --- a/proxy/http/HttpProxyServerMain.cc
> +++ b/proxy/http/HttpProxyServerMain.cc
> @@ -180,8 +180,7 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
> 
> #if TS_HAS_SPDY
>   if (port.m_session_protocol_preference.intersects(SPDY_PROTOCOL_SET)) {
> -    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY,
> -                            new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +    probe->registerEndpoint(ProtocolProbeSessionAccept::PROTO_SPDY, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>   }
> #endif
> 
> @@ -207,11 +206,11 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor& acceptor, HttpProxyPort& port, unsigned
>     // SPDY
> #if TS_HAS_SPDY
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3, new SpdySessionAccept(spdy::SESSION_VERSION_3));
>     }
> 
>     if (port.m_session_protocol_preference.contains(TS_NPN_PROTOCOL_INDEX_SPDY_3_1)) {
> -      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(SpdySessionAccept::SPDY_VERSION_3_1));
> +      ssl->registerEndpoint(TS_NPN_PROTOCOL_SPDY_3_1, new SpdySessionAccept(spdy::SESSION_VERSION_3_1));
>     }
> #endif
> 
> @@ -316,7 +315,7 @@ start_HttpProxyServerBackDoor(int port, int accept_threads)
>   opt.localhost_only = true;
>   ha_opt.backdoor = true;
>   opt.backdoor = true;
> -  
> +
>   // The backdoor only binds the loopback interface
>   netProcessor.main_accept(new HttpSessionAccept(ha_opt), NO_FD, opt);
> }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
> index 2eabc1d..02afe4a 100644
> --- a/proxy/http/HttpSM.cc
> +++ b/proxy/http/HttpSM.cc
> @@ -325,6 +325,7 @@ HttpSM::HttpSM()
>     client_response_hdr_bytes(0), client_response_body_bytes(0),
>     cache_response_hdr_bytes(0), cache_response_body_bytes(0),
>     pushed_response_hdr_bytes(0), pushed_response_body_bytes(0),
> +    plugin_tag(0), plugin_id(0),
>     hooks_set(0), cur_hook_id(TS_HTTP_LAST_HOOK), cur_hook(NULL),
>     cur_hooks(0), callout_state(HTTP_API_NO_CALLOUT), terminate_sm(false), kill_this_async_done(false)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/http/HttpSM.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
> index 2b5a6fd..13121ec 100644
> --- a/proxy/http/HttpSM.h
> +++ b/proxy/http/HttpSM.h
> @@ -488,6 +488,11 @@ public:
>   int pushed_response_hdr_bytes;
>   int64_t pushed_response_body_bytes;
>   TransactionMilestones milestones;
> +  // The next two enable plugins to tag the state machine for
> +  // the purposes of logging so the instances can be correlated
> +  // with the source plugin.
> +  char const* plugin_tag;
> +  int64_t plugin_id;
> 
>   // hooks_set records whether there are any hooks relevant
>   //  to this transaction.  Used to avoid costly calls
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/Log.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/Log.cc b/proxy/logging/Log.cc
> index 16138b4..9ee618e 100644
> --- a/proxy/logging/Log.cc
> +++ b/proxy/logging/Log.cc
> @@ -365,27 +365,19 @@ Log::init_fields()
>   global_field_list.add (field, false);
>   ink_hash_table_insert (field_symbol_hash, "caun", field);
> 
> -# if 0
> -  Ptr<LogFieldAliasTable> proto_type_map = make_ptr(new LogFieldAliasTable);
> -  proto_type_map->init(7,
> -                       // Transport protocols
> -                       TS_PROTO_UDP, "UDP",
> -                       TS_PROTO_TCP, "TCP",
> -                       TS_PROTO_TLS, "TLS",
> -                       // Application protocols
> -                       TS_PROTO_HTTP, "HTTP",
> -                       TS_PROTO_SPDY, "SPDY",
> -                       TS_PROTO_RTMP, "RTMP",
> -                       TS_PROTO_WS,   "WS");
> -
> -  field = new LogField("client_protocol_stack", "cps",
> +  field = new LogField("plugin_identity_id", "piid",
>                        LogField::sINT,
> -                       &LogAccess::marshal_client_protocol_stack,
> -                       &LogAccess::unmarshal_client_protocol_stack,
> -                       (Ptr<LogFieldAliasMap>) proto_type_map);
> +                       &LogAccess::marshal_plugin_identity_id,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_int_to_str));
>   global_field_list.add(field, false);
> -  ink_hash_table_insert(field_symbol_hash, "cps", field);
> -# endif
> +  ink_hash_table_insert(field_symbol_hash, "piid", field);
> +
> +  field = new LogField("plugin_identity_tag", "pitag",
> +                       LogField::STRING,
> +                       &LogAccess::marshal_plugin_identity_tag,
> +                       reinterpret_cast<LogField::UnmarshalFunc>(&LogAccess::unmarshal_str));
> +  global_field_list.add(field, false);
> +  ink_hash_table_insert(field_symbol_hash, "pitag", field);
> 
>   field = new LogField("client_req_timestamp_sec", "cqts",
>                        LogField::sINT,
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.cc b/proxy/logging/LogAccess.cc
> index 968762e..5009113 100644
> --- a/proxy/logging/LogAccess.cc
> +++ b/proxy/logging/LogAccess.cc
> @@ -75,12 +75,18 @@ LogAccess::init()
>   -------------------------------------------------------------------------*/
> 
> int
> -LogAccess::marshal_client_protocol_stack(char *buf)
> +LogAccess::marshal_plugin_identity_id(char *buf)
> {
>   DEFAULT_INT_FIELD;
> }
> 
> int
> +LogAccess::marshal_plugin_identity_tag(char *buf)
> +{
> +  DEFAULT_STR_FIELD;
> +}
> +
> +int
> LogAccess::marshal_client_host_ip(char *buf)
> {
>   DEFAULT_IP_FIELD;
> @@ -1311,43 +1317,6 @@ LogAccess::unmarshal_cache_write_code(char **buf, char *dest, int len, Ptr<LogFi
>   return (LogAccess::unmarshal_with_map(unmarshal_int(buf), dest, len, map, "UNKNOWN_CACHE_WRITE_CODE"));
> }
> 
> -# if 0
> -int
> -LogAccess::unmarshal_client_protocol_stack(char **buf, char *dest, int len, Ptr<LogFieldAliasMap> map)
> -{
> -  ink_assert(buf != NULL);
> -  ink_assert(*buf != NULL);
> -  ink_assert(dest != NULL);
> -
> -  char *p;
> -  size_t nr_chars = 0;
> -  int i, ret, nr_bits, left_len;
> -  TSClientProtoStack proto_stack = (TSClientProtoStack)unmarshal_int(buf);
> -
> -  p = dest;
> -  left_len = len;
> -  nr_bits = 8 * sizeof(TSClientProtoStack);
> -
> -  for (i = 0; i < nr_bits && left_len; i++) {
> -    if ((proto_stack >> i) & 0x1) {
> -      if (p != dest) {
> -        *p++ = '+';
> -        left_len--;
> -      }
> -      ret = map->asString(i, p, left_len, &nr_chars);
> -      if (ret == LogFieldAliasMap::ALL_OK) {
> -        p += nr_chars;
> -        left_len -= nr_chars;
> -      } else if (ret == LogFieldAliasMap::BUFFER_TOO_SMALL) {
> -        break;
> -      }
> -    }
> -  }
> -
> -  return (len - left_len);
> -}
> -# endif
> -
> int
> LogAccess::unmarshal_record(char **buf, char *dest, int len)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccess.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccess.h b/proxy/logging/LogAccess.h
> index 940145d..8fb35b5 100644
> --- a/proxy/logging/LogAccess.h
> +++ b/proxy/logging/LogAccess.h
> @@ -172,7 +172,6 @@ public:
>   inkcoreapi virtual int marshal_client_auth_user_name(char *); // STR
>   int marshal_client_req_timestamp_sec(char *); // INT
> 
> -  inkcoreapi virtual int marshal_client_protocol_stack(char *); // INT
>   inkcoreapi virtual int marshal_client_req_text(char *);       // STR
>   inkcoreapi virtual int marshal_client_req_http_method(char *);        // STR
>   inkcoreapi virtual int marshal_client_req_url(char *);        // STR
> @@ -251,6 +250,8 @@ public:
>   inkcoreapi virtual int marshal_transfer_time_ms(char *);      // INT
>   inkcoreapi virtual int marshal_transfer_time_s(char *);       // INT
>   inkcoreapi virtual int marshal_file_size(char *);     // INT
> +  inkcoreapi virtual int marshal_plugin_identity_id(char *); // INT
> +  inkcoreapi virtual int marshal_plugin_identity_tag(char *); // STR
>   int marshal_entry_type(char *);       // INT
> 
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.cc b/proxy/logging/LogAccessHttp.cc
> index a8dad70..b44e684 100644
> --- a/proxy/logging/LogAccessHttp.cc
> +++ b/proxy/logging/LogAccessHttp.cc
> @@ -206,16 +206,27 @@ LogAccessHttp::set_client_req_url_path(char *buf, int len)
> 
> /*-------------------------------------------------------------------------
>   -------------------------------------------------------------------------*/
> -# if 0
> int
> -LogAccessHttp::marshal_client_protocol_stack(char *buf)
> +LogAccessHttp::marshal_plugin_identity_id(char *buf)
> {
> -  if (buf) {
> -    marshal_int(buf, m_http_sm->proto_stack);
> -  }
> +  if (buf) marshal_int(buf, m_http_sm->plugin_id);
>   return INK_MIN_ALIGN;
> }
> -# endif
> +
> +int
> +LogAccessHttp::marshal_plugin_identity_tag(char *buf)
> +{
> +  int len = INK_MIN_ALIGN;
> +  char const* tag = m_http_sm->plugin_tag;
> +
> +  if (!tag) tag = "*";
> +  else len = LogAccess::strlen(tag);
> +
> +  if (buf) marshal_str(buf, tag, len);
> +
> +  return len;
> +}
> +
> int
> LogAccessHttp::marshal_client_host_ip(char *buf)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/logging/LogAccessHttp.h
> ----------------------------------------------------------------------
> diff --git a/proxy/logging/LogAccessHttp.h b/proxy/logging/LogAccessHttp.h
> index 53a0ff5..51ee9e3 100644
> --- a/proxy/logging/LogAccessHttp.h
> +++ b/proxy/logging/LogAccessHttp.h
> @@ -58,7 +58,6 @@ public:
>   virtual int marshal_client_host_ip(char *);   // STR
>   virtual int marshal_client_host_port(char *); // INT
>   virtual int marshal_client_auth_user_name(char *);    // STR
> -  //  virtual int marshal_client_protocol_stack(char *);    // INT
>   virtual int marshal_client_req_text(char *);  // STR
>   virtual int marshal_client_req_http_method(char *);   // INT
>   virtual int marshal_client_req_url(char *);   // STR
> @@ -128,6 +127,8 @@ public:
>   virtual int marshal_transfer_time_ms(char *); // INT
>   virtual int marshal_transfer_time_s(char *);  // INT
>   virtual int marshal_file_size(char *); // INT
> +  virtual int marshal_plugin_identity_id(char *);    // INT
> +  virtual int marshal_plugin_identity_tag(char *);    // STR
> 
>   //
>   // named fields from within a http header
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.cc b/proxy/spdy/SpdyClientSession.cc
> index 2c77ff6..fcb79fb 100644
> --- a/proxy/spdy/SpdyClientSession.cc
> +++ b/proxy/spdy/SpdyClientSession.cc
> @@ -27,6 +27,22 @@
> static ClassAllocator<SpdyClientSession> spdyClientSessionAllocator("spdyClientSessionAllocator");
> ClassAllocator<SpdyRequest> spdyRequestAllocator("spdyRequestAllocator");
> 
> +#if TS_HAS_SPDY
> +#include "SpdyClientSession.h"
> +
> +static const spdylay_proto_version versmap[] = {
> +  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> +  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> +  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> +};
> +
> +static char const* const  npnmap[] = {
> +  TS_NPN_PROTOCOL_SPDY_2,
> +  TS_NPN_PROTOCOL_SPDY_3,
> +  TS_NPN_PROTOCOL_SPDY_3_1
> +};
> +
> +#endif
> static int spdy_process_read(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_write(TSEvent event, SpdyClientSession *sm);
> static int spdy_process_fetch(TSEvent event, SpdyClientSession *sm, void *edata);
> @@ -69,15 +85,16 @@ SpdyRequest::clear()
> }
> 
> void
> -SpdyClientSession::init(NetVConnection * netvc, spdylay_proto_version vers)
> +SpdyClientSession::init(NetVConnection * netvc, spdy::SessionVersion vers)
> {
>   int r;
> 
>   this->mutex = new_ProxyMutex();
>   this->vc = netvc;
>   this->req_map.clear();
> +  this->version = vers;
> 
> -  r = spdylay_session_server_new(&session, vers, &SPDY_CFG.spdy.callbacks, this);
> +  r = spdylay_session_server_new(&session, versmap[vers], &SPDY_CFG.spdy.callbacks, this);
> 
>   // A bit ugly but we need a thread and I don't want to wait until the
>   // session start event in case of a time out generating a decrement
> @@ -157,7 +174,7 @@ SpdyClientSession::clear()
> }
> 
> void
> -spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader)
> +spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader)
> {
>   SpdyClientSession  *sm;
> 
> @@ -242,6 +259,19 @@ out:
>   return EVENT_CONT;
> }
> 
> +int64_t
> +SpdyClientSession::getPluginId() const
> +{
> +  return sm_id;
> +}
> +
> +char const*
> +SpdyClientSession::getPluginTag() const
> +{
> +  return npnmap[this->version];
> +}
> +
> +
> static int
> spdy_process_read(TSEvent /* event ATS_UNUSED */, SpdyClientSession *sm)
> {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdyClientSession.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdyClientSession.h b/proxy/spdy/SpdyClientSession.h
> index 2f37d35..738ff55 100644
> --- a/proxy/spdy/SpdyClientSession.h
> +++ b/proxy/spdy/SpdyClientSession.h
> @@ -24,9 +24,11 @@
> #ifndef __P_SPDY_SM_H__
> #define __P_SPDY_SM_H__
> 
> +#include "SpdyDefs.h"
> #include "SpdyCommon.h"
> #include "SpdyCallbacks.h"
> #include <openssl/md5.h>
> +#include "Plugin.h"
> 
> class SpdyClientSession;
> typedef int (*SpdyClientSessionHandler) (TSCont contp, TSEvent event, void *data);
> @@ -89,7 +91,7 @@ public:
>   MD5_CTX recv_md5;
> };
> 
> -class SpdyClientSession : public Continuation
> +class SpdyClientSession : public Continuation, public PluginIdentity
> {
> 
> public:
> @@ -101,10 +103,11 @@ public:
>     clear();
>   }
> 
> -  void init(NetVConnection * netvc, spdylay_proto_version vers);
> +  void init(NetVConnection * netvc, spdy::SessionVersion vers);
>   void clear();
> 
>   int64_t sm_id;
> +  spdy::SessionVersion version;
>   uint64_t total_size;
>   TSHRTime start_time;
> 
> @@ -124,12 +127,15 @@ public:
> 
>   map<int32_t, SpdyRequest*> req_map;
> 
> +  virtual char const* getPluginTag() const;
> +  virtual int64_t getPluginId() const;
> +
> private:
>   int state_session_start(int event, void * edata);
>   int state_session_readwrite(int event, void * edata);
> };
> 
> -void spdy_sm_create(NetVConnection * netvc, spdylay_proto_version vers, MIOBuffer * iobuf, IOBufferReader * reader);
> +void spdy_sm_create(NetVConnection * netvc, spdy::SessionVersion vers, MIOBuffer * iobuf, IOBufferReader * reader);
> 
> extern ClassAllocator<SpdyRequest> spdyRequestAllocator;
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.cc b/proxy/spdy/SpdySessionAccept.cc
> index e679adc..d7dfa11 100644
> --- a/proxy/spdy/SpdySessionAccept.cc
> +++ b/proxy/spdy/SpdySessionAccept.cc
> @@ -26,20 +26,13 @@
> 
> #if TS_HAS_SPDY
> #include "SpdyClientSession.h"
> -
> -static const spdylay_proto_version versmap[] = {
> -  SPDYLAY_PROTO_SPDY2,    // SPDY_VERSION_2
> -  SPDYLAY_PROTO_SPDY3,    // SPDY_VERSION_3
> -  SPDYLAY_PROTO_SPDY3_1,  // SPDY_VERSION_3_1
> -};
> -
> #endif
> 
> -SpdySessionAccept::SpdySessionAccept(unsigned vers)
> +SpdySessionAccept::SpdySessionAccept(spdy::SessionVersion vers)
>     : SessionAccept(new_ProxyMutex()), version(vers)
> {
> #if TS_HAS_SPDY
> -  ink_release_assert(vers < countof(versmap));
> +  ink_release_assert(spdy::SESSION_VERSION_2 <= vers && vers <= spdy::SESSION_VERSION_3_1);
> #endif
>   SET_HANDLER(&SpdySessionAccept::mainEvent);
> }
> @@ -51,7 +44,7 @@ SpdySessionAccept::mainEvent(int event, void * edata)
>     NetVConnection * netvc =static_cast<NetVConnection *>(edata);
> 
> #if TS_HAS_SPDY
> -    spdy_sm_create(netvc, versmap[this->version], NULL, NULL);
> +    spdy_sm_create(netvc, this->version, NULL, NULL);
> #else
>     Error("accepted a SPDY session, but SPDY support is not available");
>     netvc->do_io_close();
> @@ -68,7 +61,7 @@ void
> SpdySessionAccept::accept(NetVConnection * netvc, MIOBuffer * iobuf, IOBufferReader * reader)
> {
> #if TS_HAS_SPDY
> -  spdy_sm_create(netvc, versmap[this->version], iobuf, reader);
> +  spdy_sm_create(netvc, this->version, iobuf, reader);
> #else
>   (void)netvc;
>   (void)iobuf;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/aadb7522/proxy/spdy/SpdySessionAccept.h
> ----------------------------------------------------------------------
> diff --git a/proxy/spdy/SpdySessionAccept.h b/proxy/spdy/SpdySessionAccept.h
> index 9d740e2..af3d81d 100644
> --- a/proxy/spdy/SpdySessionAccept.h
> +++ b/proxy/spdy/SpdySessionAccept.h
> @@ -28,18 +28,13 @@
> #include "P_EventSystem.h"
> #include "P_UnixNet.h"
> #include "I_IOBuffer.h"
> +#include "SpdyDefs.h"
> 
> class SpdySessionAccept: public SessionAccept
> {
> public:
> 
> -  enum {
> -    SPDY_VERSION_2 = 0,
> -    SPDY_VERSION_3,
> -    SPDY_VERSION_3_1,
> -  };
> -
> -  explicit SpdySessionAccept(unsigned vers);
> +  explicit SpdySessionAccept(spdy::SessionVersion vers);
>   ~SpdySessionAccept() {}
> 
>   void accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
> @@ -49,7 +44,7 @@ private:
>   SpdySessionAccept(const SpdySessionAccept &); // disabled
>   SpdySessionAccept& operator =(const SpdySessionAccept&); // disabled
> 
> -  unsigned version;
> +  spdy::SessionVersion version;
> };
> 
> #endif /* SpdySessionAccept_H_ */
>