You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by zw...@apache.org on 2014/03/04 09:19:25 UTC

[1/2] git commit: TS-2522 Better hook management, and cleanup

Repository: trafficserver
Updated Branches:
  refs/heads/master 17f680231 -> 4f0b0b447


TS-2522 Better hook management, and cleanup

This also does

     - Add support for system wide config directory as default
     - Better debug messages (textual hook names etc.)
     - Less overhead on creating TXN hooks


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

Branch: refs/heads/master
Commit: 31792d2aae9519e0feb66ab9e80b1a9d5d51d7de
Parents: 17f6802
Author: Leif Hedstrom <zw...@apache.org>
Authored: Thu Feb 27 19:55:53 2014 -0700
Committer: Leif Hedstrom <zw...@apache.org>
Committed: Tue Mar 4 01:13:59 2014 -0700

----------------------------------------------------------------------
 CHANGES                                  |   3 +
 plugins/header_rewrite/header_rewrite.cc | 216 ++++++++++++++++----------
 plugins/header_rewrite/lulu.h            |   4 +-
 plugins/header_rewrite/operators.cc      |   8 +-
 plugins/header_rewrite/parser.cc         |   2 +-
 plugins/header_rewrite/resources.cc      |   7 +-
 plugins/header_rewrite/resources.h       |   1 -
 plugins/header_rewrite/ruleset.cc        |   4 +-
 8 files changed, 152 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 3a83537..363ab16 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,9 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.0.0
 
+  *) [TS-2522] Better hook management for header_rewrite plugin, and some
+   cleanup.
+
   *) [TS-2169] Add SSL statistics
    Author: Ron Barber <rb...@yahoo-inc.com>
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/header_rewrite.cc
----------------------------------------------------------------------
diff --git a/plugins/header_rewrite/header_rewrite.cc b/plugins/header_rewrite/header_rewrite.cc
index 6fc7960..614d287 100644
--- a/plugins/header_rewrite/header_rewrite.cc
+++ b/plugins/header_rewrite/header_rewrite.cc
@@ -27,40 +27,83 @@
 #include "resources.h"
 
 // Debugs
-const char* PLUGIN_NAME = "header_rewrite";
-const char* PLUGIN_NAME_DBG = "header_rewrite_dbg";
+const char PLUGIN_NAME[] = "header_rewrite";
+const char PLUGIN_NAME_DBG[] = "dbg_header_rewrite";
+
+const char* HOOK_NAMES[] = {
+  "TS_HTTP_READ_REQUEST_HDR_HOOK",
+  "TS_HTTP_OS_DNS_HOOK",
+  "TS_HTTP_SEND_REQUEST_HDR_HOOK",
+  "TS_HTTP_READ_CACHE_HDR_HOOK",
+  "TS_HTTP_READ_RESPONSE_HDR_HOOK",
+  "TS_HTTP_SEND_RESPONSE_HDR_HOOK",
+  "TS_HTTP_REQUEST_TRANSFORM_HOOK",
+  "TS_HTTP_RESPONSE_TRANSFORM_HOOK",
+  "TS_HTTP_SELECT_ALT_HOOK",
+  "TS_HTTP_TXN_START_HOOK",
+  "TS_HTTP_TXN_CLOSE_HOOK",
+  "TS_HTTP_SSN_START_HOOK",
+  "TS_HTTP_SSN_CLOSE_HOOK",
+  "TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK",
+  "TS_HTTP_PRE_REMAP_HOOK",
+  "TS_HTTP_POST_REMAP_HOOK",
+  "TS_HTTP_RESPONSE_CLIENT_HOOK",
+  "TS_HTTP_LAST_HOOK"
+};
+
+
+// Forward declaration for the main continuation.
+static int cont_rewrite_headers(TSCont, TSEvent, void *);
 
-static const char* DEFAULT_CONF_PATH = "/usr/local/etc/header_rewrite/";
 
 // Simple wrapper around a configuration file / set. This is useful such that
 // we can reuse most of the code for both global and per-remap rule sets.
-struct RulesConfig
+class RulesConfig
 {
+public:
   RulesConfig()
   {
-    memset(rules, 0, sizeof(rules));
-    memset(resids, 0, sizeof(resids));
+    memset(_rules, 0, sizeof(_rules));
+    memset(_resids, 0, sizeof(_resids));
+
+    _cont = TSContCreate(cont_rewrite_headers, NULL);
+    TSContDataSet(_cont, static_cast<void*>(this));
   }
 
   ~RulesConfig()
   {
-    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i)
-      delete rules[i];
+    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
+      delete _rules[i];
+    }
+
+    TSContDestroy(_cont);
   }
 
-  RuleSet* rules[TS_HTTP_LAST_HOOK+1];
-  ResourceIDs resids[TS_HTTP_LAST_HOOK+1];
+  TSCont continuation() const { return _cont; }
+
+  ResourceIDs resid(int hook) const { return _resids[hook]; }
+  RuleSet* rule(int hook) const { return _rules[hook]; }
+
+  bool parse_config(const std::string fname, TSHttpHookID default_hook);
+
+private:
+  bool add_rule(RuleSet* rule);
+
+  TSCont _cont;
+  RuleSet* _rules[TS_HTTP_LAST_HOOK+1];
+  ResourceIDs _resids[TS_HTTP_LAST_HOOK+1];
 };
 
 // Helper function to add a rule to the rulesets
-static bool
-add_rule(RuleSet* rule, RulesConfig *conf) {
+bool
+RulesConfig::add_rule(RuleSet* rule)
+{
   if (rule && rule->has_operator()) {
-    TSDebug(PLUGIN_NAME, "Adding rule to hook=%d\n", rule->get_hook());
-    if (NULL == conf->rules[rule->get_hook()]) {
-      conf->rules[rule->get_hook()] = rule;
+    TSDebug(PLUGIN_NAME_DBG, "   Adding rule to hook=%s\n", HOOK_NAMES[rule->get_hook()]);
+    if (NULL == _rules[rule->get_hook()]) {
+      _rules[rule->get_hook()] = rule;
     } else {
-      conf->rules[rule->get_hook()]->append(rule);
+      _rules[rule->get_hook()]->append(rule);
     }
     return true;
   }
@@ -76,17 +119,24 @@ add_rule(RuleSet* rule, RulesConfig *conf) {
 // anyways (or reload for remap.config), so not really in the critical path.
 //
 bool
-parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *conf)
+RulesConfig::parse_config(const std::string fname, TSHttpHookID default_hook)
 {
   RuleSet* rule = NULL;
-  std::string filename = fname;
+  std::string filename;
   std::ifstream f;
   int lineno = 0;
 
-  // Try appending the default conf path if the fname doesn't exist.
-  if (0 != access(filename.c_str(), R_OK)) {
-    filename = DEFAULT_CONF_PATH;
-    filename += fname;
+  if (0 == fname.size()) {
+    TSError("%s: no config filename provided", PLUGIN_NAME);
+    return false;
+
+  }
+
+  if (fname[0] != '/') {
+    filename = TSConfigDirGet();
+    filename += "/" + fname;
+  } else {
+    filename = fname;
   }
 
   f.open(filename.c_str(), std::ios::in);
@@ -95,8 +145,6 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
     return false;
   }
 
-  TSDebug(PLUGIN_NAME, "Loading header_rewrite config from %s", filename.c_str());
-
   while (!f.eof()) {
     std::string line;
 
@@ -105,16 +153,19 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
     TSDebug(PLUGIN_NAME_DBG, "Reading line: %d: %s", lineno, line.c_str());
 
     boost::trim(line);
-    if (line.empty() || (line[0] == '#'))
+    if (line.empty() || (line[0] == '#')) {
       continue;
+    }
 
     Parser p(line);  // Tokenize and parse this line
-    if (p.empty())
+    if (p.empty()) {
       continue;
+    }
 
     // If we are at the beginning of a new condition, save away the previous rule (but only if it has operators).
-    if (p.is_cond() && add_rule(rule, conf))
+    if (p.is_cond() && add_rule(rule)) {
       rule = NULL;
+    }
 
     if (NULL == rule) {
       rule = new RuleSet();
@@ -151,12 +202,12 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
   }
 
   // Add the last rule (possibly the only rule)
-  add_rule(rule, conf);
+  add_rule(rule);
 
   // Collect all resource IDs that we need
   for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
-    if (conf->rules[i]) {
-      conf->resids[i] = conf->rules[i]->get_all_resource_ids();
+    if (_rules[i]) {
+      _resids[i] = _rules[i]->get_all_resource_ids();
     }
   }
 
@@ -174,7 +225,6 @@ cont_rewrite_headers(TSCont contp, TSEvent event, void *edata)
   TSHttpHookID hook = TS_HTTP_LAST_HOOK;
   RulesConfig* conf = static_cast<RulesConfig*>(TSContDataGet(contp));
 
-  // Get the resources necessary to process this event
   switch (event) {
   case TS_EVENT_HTTP_READ_RESPONSE_HDR:
     hook = TS_HTTP_READ_RESPONSE_HDR_HOOK;
@@ -198,10 +248,11 @@ cont_rewrite_headers(TSCont contp, TSEvent event, void *edata)
   }
 
   if (hook != TS_HTTP_LAST_HOOK) {
-    const RuleSet* rule = conf->rules[hook];
+    const RuleSet* rule = conf->rule(hook);
     Resources res(txnp, contp);
 
-    res.gather(conf->resids[hook], hook);
+    // Get the resources necessary to process this event
+    res.gather(conf->resid(hook), hook);
 
     // Evaluation of all rules. This code is sort of duplicate in DoRemap as well.
     while (rule) {
@@ -236,7 +287,6 @@ TSPluginInit(int argc, const char *argv[])
   if (TS_SUCCESS != TSPluginRegister(TS_SDK_VERSION_3_0 , &info)) {
     TSError("%s: plugin registration failed.\n", PLUGIN_NAME);
   }
-  TSDebug(PLUGIN_NAME, "number of arguments: %d", argc);
 
   // Parse the global config file(s). All rules are just appended
   // to the "global" Rules configuration.
@@ -246,10 +296,12 @@ TSPluginInit(int argc, const char *argv[])
   for (int i=1; i < argc; ++i) {
     // Parse the config file(s). Note that multiple config files are
     // just appended to the configurations.
-    if (!parse_config(argv[i], TS_HTTP_READ_RESPONSE_HDR_HOOK, conf)) {
-      TSError("header_rewrite: failed to parse configuration file %s", argv[argc]);
-    } else {
+    TSDebug(PLUGIN_NAME, "Loading global configuration file %s", argv[i]);
+    if (conf->parse_config(argv[i], TS_HTTP_READ_RESPONSE_HDR_HOOK)) {
+      TSDebug(PLUGIN_NAME, "Succesfully loaded global config file %s", argv[i]);
       got_config = true;
+    } else {
+      TSError("header_rewrite: failed to parse configuration file %s", argv[i]);
     }
   }
 
@@ -258,8 +310,8 @@ TSPluginInit(int argc, const char *argv[])
     TSContDataSet(contp, conf);
 
     for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
-      if (conf->rules[i]) {
-        TSDebug(PLUGIN_NAME, "adding hook: %d", i);
+      if (conf->rule(i)) {
+        TSDebug(PLUGIN_NAME, "Adding global ruleset to hook=%s", HOOK_NAMES[i]);
         TSHttpHookAdd(static_cast<TSHttpHookID>(i), contp);
       }
     }
@@ -293,7 +345,7 @@ TSRemapInit(TSRemapInterface *api_info, char *errbuf, int errbuf_size)
     return TS_ERROR;
   }
 
-  TSDebug(PLUGIN_NAME, "remap plugin is successfully initialized");
+  TSDebug(PLUGIN_NAME, "Remap plugin is successfully initialized");
   return TS_SUCCESS;
 }
 
@@ -301,7 +353,7 @@ TSRemapInit(TSRemapInterface *api_info, char *errbuf, int errbuf_size)
 TSReturnCode
 TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf ATS_UNUSED */, int /* errbuf_size ATS_UNUSED */)
 {
-  TSDebug(PLUGIN_NAME, "initializing the remap plugin header_rewrite");
+  TSDebug(PLUGIN_NAME, "Instantiating a new remap.config plugin rule");
 
   if (argc < 3) {
     TSError("%s: Unable to create remap instance, need config file", PLUGIN_NAME);
@@ -311,14 +363,25 @@ TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf ATS_UNUSE
   RulesConfig* conf = new RulesConfig;
 
   for (int i=2; i < argc; ++i) {
-    if (!parse_config(argv[i], TS_REMAP_PSEUDO_HOOK, conf)) {
+    TSDebug(PLUGIN_NAME, "Loading remap configuration file %s", argv[i]);
+    if (!conf->parse_config(argv[i], TS_REMAP_PSEUDO_HOOK)) {
       TSError("%s: Unable to create remap instance", PLUGIN_NAME);
       return TS_ERROR;
+    } else {
+      TSDebug(PLUGIN_NAME, "Succesfully loaded remap config file %s", argv[i]);
     }
   }
 
-  *ih = conf;
-  TSDebug(PLUGIN_NAME, "added header_rewrite remap rule set");
+  // For debugging only
+  if (TSIsDebugTagSet(PLUGIN_NAME)) {
+    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
+      if (conf->rule(i)) {
+        TSDebug(PLUGIN_NAME, "Adding remap ruleset to hook=%s", HOOK_NAMES[i]);
+      }
+    }
+  }
+
+  *ih = static_cast<void*>(conf);
 
   return TS_SUCCESS;
 }
@@ -338,57 +401,46 @@ TSRemapDeleteInstance(void *ih)
 TSRemapStatus
 TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
 {
-  TSRemapStatus rval = TSREMAP_NO_REMAP;
-
+  // Make sure things are properly setup (this should never happen)
   if (NULL == ih) {
     TSDebug(PLUGIN_NAME, "No Rules configured, falling back to default");
-    return rval;
-  } else {
-    RulesConfig* conf = static_cast<RulesConfig*>(ih);
-
-    // TODO: We should optimize this to not create continuations every time,
-    // but instead save the continuations, with configurations, as the per
-    // remap instance data.
+    return TSREMAP_NO_REMAP;
+  }
 
-    // Go through all hooks we support, and setup the txn hook(s) as necessary
-    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
-      TSCont contp = NULL;
+  TSRemapStatus rval = TSREMAP_NO_REMAP;
+  RulesConfig* conf = static_cast<RulesConfig*>(ih);
 
-      if (conf->rules[i]) {
-        if (NULL == contp) {
-          contp = TSContCreate(cont_rewrite_headers, NULL);
-          TSContDataSet(contp, conf);
-        }
-        TSHttpTxnHookAdd(rh, static_cast<TSHttpHookID>(i), contp);
-        TSDebug(PLUGIN_NAME, "activated transaction hook via remap.config: new hook=%d", i);
-      }
+  // Go through all hooks we support, and setup the txn hook(s) as necessary
+  for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
+    if (conf->rule(i)) {
+      TSHttpTxnHookAdd(rh, static_cast<TSHttpHookID>(i), conf->continuation());
+      TSDebug(PLUGIN_NAME, "Added remapped TXN hook=%s", HOOK_NAMES[i]);
     }
+  }
 
-    // Now handle the remap specific rules for the "remap hook" (which is not a real hook).
-    // This avoids scheduling an additional continuation for a very common case.
-    RuleSet* rule = conf->rules[TS_REMAP_PSEUDO_HOOK];
-    Resources res(rh, rri);
-
-    // res.gather(conf->resids[TS_REMAP_PSEUDO_HOOK], TS_REMAP_PSEUDO_HOOK);
-    res.gather(RSRC_CLIENT_REQUEST_HEADERS, TS_REMAP_PSEUDO_HOOK);
+  // Now handle the remap specific rules for the "remap hook" (which is not a real hook).
+  // This is sufficiently differen than the normal cont_rewrite_headers() callback, and
+  // we can't (shouldn't) schedule this as a TXN hook.
+  RuleSet* rule = conf->rule(TS_REMAP_PSEUDO_HOOK);
+  Resources res(rh, rri);
 
-    // Evaluation. This code is duplicated sort of, should we merge with the continuation evaluator ?
-    while (rule) {
-      if (rule->eval(res)) {
-        OperModifiers rt = rule->exec(res);
+  res.gather(RSRC_CLIENT_REQUEST_HEADERS, TS_REMAP_PSEUDO_HOOK);
+  while (rule) {
+    if (rule->eval(res)) {
+      OperModifiers rt = rule->exec(res);
 
-        if (res.changed_url == true)
-          rval = TSREMAP_DID_REMAP;
+      if (res.changed_url == true) {
+        rval = TSREMAP_DID_REMAP;
+      }
 
-        if (rule->last() || (rt & OPER_LAST)) {
-          break; // Conditional break, force a break with [L]
-        }
+      if (rule->last() || (rt & OPER_LAST)) {
+        break; // Conditional break, force a break with [L]
       }
-      rule = rule->next;
     }
+    rule = rule->next;
   }
 
-  TSDebug(PLUGIN_NAME, "returing with status: %d", rval);
+  TSDebug(PLUGIN_NAME_DBG, "Returing from TSRemapDoRemap with status: %d", rval);
   return rval;
 }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/lulu.h
----------------------------------------------------------------------
diff --git a/plugins/header_rewrite/lulu.h b/plugins/header_rewrite/lulu.h
index 0dd3cb1..eb89031 100644
--- a/plugins/header_rewrite/lulu.h
+++ b/plugins/header_rewrite/lulu.h
@@ -43,8 +43,8 @@
 #error "Define barriers"
 #endif
 
-extern const char* PLUGIN_NAME;
-extern const char* PLUGIN_NAME_DBG;
+extern const char PLUGIN_NAME[];
+extern const char PLUGIN_NAME_DBG[];
 
 
 // From google styleguide: http://google-styleguide.googlecode.com/svn/trunk/cppguide.xml

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/operators.cc
----------------------------------------------------------------------
diff --git a/plugins/header_rewrite/operators.cc b/plugins/header_rewrite/operators.cc
index d6a60e5..3b2cd27 100644
--- a/plugins/header_rewrite/operators.cc
+++ b/plugins/header_rewrite/operators.cc
@@ -361,7 +361,7 @@ OperatorRMHeader::exec(const Resources& res) const
     TSDebug(PLUGIN_NAME, "OperatorRMHeader::exec() invoked on header %s", _header.c_str());
     field_loc = TSMimeHdrFieldFind(res.bufp, res.hdr_loc, _header.c_str(), _header.size());
     while (field_loc) {
-      TSDebug(PLUGIN_NAME, "\tdeleting header %s", _header.c_str());
+      TSDebug(PLUGIN_NAME, "   Deleting header %s", _header.c_str());
       tmp = TSMimeHdrFieldNextDup(res.bufp, res.hdr_loc, field_loc);
       TSMimeHdrFieldDestroy(res.bufp, res.hdr_loc, field_loc);
       TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
@@ -405,7 +405,7 @@ OperatorAddHeader::exec(const Resources& res) const
 
     if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(res.bufp, res.hdr_loc, _header.c_str(), _header.size(), &field_loc)) {
       if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
-        TSDebug(PLUGIN_NAME, "   adding header %s", _header.c_str());
+        TSDebug(PLUGIN_NAME, "   Adding header %s", _header.c_str());
         TSMimeHdrFieldAppend(res.bufp, res.hdr_loc, field_loc);
       }
       TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
@@ -445,7 +445,7 @@ OperatorSetHeader::exec(const Resources& res) const
       // No existing header, so create one
       if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(res.bufp, res.hdr_loc, _header.c_str(), _header.size(), &field_loc)) {
         if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
-          TSDebug(PLUGIN_NAME, "   adding header %s", _header.c_str());
+          TSDebug(PLUGIN_NAME, "   Adding header %s", _header.c_str());
           TSMimeHdrFieldAppend(res.bufp, res.hdr_loc, field_loc);
         }
         TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
@@ -458,7 +458,7 @@ OperatorSetHeader::exec(const Resources& res) const
         if (first) {
           first = false;
           if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
-            TSDebug(PLUGIN_NAME, "   overwriting header %s", _header.c_str());
+            TSDebug(PLUGIN_NAME, "   Overwriting header %s", _header.c_str());
           }
         } else {
           TSMimeHdrFieldDestroy(res.bufp, res.hdr_loc, field_loc);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/parser.cc
----------------------------------------------------------------------
diff --git a/plugins/header_rewrite/parser.cc b/plugins/header_rewrite/parser.cc
index cdff4ee..a389858 100644
--- a/plugins/header_rewrite/parser.cc
+++ b/plugins/header_rewrite/parser.cc
@@ -98,7 +98,7 @@ Parser::preprocess(std::vector<std::string>& tokens)
 Parser::Parser(const std::string& line) :
   _cond(false), _empty(false)
 {
-  TSDebug("header_rewrite_dbg", "Calling CTOR for Parser");
+  TSDebug(PLUGIN_NAME_DBG, "Calling CTOR for Parser");
 
   if (line[0] == '#') {
     _empty = true;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/resources.cc
----------------------------------------------------------------------
diff --git a/plugins/header_rewrite/resources.cc b/plugins/header_rewrite/resources.cc
index d89149a..eb4afaf 100644
--- a/plugins/header_rewrite/resources.cc
+++ b/plugins/header_rewrite/resources.cc
@@ -24,10 +24,15 @@
 #include "resources.h"
 #include "lulu.h"
 
+// This is defined in header_rewrite.cc, used here for debugging
+extern char* HOOK_NAMES[];
+
+
+// Collect all resources
 void
 Resources::gather(const ResourceIDs ids, TSHttpHookID hook)
 {
-  TSDebug(PLUGIN_NAME, "Building resource structure for hook (%d)", hook);
+  TSDebug(PLUGIN_NAME, "Building resources, hook=%s", HOOK_NAMES[hook]);
 
   // If we need the client request headers, make sure it's also available in the client vars.
   if (ids & RSRC_CLIENT_REQUEST_HEADERS) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/resources.h
----------------------------------------------------------------------
diff --git a/plugins/header_rewrite/resources.h b/plugins/header_rewrite/resources.h
index cb9d062..5bda0da 100644
--- a/plugins/header_rewrite/resources.h
+++ b/plugins/header_rewrite/resources.h
@@ -60,7 +60,6 @@ public:
       _rri(rri), changed_url(false), _ready(false)
   {
     TSDebug(PLUGIN_NAME_DBG, "Calling CTOR for Resources (RemapAPI)");
-    TSDebug(PLUGIN_NAME, "rri: %p", _rri);
   }
 
   ~Resources() { destroy(); }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/ruleset.cc
----------------------------------------------------------------------
diff --git a/plugins/header_rewrite/ruleset.cc b/plugins/header_rewrite/ruleset.cc
index 9c6bc06..e210e47 100644
--- a/plugins/header_rewrite/ruleset.cc
+++ b/plugins/header_rewrite/ruleset.cc
@@ -47,7 +47,7 @@ RuleSet::add_condition(Parser& p)
   Condition* c = condition_factory(p.get_op());
 
   if (NULL != c) {
-    TSDebug(PLUGIN_NAME, "Adding condition: %%{%s} with arg: %s\n", p.get_op().c_str(), p.get_arg().c_str());
+    TSDebug(PLUGIN_NAME, "   Adding condition: %%{%s} with arg: %s\n", p.get_op().c_str(), p.get_arg().c_str());
     c->initialize(p);
     if (!c->set_hook(_hook)) {
       TSError("%s: can't use this condition in this hook", PLUGIN_NAME);
@@ -73,7 +73,7 @@ RuleSet::add_operator(Parser& p)
 
   if (NULL != o) {
     // TODO: This should be extended to show both the "argument" and the "value" (if both are used)
-    TSDebug(PLUGIN_NAME, "Adding operator: %s(%s)\n", p.get_op().c_str(), p.get_arg().c_str());
+    TSDebug(PLUGIN_NAME, "   Adding operator: %s(%s)\n", p.get_op().c_str(), p.get_arg().c_str());
     o->initialize(p);
     if (!o->set_hook(_hook)) {
       TSError("%s: can't use this operator in this hook", PLUGIN_NAME);


[2/2] git commit: TS-2522 Update the docs with new default conf dir

Posted by zw...@apache.org.
TS-2522 Update the docs with new default conf dir


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

Branch: refs/heads/master
Commit: 4f0b0b447ed76244287ae5385719fc56b3385a24
Parents: 31792d2
Author: Leif Hedstrom <zw...@apache.org>
Authored: Tue Mar 4 01:19:13 2014 -0700
Committer: Leif Hedstrom <zw...@apache.org>
Committed: Tue Mar 4 01:19:13 2014 -0700

----------------------------------------------------------------------
 doc/reference/plugins/header_rewrite.en.rst | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4f0b0b44/doc/reference/plugins/header_rewrite.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/plugins/header_rewrite.en.rst b/doc/reference/plugins/header_rewrite.en.rst
index d433b44..1df31d2 100644
--- a/doc/reference/plugins/header_rewrite.en.rst
+++ b/doc/reference/plugins/header_rewrite.en.rst
@@ -42,6 +42,10 @@ phase (the default) causes a transaction hook to be instantiated and used
 at a later time. This allows you to setup e.g. a rule that gets executed
 during the origin response header parsing, using READ_RESPONSE_HDR_HOOK.
 
+Configuration filenames without an absolute paths are searched for in the
+default configuration directory. This is typically where your main
+configuration files are, e.g. ``/usr/local/etc/trafficserver``.
+
 Operators
 ---------
 


Re: [1/2] git commit: TS-2522 Better hook management, and cleanup

Posted by Leif Hedstrom <zw...@apache.org>.
On Mar 4, 2014, at 4:36 PM, James Peach <jp...@apache.org> wrote:

>> +const char PLUGIN_NAME[] = "header_rewrite";
>> +const char PLUGIN_NAME_DBG[] = "dbg_header_rewrite";
>> +
>> +const char* HOOK_NAMES[] = {
> 
> You should do some kind of static assert to force a build break if new hooks are added.


I’m hoping to replace this with the new APIs we’ve been discussing. This junk is the reason why I filed that Jira :).


— Leif


Re: [1/2] git commit: TS-2522 Better hook management, and cleanup

Posted by Leif Hedstrom <zw...@apache.org>.
On Mar 4, 2014, at 4:36 PM, James Peach <jp...@apache.org> wrote:

>> +const char PLUGIN_NAME[] = "header_rewrite";
>> +const char PLUGIN_NAME_DBG[] = "dbg_header_rewrite";
>> +
>> +const char* HOOK_NAMES[] = {
> 
> You should do some kind of static assert to force a build break if new hooks are added.


I’m hoping to replace this with the new APIs we’ve been discussing. This junk is the reason why I filed that Jira :).


— Leif


Re: [1/2] git commit: TS-2522 Better hook management, and cleanup

Posted by James Peach <jp...@apache.org>.
On Mar 4, 2014, at 12:19 AM, zwoop@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master 17f680231 -> 4f0b0b447
> 
> 
> TS-2522 Better hook management, and cleanup
> 
> This also does
> 
>     - Add support for system wide config directory as default
>     - Better debug messages (textual hook names etc.)
>     - Less overhead on creating TXN hooks
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/31792d2a
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/31792d2a
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/31792d2a
> 
> Branch: refs/heads/master
> Commit: 31792d2aae9519e0feb66ab9e80b1a9d5d51d7de
> Parents: 17f6802
> Author: Leif Hedstrom <zw...@apache.org>
> Authored: Thu Feb 27 19:55:53 2014 -0700
> Committer: Leif Hedstrom <zw...@apache.org>
> Committed: Tue Mar 4 01:13:59 2014 -0700
> 
> ----------------------------------------------------------------------
> CHANGES                                  |   3 +
> plugins/header_rewrite/header_rewrite.cc | 216 ++++++++++++++++----------
> plugins/header_rewrite/lulu.h            |   4 +-
> plugins/header_rewrite/operators.cc      |   8 +-
> plugins/header_rewrite/parser.cc         |   2 +-
> plugins/header_rewrite/resources.cc      |   7 +-
> plugins/header_rewrite/resources.h       |   1 -
> plugins/header_rewrite/ruleset.cc        |   4 +-
> 8 files changed, 152 insertions(+), 93 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/CHANGES
> ----------------------------------------------------------------------
> diff --git a/CHANGES b/CHANGES
> index 3a83537..363ab16 100644
> --- a/CHANGES
> +++ b/CHANGES
> @@ -1,6 +1,9 @@
>                                                          -*- coding: utf-8 -*-
> Changes with Apache Traffic Server 5.0.0
> 
> +  *) [TS-2522] Better hook management for header_rewrite plugin, and some
> +   cleanup.
> +
>   *) [TS-2169] Add SSL statistics
>    Author: Ron Barber <rb...@yahoo-inc.com>
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/header_rewrite.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/header_rewrite.cc b/plugins/header_rewrite/header_rewrite.cc
> index 6fc7960..614d287 100644
> --- a/plugins/header_rewrite/header_rewrite.cc
> +++ b/plugins/header_rewrite/header_rewrite.cc
> @@ -27,40 +27,83 @@
> #include "resources.h"
> 
> // Debugs
> -const char* PLUGIN_NAME = "header_rewrite";
> -const char* PLUGIN_NAME_DBG = "header_rewrite_dbg";
> +const char PLUGIN_NAME[] = "header_rewrite";
> +const char PLUGIN_NAME_DBG[] = "dbg_header_rewrite";
> +
> +const char* HOOK_NAMES[] = {

You should do some kind of static assert to force a build break if new hooks are added.

> +  "TS_HTTP_READ_REQUEST_HDR_HOOK",
> +  "TS_HTTP_OS_DNS_HOOK",
> +  "TS_HTTP_SEND_REQUEST_HDR_HOOK",
> +  "TS_HTTP_READ_CACHE_HDR_HOOK",
> +  "TS_HTTP_READ_RESPONSE_HDR_HOOK",
> +  "TS_HTTP_SEND_RESPONSE_HDR_HOOK",
> +  "TS_HTTP_REQUEST_TRANSFORM_HOOK",
> +  "TS_HTTP_RESPONSE_TRANSFORM_HOOK",
> +  "TS_HTTP_SELECT_ALT_HOOK",
> +  "TS_HTTP_TXN_START_HOOK",
> +  "TS_HTTP_TXN_CLOSE_HOOK",
> +  "TS_HTTP_SSN_START_HOOK",
> +  "TS_HTTP_SSN_CLOSE_HOOK",
> +  "TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK",
> +  "TS_HTTP_PRE_REMAP_HOOK",
> +  "TS_HTTP_POST_REMAP_HOOK",
> +  "TS_HTTP_RESPONSE_CLIENT_HOOK",
> +  "TS_HTTP_LAST_HOOK"
> +};
> +
> +
> +// Forward declaration for the main continuation.
> +static int cont_rewrite_headers(TSCont, TSEvent, void *);
> 
> -static const char* DEFAULT_CONF_PATH = "/usr/local/etc/header_rewrite/";
> 
> // Simple wrapper around a configuration file / set. This is useful such that
> // we can reuse most of the code for both global and per-remap rule sets.
> -struct RulesConfig
> +class RulesConfig
> {
> +public:
>   RulesConfig()
>   {
> -    memset(rules, 0, sizeof(rules));
> -    memset(resids, 0, sizeof(resids));
> +    memset(_rules, 0, sizeof(_rules));
> +    memset(_resids, 0, sizeof(_resids));
> +
> +    _cont = TSContCreate(cont_rewrite_headers, NULL);
> +    TSContDataSet(_cont, static_cast<void*>(this));
>   }
> 
>   ~RulesConfig()
>   {
> -    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i)
> -      delete rules[i];
> +    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> +      delete _rules[i];
> +    }
> +
> +    TSContDestroy(_cont);
>   }
> 
> -  RuleSet* rules[TS_HTTP_LAST_HOOK+1];
> -  ResourceIDs resids[TS_HTTP_LAST_HOOK+1];
> +  TSCont continuation() const { return _cont; }
> +
> +  ResourceIDs resid(int hook) const { return _resids[hook]; }
> +  RuleSet* rule(int hook) const { return _rules[hook]; }
> +
> +  bool parse_config(const std::string fname, TSHttpHookID default_hook);
> +
> +private:
> +  bool add_rule(RuleSet* rule);
> +
> +  TSCont _cont;
> +  RuleSet* _rules[TS_HTTP_LAST_HOOK+1];
> +  ResourceIDs _resids[TS_HTTP_LAST_HOOK+1];
> };
> 
> // Helper function to add a rule to the rulesets
> -static bool
> -add_rule(RuleSet* rule, RulesConfig *conf) {
> +bool
> +RulesConfig::add_rule(RuleSet* rule)
> +{
>   if (rule && rule->has_operator()) {
> -    TSDebug(PLUGIN_NAME, "Adding rule to hook=%d\n", rule->get_hook());
> -    if (NULL == conf->rules[rule->get_hook()]) {
> -      conf->rules[rule->get_hook()] = rule;
> +    TSDebug(PLUGIN_NAME_DBG, "   Adding rule to hook=%s\n", HOOK_NAMES[rule->get_hook()]);
> +    if (NULL == _rules[rule->get_hook()]) {
> +      _rules[rule->get_hook()] = rule;
>     } else {
> -      conf->rules[rule->get_hook()]->append(rule);
> +      _rules[rule->get_hook()]->append(rule);
>     }
>     return true;
>   }
> @@ -76,17 +119,24 @@ add_rule(RuleSet* rule, RulesConfig *conf) {
> // anyways (or reload for remap.config), so not really in the critical path.
> //
> bool
> -parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *conf)
> +RulesConfig::parse_config(const std::string fname, TSHttpHookID default_hook)
> {
>   RuleSet* rule = NULL;
> -  std::string filename = fname;
> +  std::string filename;
>   std::ifstream f;
>   int lineno = 0;
> 
> -  // Try appending the default conf path if the fname doesn't exist.
> -  if (0 != access(filename.c_str(), R_OK)) {
> -    filename = DEFAULT_CONF_PATH;
> -    filename += fname;
> +  if (0 == fname.size()) {
> +    TSError("%s: no config filename provided", PLUGIN_NAME);
> +    return false;
> +
> +  }
> +
> +  if (fname[0] != '/') {
> +    filename = TSConfigDirGet();
> +    filename += "/" + fname;
> +  } else {
> +    filename = fname;
>   }
> 
>   f.open(filename.c_str(), std::ios::in);
> @@ -95,8 +145,6 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
>     return false;
>   }
> 
> -  TSDebug(PLUGIN_NAME, "Loading header_rewrite config from %s", filename.c_str());
> -
>   while (!f.eof()) {
>     std::string line;
> 
> @@ -105,16 +153,19 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
>     TSDebug(PLUGIN_NAME_DBG, "Reading line: %d: %s", lineno, line.c_str());
> 
>     boost::trim(line);
> -    if (line.empty() || (line[0] == '#'))
> +    if (line.empty() || (line[0] == '#')) {
>       continue;
> +    }
> 
>     Parser p(line);  // Tokenize and parse this line
> -    if (p.empty())
> +    if (p.empty()) {
>       continue;
> +    }
> 
>     // If we are at the beginning of a new condition, save away the previous rule (but only if it has operators).
> -    if (p.is_cond() && add_rule(rule, conf))
> +    if (p.is_cond() && add_rule(rule)) {
>       rule = NULL;
> +    }
> 
>     if (NULL == rule) {
>       rule = new RuleSet();
> @@ -151,12 +202,12 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
>   }
> 
>   // Add the last rule (possibly the only rule)
> -  add_rule(rule, conf);
> +  add_rule(rule);
> 
>   // Collect all resource IDs that we need
>   for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> -    if (conf->rules[i]) {
> -      conf->resids[i] = conf->rules[i]->get_all_resource_ids();
> +    if (_rules[i]) {
> +      _resids[i] = _rules[i]->get_all_resource_ids();
>     }
>   }
> 
> @@ -174,7 +225,6 @@ cont_rewrite_headers(TSCont contp, TSEvent event, void *edata)
>   TSHttpHookID hook = TS_HTTP_LAST_HOOK;
>   RulesConfig* conf = static_cast<RulesConfig*>(TSContDataGet(contp));
> 
> -  // Get the resources necessary to process this event
>   switch (event) {
>   case TS_EVENT_HTTP_READ_RESPONSE_HDR:
>     hook = TS_HTTP_READ_RESPONSE_HDR_HOOK;
> @@ -198,10 +248,11 @@ cont_rewrite_headers(TSCont contp, TSEvent event, void *edata)
>   }
> 
>   if (hook != TS_HTTP_LAST_HOOK) {
> -    const RuleSet* rule = conf->rules[hook];
> +    const RuleSet* rule = conf->rule(hook);
>     Resources res(txnp, contp);
> 
> -    res.gather(conf->resids[hook], hook);
> +    // Get the resources necessary to process this event
> +    res.gather(conf->resid(hook), hook);
> 
>     // Evaluation of all rules. This code is sort of duplicate in DoRemap as well.
>     while (rule) {
> @@ -236,7 +287,6 @@ TSPluginInit(int argc, const char *argv[])
>   if (TS_SUCCESS != TSPluginRegister(TS_SDK_VERSION_3_0 , &info)) {
>     TSError("%s: plugin registration failed.\n", PLUGIN_NAME);
>   }
> -  TSDebug(PLUGIN_NAME, "number of arguments: %d", argc);
> 
>   // Parse the global config file(s). All rules are just appended
>   // to the "global" Rules configuration.
> @@ -246,10 +296,12 @@ TSPluginInit(int argc, const char *argv[])
>   for (int i=1; i < argc; ++i) {
>     // Parse the config file(s). Note that multiple config files are
>     // just appended to the configurations.
> -    if (!parse_config(argv[i], TS_HTTP_READ_RESPONSE_HDR_HOOK, conf)) {
> -      TSError("header_rewrite: failed to parse configuration file %s", argv[argc]);
> -    } else {
> +    TSDebug(PLUGIN_NAME, "Loading global configuration file %s", argv[i]);
> +    if (conf->parse_config(argv[i], TS_HTTP_READ_RESPONSE_HDR_HOOK)) {
> +      TSDebug(PLUGIN_NAME, "Succesfully loaded global config file %s", argv[i]);
>       got_config = true;
> +    } else {
> +      TSError("header_rewrite: failed to parse configuration file %s", argv[i]);
>     }
>   }
> 
> @@ -258,8 +310,8 @@ TSPluginInit(int argc, const char *argv[])
>     TSContDataSet(contp, conf);
> 
>     for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> -      if (conf->rules[i]) {
> -        TSDebug(PLUGIN_NAME, "adding hook: %d", i);
> +      if (conf->rule(i)) {
> +        TSDebug(PLUGIN_NAME, "Adding global ruleset to hook=%s", HOOK_NAMES[i]);
>         TSHttpHookAdd(static_cast<TSHttpHookID>(i), contp);
>       }
>     }
> @@ -293,7 +345,7 @@ TSRemapInit(TSRemapInterface *api_info, char *errbuf, int errbuf_size)
>     return TS_ERROR;
>   }
> 
> -  TSDebug(PLUGIN_NAME, "remap plugin is successfully initialized");
> +  TSDebug(PLUGIN_NAME, "Remap plugin is successfully initialized");
>   return TS_SUCCESS;
> }
> 
> @@ -301,7 +353,7 @@ TSRemapInit(TSRemapInterface *api_info, char *errbuf, int errbuf_size)
> TSReturnCode
> TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf ATS_UNUSED */, int /* errbuf_size ATS_UNUSED */)
> {
> -  TSDebug(PLUGIN_NAME, "initializing the remap plugin header_rewrite");
> +  TSDebug(PLUGIN_NAME, "Instantiating a new remap.config plugin rule");
> 
>   if (argc < 3) {
>     TSError("%s: Unable to create remap instance, need config file", PLUGIN_NAME);
> @@ -311,14 +363,25 @@ TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf ATS_UNUSE
>   RulesConfig* conf = new RulesConfig;
> 
>   for (int i=2; i < argc; ++i) {
> -    if (!parse_config(argv[i], TS_REMAP_PSEUDO_HOOK, conf)) {
> +    TSDebug(PLUGIN_NAME, "Loading remap configuration file %s", argv[i]);
> +    if (!conf->parse_config(argv[i], TS_REMAP_PSEUDO_HOOK)) {
>       TSError("%s: Unable to create remap instance", PLUGIN_NAME);
>       return TS_ERROR;
> +    } else {
> +      TSDebug(PLUGIN_NAME, "Succesfully loaded remap config file %s", argv[i]);
>     }
>   }
> 
> -  *ih = conf;
> -  TSDebug(PLUGIN_NAME, "added header_rewrite remap rule set");
> +  // For debugging only
> +  if (TSIsDebugTagSet(PLUGIN_NAME)) {
> +    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> +      if (conf->rule(i)) {
> +        TSDebug(PLUGIN_NAME, "Adding remap ruleset to hook=%s", HOOK_NAMES[i]);
> +      }
> +    }
> +  }
> +
> +  *ih = static_cast<void*>(conf);
> 
>   return TS_SUCCESS;
> }
> @@ -338,57 +401,46 @@ TSRemapDeleteInstance(void *ih)
> TSRemapStatus
> TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
> {
> -  TSRemapStatus rval = TSREMAP_NO_REMAP;
> -
> +  // Make sure things are properly setup (this should never happen)
>   if (NULL == ih) {
>     TSDebug(PLUGIN_NAME, "No Rules configured, falling back to default");
> -    return rval;
> -  } else {
> -    RulesConfig* conf = static_cast<RulesConfig*>(ih);
> -
> -    // TODO: We should optimize this to not create continuations every time,
> -    // but instead save the continuations, with configurations, as the per
> -    // remap instance data.
> +    return TSREMAP_NO_REMAP;
> +  }
> 
> -    // Go through all hooks we support, and setup the txn hook(s) as necessary
> -    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> -      TSCont contp = NULL;
> +  TSRemapStatus rval = TSREMAP_NO_REMAP;
> +  RulesConfig* conf = static_cast<RulesConfig*>(ih);
> 
> -      if (conf->rules[i]) {
> -        if (NULL == contp) {
> -          contp = TSContCreate(cont_rewrite_headers, NULL);
> -          TSContDataSet(contp, conf);
> -        }
> -        TSHttpTxnHookAdd(rh, static_cast<TSHttpHookID>(i), contp);
> -        TSDebug(PLUGIN_NAME, "activated transaction hook via remap.config: new hook=%d", i);
> -      }
> +  // Go through all hooks we support, and setup the txn hook(s) as necessary
> +  for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> +    if (conf->rule(i)) {
> +      TSHttpTxnHookAdd(rh, static_cast<TSHttpHookID>(i), conf->continuation());
> +      TSDebug(PLUGIN_NAME, "Added remapped TXN hook=%s", HOOK_NAMES[i]);
>     }
> +  }
> 
> -    // Now handle the remap specific rules for the "remap hook" (which is not a real hook).
> -    // This avoids scheduling an additional continuation for a very common case.
> -    RuleSet* rule = conf->rules[TS_REMAP_PSEUDO_HOOK];
> -    Resources res(rh, rri);
> -
> -    // res.gather(conf->resids[TS_REMAP_PSEUDO_HOOK], TS_REMAP_PSEUDO_HOOK);
> -    res.gather(RSRC_CLIENT_REQUEST_HEADERS, TS_REMAP_PSEUDO_HOOK);
> +  // Now handle the remap specific rules for the "remap hook" (which is not a real hook).
> +  // This is sufficiently differen than the normal cont_rewrite_headers() callback, and
> +  // we can't (shouldn't) schedule this as a TXN hook.
> +  RuleSet* rule = conf->rule(TS_REMAP_PSEUDO_HOOK);
> +  Resources res(rh, rri);
> 
> -    // Evaluation. This code is duplicated sort of, should we merge with the continuation evaluator ?
> -    while (rule) {
> -      if (rule->eval(res)) {
> -        OperModifiers rt = rule->exec(res);
> +  res.gather(RSRC_CLIENT_REQUEST_HEADERS, TS_REMAP_PSEUDO_HOOK);
> +  while (rule) {
> +    if (rule->eval(res)) {
> +      OperModifiers rt = rule->exec(res);
> 
> -        if (res.changed_url == true)
> -          rval = TSREMAP_DID_REMAP;
> +      if (res.changed_url == true) {
> +        rval = TSREMAP_DID_REMAP;
> +      }
> 
> -        if (rule->last() || (rt & OPER_LAST)) {
> -          break; // Conditional break, force a break with [L]
> -        }
> +      if (rule->last() || (rt & OPER_LAST)) {
> +        break; // Conditional break, force a break with [L]
>       }
> -      rule = rule->next;
>     }
> +    rule = rule->next;
>   }
> 
> -  TSDebug(PLUGIN_NAME, "returing with status: %d", rval);
> +  TSDebug(PLUGIN_NAME_DBG, "Returing from TSRemapDoRemap with status: %d", rval);
>   return rval;
> }
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/lulu.h
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/lulu.h b/plugins/header_rewrite/lulu.h
> index 0dd3cb1..eb89031 100644
> --- a/plugins/header_rewrite/lulu.h
> +++ b/plugins/header_rewrite/lulu.h
> @@ -43,8 +43,8 @@
> #error "Define barriers"
> #endif
> 
> -extern const char* PLUGIN_NAME;
> -extern const char* PLUGIN_NAME_DBG;
> +extern const char PLUGIN_NAME[];
> +extern const char PLUGIN_NAME_DBG[];
> 
> 
> // From google styleguide: http://google-styleguide.googlecode.com/svn/trunk/cppguide.xml
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/operators.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/operators.cc b/plugins/header_rewrite/operators.cc
> index d6a60e5..3b2cd27 100644
> --- a/plugins/header_rewrite/operators.cc
> +++ b/plugins/header_rewrite/operators.cc
> @@ -361,7 +361,7 @@ OperatorRMHeader::exec(const Resources& res) const
>     TSDebug(PLUGIN_NAME, "OperatorRMHeader::exec() invoked on header %s", _header.c_str());
>     field_loc = TSMimeHdrFieldFind(res.bufp, res.hdr_loc, _header.c_str(), _header.size());
>     while (field_loc) {
> -      TSDebug(PLUGIN_NAME, "\tdeleting header %s", _header.c_str());
> +      TSDebug(PLUGIN_NAME, "   Deleting header %s", _header.c_str());
>       tmp = TSMimeHdrFieldNextDup(res.bufp, res.hdr_loc, field_loc);
>       TSMimeHdrFieldDestroy(res.bufp, res.hdr_loc, field_loc);
>       TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
> @@ -405,7 +405,7 @@ OperatorAddHeader::exec(const Resources& res) const
> 
>     if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(res.bufp, res.hdr_loc, _header.c_str(), _header.size(), &field_loc)) {
>       if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
> -        TSDebug(PLUGIN_NAME, "   adding header %s", _header.c_str());
> +        TSDebug(PLUGIN_NAME, "   Adding header %s", _header.c_str());
>         TSMimeHdrFieldAppend(res.bufp, res.hdr_loc, field_loc);
>       }
>       TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
> @@ -445,7 +445,7 @@ OperatorSetHeader::exec(const Resources& res) const
>       // No existing header, so create one
>       if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(res.bufp, res.hdr_loc, _header.c_str(), _header.size(), &field_loc)) {
>         if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
> -          TSDebug(PLUGIN_NAME, "   adding header %s", _header.c_str());
> +          TSDebug(PLUGIN_NAME, "   Adding header %s", _header.c_str());
>           TSMimeHdrFieldAppend(res.bufp, res.hdr_loc, field_loc);
>         }
>         TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
> @@ -458,7 +458,7 @@ OperatorSetHeader::exec(const Resources& res) const
>         if (first) {
>           first = false;
>           if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
> -            TSDebug(PLUGIN_NAME, "   overwriting header %s", _header.c_str());
> +            TSDebug(PLUGIN_NAME, "   Overwriting header %s", _header.c_str());
>           }
>         } else {
>           TSMimeHdrFieldDestroy(res.bufp, res.hdr_loc, field_loc);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/parser.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/parser.cc b/plugins/header_rewrite/parser.cc
> index cdff4ee..a389858 100644
> --- a/plugins/header_rewrite/parser.cc
> +++ b/plugins/header_rewrite/parser.cc
> @@ -98,7 +98,7 @@ Parser::preprocess(std::vector<std::string>& tokens)
> Parser::Parser(const std::string& line) :
>   _cond(false), _empty(false)
> {
> -  TSDebug("header_rewrite_dbg", "Calling CTOR for Parser");
> +  TSDebug(PLUGIN_NAME_DBG, "Calling CTOR for Parser");
> 
>   if (line[0] == '#') {
>     _empty = true;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/resources.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/resources.cc b/plugins/header_rewrite/resources.cc
> index d89149a..eb4afaf 100644
> --- a/plugins/header_rewrite/resources.cc
> +++ b/plugins/header_rewrite/resources.cc
> @@ -24,10 +24,15 @@
> #include "resources.h"
> #include "lulu.h"
> 
> +// This is defined in header_rewrite.cc, used here for debugging
> +extern char* HOOK_NAMES[];
> +
> +
> +// Collect all resources
> void
> Resources::gather(const ResourceIDs ids, TSHttpHookID hook)
> {
> -  TSDebug(PLUGIN_NAME, "Building resource structure for hook (%d)", hook);
> +  TSDebug(PLUGIN_NAME, "Building resources, hook=%s", HOOK_NAMES[hook]);
> 
>   // If we need the client request headers, make sure it's also available in the client vars.
>   if (ids & RSRC_CLIENT_REQUEST_HEADERS) {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/resources.h
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/resources.h b/plugins/header_rewrite/resources.h
> index cb9d062..5bda0da 100644
> --- a/plugins/header_rewrite/resources.h
> +++ b/plugins/header_rewrite/resources.h
> @@ -60,7 +60,6 @@ public:
>       _rri(rri), changed_url(false), _ready(false)
>   {
>     TSDebug(PLUGIN_NAME_DBG, "Calling CTOR for Resources (RemapAPI)");
> -    TSDebug(PLUGIN_NAME, "rri: %p", _rri);
>   }
> 
>   ~Resources() { destroy(); }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/ruleset.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/ruleset.cc b/plugins/header_rewrite/ruleset.cc
> index 9c6bc06..e210e47 100644
> --- a/plugins/header_rewrite/ruleset.cc
> +++ b/plugins/header_rewrite/ruleset.cc
> @@ -47,7 +47,7 @@ RuleSet::add_condition(Parser& p)
>   Condition* c = condition_factory(p.get_op());
> 
>   if (NULL != c) {
> -    TSDebug(PLUGIN_NAME, "Adding condition: %%{%s} with arg: %s\n", p.get_op().c_str(), p.get_arg().c_str());
> +    TSDebug(PLUGIN_NAME, "   Adding condition: %%{%s} with arg: %s\n", p.get_op().c_str(), p.get_arg().c_str());
>     c->initialize(p);
>     if (!c->set_hook(_hook)) {
>       TSError("%s: can't use this condition in this hook", PLUGIN_NAME);
> @@ -73,7 +73,7 @@ RuleSet::add_operator(Parser& p)
> 
>   if (NULL != o) {
>     // TODO: This should be extended to show both the "argument" and the "value" (if both are used)
> -    TSDebug(PLUGIN_NAME, "Adding operator: %s(%s)\n", p.get_op().c_str(), p.get_arg().c_str());
> +    TSDebug(PLUGIN_NAME, "   Adding operator: %s(%s)\n", p.get_op().c_str(), p.get_arg().c_str());
>     o->initialize(p);
>     if (!o->set_hook(_hook)) {
>       TSError("%s: can't use this operator in this hook", PLUGIN_NAME);
> 


Re: [1/2] git commit: TS-2522 Better hook management, and cleanup

Posted by James Peach <jp...@apache.org>.
On Mar 4, 2014, at 12:19 AM, zwoop@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master 17f680231 -> 4f0b0b447
> 
> 
> TS-2522 Better hook management, and cleanup
> 
> This also does
> 
>     - Add support for system wide config directory as default
>     - Better debug messages (textual hook names etc.)
>     - Less overhead on creating TXN hooks
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/31792d2a
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/31792d2a
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/31792d2a
> 
> Branch: refs/heads/master
> Commit: 31792d2aae9519e0feb66ab9e80b1a9d5d51d7de
> Parents: 17f6802
> Author: Leif Hedstrom <zw...@apache.org>
> Authored: Thu Feb 27 19:55:53 2014 -0700
> Committer: Leif Hedstrom <zw...@apache.org>
> Committed: Tue Mar 4 01:13:59 2014 -0700
> 
> ----------------------------------------------------------------------
> CHANGES                                  |   3 +
> plugins/header_rewrite/header_rewrite.cc | 216 ++++++++++++++++----------
> plugins/header_rewrite/lulu.h            |   4 +-
> plugins/header_rewrite/operators.cc      |   8 +-
> plugins/header_rewrite/parser.cc         |   2 +-
> plugins/header_rewrite/resources.cc      |   7 +-
> plugins/header_rewrite/resources.h       |   1 -
> plugins/header_rewrite/ruleset.cc        |   4 +-
> 8 files changed, 152 insertions(+), 93 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/CHANGES
> ----------------------------------------------------------------------
> diff --git a/CHANGES b/CHANGES
> index 3a83537..363ab16 100644
> --- a/CHANGES
> +++ b/CHANGES
> @@ -1,6 +1,9 @@
>                                                          -*- coding: utf-8 -*-
> Changes with Apache Traffic Server 5.0.0
> 
> +  *) [TS-2522] Better hook management for header_rewrite plugin, and some
> +   cleanup.
> +
>   *) [TS-2169] Add SSL statistics
>    Author: Ron Barber <rb...@yahoo-inc.com>
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/header_rewrite.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/header_rewrite.cc b/plugins/header_rewrite/header_rewrite.cc
> index 6fc7960..614d287 100644
> --- a/plugins/header_rewrite/header_rewrite.cc
> +++ b/plugins/header_rewrite/header_rewrite.cc
> @@ -27,40 +27,83 @@
> #include "resources.h"
> 
> // Debugs
> -const char* PLUGIN_NAME = "header_rewrite";
> -const char* PLUGIN_NAME_DBG = "header_rewrite_dbg";
> +const char PLUGIN_NAME[] = "header_rewrite";
> +const char PLUGIN_NAME_DBG[] = "dbg_header_rewrite";
> +
> +const char* HOOK_NAMES[] = {

You should do some kind of static assert to force a build break if new hooks are added.

> +  "TS_HTTP_READ_REQUEST_HDR_HOOK",
> +  "TS_HTTP_OS_DNS_HOOK",
> +  "TS_HTTP_SEND_REQUEST_HDR_HOOK",
> +  "TS_HTTP_READ_CACHE_HDR_HOOK",
> +  "TS_HTTP_READ_RESPONSE_HDR_HOOK",
> +  "TS_HTTP_SEND_RESPONSE_HDR_HOOK",
> +  "TS_HTTP_REQUEST_TRANSFORM_HOOK",
> +  "TS_HTTP_RESPONSE_TRANSFORM_HOOK",
> +  "TS_HTTP_SELECT_ALT_HOOK",
> +  "TS_HTTP_TXN_START_HOOK",
> +  "TS_HTTP_TXN_CLOSE_HOOK",
> +  "TS_HTTP_SSN_START_HOOK",
> +  "TS_HTTP_SSN_CLOSE_HOOK",
> +  "TS_HTTP_CACHE_LOOKUP_COMPLETE_HOOK",
> +  "TS_HTTP_PRE_REMAP_HOOK",
> +  "TS_HTTP_POST_REMAP_HOOK",
> +  "TS_HTTP_RESPONSE_CLIENT_HOOK",
> +  "TS_HTTP_LAST_HOOK"
> +};
> +
> +
> +// Forward declaration for the main continuation.
> +static int cont_rewrite_headers(TSCont, TSEvent, void *);
> 
> -static const char* DEFAULT_CONF_PATH = "/usr/local/etc/header_rewrite/";
> 
> // Simple wrapper around a configuration file / set. This is useful such that
> // we can reuse most of the code for both global and per-remap rule sets.
> -struct RulesConfig
> +class RulesConfig
> {
> +public:
>   RulesConfig()
>   {
> -    memset(rules, 0, sizeof(rules));
> -    memset(resids, 0, sizeof(resids));
> +    memset(_rules, 0, sizeof(_rules));
> +    memset(_resids, 0, sizeof(_resids));
> +
> +    _cont = TSContCreate(cont_rewrite_headers, NULL);
> +    TSContDataSet(_cont, static_cast<void*>(this));
>   }
> 
>   ~RulesConfig()
>   {
> -    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i)
> -      delete rules[i];
> +    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> +      delete _rules[i];
> +    }
> +
> +    TSContDestroy(_cont);
>   }
> 
> -  RuleSet* rules[TS_HTTP_LAST_HOOK+1];
> -  ResourceIDs resids[TS_HTTP_LAST_HOOK+1];
> +  TSCont continuation() const { return _cont; }
> +
> +  ResourceIDs resid(int hook) const { return _resids[hook]; }
> +  RuleSet* rule(int hook) const { return _rules[hook]; }
> +
> +  bool parse_config(const std::string fname, TSHttpHookID default_hook);
> +
> +private:
> +  bool add_rule(RuleSet* rule);
> +
> +  TSCont _cont;
> +  RuleSet* _rules[TS_HTTP_LAST_HOOK+1];
> +  ResourceIDs _resids[TS_HTTP_LAST_HOOK+1];
> };
> 
> // Helper function to add a rule to the rulesets
> -static bool
> -add_rule(RuleSet* rule, RulesConfig *conf) {
> +bool
> +RulesConfig::add_rule(RuleSet* rule)
> +{
>   if (rule && rule->has_operator()) {
> -    TSDebug(PLUGIN_NAME, "Adding rule to hook=%d\n", rule->get_hook());
> -    if (NULL == conf->rules[rule->get_hook()]) {
> -      conf->rules[rule->get_hook()] = rule;
> +    TSDebug(PLUGIN_NAME_DBG, "   Adding rule to hook=%s\n", HOOK_NAMES[rule->get_hook()]);
> +    if (NULL == _rules[rule->get_hook()]) {
> +      _rules[rule->get_hook()] = rule;
>     } else {
> -      conf->rules[rule->get_hook()]->append(rule);
> +      _rules[rule->get_hook()]->append(rule);
>     }
>     return true;
>   }
> @@ -76,17 +119,24 @@ add_rule(RuleSet* rule, RulesConfig *conf) {
> // anyways (or reload for remap.config), so not really in the critical path.
> //
> bool
> -parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *conf)
> +RulesConfig::parse_config(const std::string fname, TSHttpHookID default_hook)
> {
>   RuleSet* rule = NULL;
> -  std::string filename = fname;
> +  std::string filename;
>   std::ifstream f;
>   int lineno = 0;
> 
> -  // Try appending the default conf path if the fname doesn't exist.
> -  if (0 != access(filename.c_str(), R_OK)) {
> -    filename = DEFAULT_CONF_PATH;
> -    filename += fname;
> +  if (0 == fname.size()) {
> +    TSError("%s: no config filename provided", PLUGIN_NAME);
> +    return false;
> +
> +  }
> +
> +  if (fname[0] != '/') {
> +    filename = TSConfigDirGet();
> +    filename += "/" + fname;
> +  } else {
> +    filename = fname;
>   }
> 
>   f.open(filename.c_str(), std::ios::in);
> @@ -95,8 +145,6 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
>     return false;
>   }
> 
> -  TSDebug(PLUGIN_NAME, "Loading header_rewrite config from %s", filename.c_str());
> -
>   while (!f.eof()) {
>     std::string line;
> 
> @@ -105,16 +153,19 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
>     TSDebug(PLUGIN_NAME_DBG, "Reading line: %d: %s", lineno, line.c_str());
> 
>     boost::trim(line);
> -    if (line.empty() || (line[0] == '#'))
> +    if (line.empty() || (line[0] == '#')) {
>       continue;
> +    }
> 
>     Parser p(line);  // Tokenize and parse this line
> -    if (p.empty())
> +    if (p.empty()) {
>       continue;
> +    }
> 
>     // If we are at the beginning of a new condition, save away the previous rule (but only if it has operators).
> -    if (p.is_cond() && add_rule(rule, conf))
> +    if (p.is_cond() && add_rule(rule)) {
>       rule = NULL;
> +    }
> 
>     if (NULL == rule) {
>       rule = new RuleSet();
> @@ -151,12 +202,12 @@ parse_config(const std::string fname, TSHttpHookID default_hook, RulesConfig *co
>   }
> 
>   // Add the last rule (possibly the only rule)
> -  add_rule(rule, conf);
> +  add_rule(rule);
> 
>   // Collect all resource IDs that we need
>   for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> -    if (conf->rules[i]) {
> -      conf->resids[i] = conf->rules[i]->get_all_resource_ids();
> +    if (_rules[i]) {
> +      _resids[i] = _rules[i]->get_all_resource_ids();
>     }
>   }
> 
> @@ -174,7 +225,6 @@ cont_rewrite_headers(TSCont contp, TSEvent event, void *edata)
>   TSHttpHookID hook = TS_HTTP_LAST_HOOK;
>   RulesConfig* conf = static_cast<RulesConfig*>(TSContDataGet(contp));
> 
> -  // Get the resources necessary to process this event
>   switch (event) {
>   case TS_EVENT_HTTP_READ_RESPONSE_HDR:
>     hook = TS_HTTP_READ_RESPONSE_HDR_HOOK;
> @@ -198,10 +248,11 @@ cont_rewrite_headers(TSCont contp, TSEvent event, void *edata)
>   }
> 
>   if (hook != TS_HTTP_LAST_HOOK) {
> -    const RuleSet* rule = conf->rules[hook];
> +    const RuleSet* rule = conf->rule(hook);
>     Resources res(txnp, contp);
> 
> -    res.gather(conf->resids[hook], hook);
> +    // Get the resources necessary to process this event
> +    res.gather(conf->resid(hook), hook);
> 
>     // Evaluation of all rules. This code is sort of duplicate in DoRemap as well.
>     while (rule) {
> @@ -236,7 +287,6 @@ TSPluginInit(int argc, const char *argv[])
>   if (TS_SUCCESS != TSPluginRegister(TS_SDK_VERSION_3_0 , &info)) {
>     TSError("%s: plugin registration failed.\n", PLUGIN_NAME);
>   }
> -  TSDebug(PLUGIN_NAME, "number of arguments: %d", argc);
> 
>   // Parse the global config file(s). All rules are just appended
>   // to the "global" Rules configuration.
> @@ -246,10 +296,12 @@ TSPluginInit(int argc, const char *argv[])
>   for (int i=1; i < argc; ++i) {
>     // Parse the config file(s). Note that multiple config files are
>     // just appended to the configurations.
> -    if (!parse_config(argv[i], TS_HTTP_READ_RESPONSE_HDR_HOOK, conf)) {
> -      TSError("header_rewrite: failed to parse configuration file %s", argv[argc]);
> -    } else {
> +    TSDebug(PLUGIN_NAME, "Loading global configuration file %s", argv[i]);
> +    if (conf->parse_config(argv[i], TS_HTTP_READ_RESPONSE_HDR_HOOK)) {
> +      TSDebug(PLUGIN_NAME, "Succesfully loaded global config file %s", argv[i]);
>       got_config = true;
> +    } else {
> +      TSError("header_rewrite: failed to parse configuration file %s", argv[i]);
>     }
>   }
> 
> @@ -258,8 +310,8 @@ TSPluginInit(int argc, const char *argv[])
>     TSContDataSet(contp, conf);
> 
>     for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> -      if (conf->rules[i]) {
> -        TSDebug(PLUGIN_NAME, "adding hook: %d", i);
> +      if (conf->rule(i)) {
> +        TSDebug(PLUGIN_NAME, "Adding global ruleset to hook=%s", HOOK_NAMES[i]);
>         TSHttpHookAdd(static_cast<TSHttpHookID>(i), contp);
>       }
>     }
> @@ -293,7 +345,7 @@ TSRemapInit(TSRemapInterface *api_info, char *errbuf, int errbuf_size)
>     return TS_ERROR;
>   }
> 
> -  TSDebug(PLUGIN_NAME, "remap plugin is successfully initialized");
> +  TSDebug(PLUGIN_NAME, "Remap plugin is successfully initialized");
>   return TS_SUCCESS;
> }
> 
> @@ -301,7 +353,7 @@ TSRemapInit(TSRemapInterface *api_info, char *errbuf, int errbuf_size)
> TSReturnCode
> TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf ATS_UNUSED */, int /* errbuf_size ATS_UNUSED */)
> {
> -  TSDebug(PLUGIN_NAME, "initializing the remap plugin header_rewrite");
> +  TSDebug(PLUGIN_NAME, "Instantiating a new remap.config plugin rule");
> 
>   if (argc < 3) {
>     TSError("%s: Unable to create remap instance, need config file", PLUGIN_NAME);
> @@ -311,14 +363,25 @@ TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf ATS_UNUSE
>   RulesConfig* conf = new RulesConfig;
> 
>   for (int i=2; i < argc; ++i) {
> -    if (!parse_config(argv[i], TS_REMAP_PSEUDO_HOOK, conf)) {
> +    TSDebug(PLUGIN_NAME, "Loading remap configuration file %s", argv[i]);
> +    if (!conf->parse_config(argv[i], TS_REMAP_PSEUDO_HOOK)) {
>       TSError("%s: Unable to create remap instance", PLUGIN_NAME);
>       return TS_ERROR;
> +    } else {
> +      TSDebug(PLUGIN_NAME, "Succesfully loaded remap config file %s", argv[i]);
>     }
>   }
> 
> -  *ih = conf;
> -  TSDebug(PLUGIN_NAME, "added header_rewrite remap rule set");
> +  // For debugging only
> +  if (TSIsDebugTagSet(PLUGIN_NAME)) {
> +    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> +      if (conf->rule(i)) {
> +        TSDebug(PLUGIN_NAME, "Adding remap ruleset to hook=%s", HOOK_NAMES[i]);
> +      }
> +    }
> +  }
> +
> +  *ih = static_cast<void*>(conf);
> 
>   return TS_SUCCESS;
> }
> @@ -338,57 +401,46 @@ TSRemapDeleteInstance(void *ih)
> TSRemapStatus
> TSRemapDoRemap(void *ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
> {
> -  TSRemapStatus rval = TSREMAP_NO_REMAP;
> -
> +  // Make sure things are properly setup (this should never happen)
>   if (NULL == ih) {
>     TSDebug(PLUGIN_NAME, "No Rules configured, falling back to default");
> -    return rval;
> -  } else {
> -    RulesConfig* conf = static_cast<RulesConfig*>(ih);
> -
> -    // TODO: We should optimize this to not create continuations every time,
> -    // but instead save the continuations, with configurations, as the per
> -    // remap instance data.
> +    return TSREMAP_NO_REMAP;
> +  }
> 
> -    // Go through all hooks we support, and setup the txn hook(s) as necessary
> -    for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> -      TSCont contp = NULL;
> +  TSRemapStatus rval = TSREMAP_NO_REMAP;
> +  RulesConfig* conf = static_cast<RulesConfig*>(ih);
> 
> -      if (conf->rules[i]) {
> -        if (NULL == contp) {
> -          contp = TSContCreate(cont_rewrite_headers, NULL);
> -          TSContDataSet(contp, conf);
> -        }
> -        TSHttpTxnHookAdd(rh, static_cast<TSHttpHookID>(i), contp);
> -        TSDebug(PLUGIN_NAME, "activated transaction hook via remap.config: new hook=%d", i);
> -      }
> +  // Go through all hooks we support, and setup the txn hook(s) as necessary
> +  for (int i=TS_HTTP_READ_REQUEST_HDR_HOOK; i<TS_HTTP_LAST_HOOK; ++i) {
> +    if (conf->rule(i)) {
> +      TSHttpTxnHookAdd(rh, static_cast<TSHttpHookID>(i), conf->continuation());
> +      TSDebug(PLUGIN_NAME, "Added remapped TXN hook=%s", HOOK_NAMES[i]);
>     }
> +  }
> 
> -    // Now handle the remap specific rules for the "remap hook" (which is not a real hook).
> -    // This avoids scheduling an additional continuation for a very common case.
> -    RuleSet* rule = conf->rules[TS_REMAP_PSEUDO_HOOK];
> -    Resources res(rh, rri);
> -
> -    // res.gather(conf->resids[TS_REMAP_PSEUDO_HOOK], TS_REMAP_PSEUDO_HOOK);
> -    res.gather(RSRC_CLIENT_REQUEST_HEADERS, TS_REMAP_PSEUDO_HOOK);
> +  // Now handle the remap specific rules for the "remap hook" (which is not a real hook).
> +  // This is sufficiently differen than the normal cont_rewrite_headers() callback, and
> +  // we can't (shouldn't) schedule this as a TXN hook.
> +  RuleSet* rule = conf->rule(TS_REMAP_PSEUDO_HOOK);
> +  Resources res(rh, rri);
> 
> -    // Evaluation. This code is duplicated sort of, should we merge with the continuation evaluator ?
> -    while (rule) {
> -      if (rule->eval(res)) {
> -        OperModifiers rt = rule->exec(res);
> +  res.gather(RSRC_CLIENT_REQUEST_HEADERS, TS_REMAP_PSEUDO_HOOK);
> +  while (rule) {
> +    if (rule->eval(res)) {
> +      OperModifiers rt = rule->exec(res);
> 
> -        if (res.changed_url == true)
> -          rval = TSREMAP_DID_REMAP;
> +      if (res.changed_url == true) {
> +        rval = TSREMAP_DID_REMAP;
> +      }
> 
> -        if (rule->last() || (rt & OPER_LAST)) {
> -          break; // Conditional break, force a break with [L]
> -        }
> +      if (rule->last() || (rt & OPER_LAST)) {
> +        break; // Conditional break, force a break with [L]
>       }
> -      rule = rule->next;
>     }
> +    rule = rule->next;
>   }
> 
> -  TSDebug(PLUGIN_NAME, "returing with status: %d", rval);
> +  TSDebug(PLUGIN_NAME_DBG, "Returing from TSRemapDoRemap with status: %d", rval);
>   return rval;
> }
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/lulu.h
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/lulu.h b/plugins/header_rewrite/lulu.h
> index 0dd3cb1..eb89031 100644
> --- a/plugins/header_rewrite/lulu.h
> +++ b/plugins/header_rewrite/lulu.h
> @@ -43,8 +43,8 @@
> #error "Define barriers"
> #endif
> 
> -extern const char* PLUGIN_NAME;
> -extern const char* PLUGIN_NAME_DBG;
> +extern const char PLUGIN_NAME[];
> +extern const char PLUGIN_NAME_DBG[];
> 
> 
> // From google styleguide: http://google-styleguide.googlecode.com/svn/trunk/cppguide.xml
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/operators.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/operators.cc b/plugins/header_rewrite/operators.cc
> index d6a60e5..3b2cd27 100644
> --- a/plugins/header_rewrite/operators.cc
> +++ b/plugins/header_rewrite/operators.cc
> @@ -361,7 +361,7 @@ OperatorRMHeader::exec(const Resources& res) const
>     TSDebug(PLUGIN_NAME, "OperatorRMHeader::exec() invoked on header %s", _header.c_str());
>     field_loc = TSMimeHdrFieldFind(res.bufp, res.hdr_loc, _header.c_str(), _header.size());
>     while (field_loc) {
> -      TSDebug(PLUGIN_NAME, "\tdeleting header %s", _header.c_str());
> +      TSDebug(PLUGIN_NAME, "   Deleting header %s", _header.c_str());
>       tmp = TSMimeHdrFieldNextDup(res.bufp, res.hdr_loc, field_loc);
>       TSMimeHdrFieldDestroy(res.bufp, res.hdr_loc, field_loc);
>       TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
> @@ -405,7 +405,7 @@ OperatorAddHeader::exec(const Resources& res) const
> 
>     if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(res.bufp, res.hdr_loc, _header.c_str(), _header.size(), &field_loc)) {
>       if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
> -        TSDebug(PLUGIN_NAME, "   adding header %s", _header.c_str());
> +        TSDebug(PLUGIN_NAME, "   Adding header %s", _header.c_str());
>         TSMimeHdrFieldAppend(res.bufp, res.hdr_loc, field_loc);
>       }
>       TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
> @@ -445,7 +445,7 @@ OperatorSetHeader::exec(const Resources& res) const
>       // No existing header, so create one
>       if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(res.bufp, res.hdr_loc, _header.c_str(), _header.size(), &field_loc)) {
>         if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
> -          TSDebug(PLUGIN_NAME, "   adding header %s", _header.c_str());
> +          TSDebug(PLUGIN_NAME, "   Adding header %s", _header.c_str());
>           TSMimeHdrFieldAppend(res.bufp, res.hdr_loc, field_loc);
>         }
>         TSHandleMLocRelease(res.bufp, res.hdr_loc, field_loc);
> @@ -458,7 +458,7 @@ OperatorSetHeader::exec(const Resources& res) const
>         if (first) {
>           first = false;
>           if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(res.bufp, res.hdr_loc, field_loc, -1, value.c_str(), value.size())) {
> -            TSDebug(PLUGIN_NAME, "   overwriting header %s", _header.c_str());
> +            TSDebug(PLUGIN_NAME, "   Overwriting header %s", _header.c_str());
>           }
>         } else {
>           TSMimeHdrFieldDestroy(res.bufp, res.hdr_loc, field_loc);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/parser.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/parser.cc b/plugins/header_rewrite/parser.cc
> index cdff4ee..a389858 100644
> --- a/plugins/header_rewrite/parser.cc
> +++ b/plugins/header_rewrite/parser.cc
> @@ -98,7 +98,7 @@ Parser::preprocess(std::vector<std::string>& tokens)
> Parser::Parser(const std::string& line) :
>   _cond(false), _empty(false)
> {
> -  TSDebug("header_rewrite_dbg", "Calling CTOR for Parser");
> +  TSDebug(PLUGIN_NAME_DBG, "Calling CTOR for Parser");
> 
>   if (line[0] == '#') {
>     _empty = true;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/resources.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/resources.cc b/plugins/header_rewrite/resources.cc
> index d89149a..eb4afaf 100644
> --- a/plugins/header_rewrite/resources.cc
> +++ b/plugins/header_rewrite/resources.cc
> @@ -24,10 +24,15 @@
> #include "resources.h"
> #include "lulu.h"
> 
> +// This is defined in header_rewrite.cc, used here for debugging
> +extern char* HOOK_NAMES[];
> +
> +
> +// Collect all resources
> void
> Resources::gather(const ResourceIDs ids, TSHttpHookID hook)
> {
> -  TSDebug(PLUGIN_NAME, "Building resource structure for hook (%d)", hook);
> +  TSDebug(PLUGIN_NAME, "Building resources, hook=%s", HOOK_NAMES[hook]);
> 
>   // If we need the client request headers, make sure it's also available in the client vars.
>   if (ids & RSRC_CLIENT_REQUEST_HEADERS) {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/resources.h
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/resources.h b/plugins/header_rewrite/resources.h
> index cb9d062..5bda0da 100644
> --- a/plugins/header_rewrite/resources.h
> +++ b/plugins/header_rewrite/resources.h
> @@ -60,7 +60,6 @@ public:
>       _rri(rri), changed_url(false), _ready(false)
>   {
>     TSDebug(PLUGIN_NAME_DBG, "Calling CTOR for Resources (RemapAPI)");
> -    TSDebug(PLUGIN_NAME, "rri: %p", _rri);
>   }
> 
>   ~Resources() { destroy(); }
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/31792d2a/plugins/header_rewrite/ruleset.cc
> ----------------------------------------------------------------------
> diff --git a/plugins/header_rewrite/ruleset.cc b/plugins/header_rewrite/ruleset.cc
> index 9c6bc06..e210e47 100644
> --- a/plugins/header_rewrite/ruleset.cc
> +++ b/plugins/header_rewrite/ruleset.cc
> @@ -47,7 +47,7 @@ RuleSet::add_condition(Parser& p)
>   Condition* c = condition_factory(p.get_op());
> 
>   if (NULL != c) {
> -    TSDebug(PLUGIN_NAME, "Adding condition: %%{%s} with arg: %s\n", p.get_op().c_str(), p.get_arg().c_str());
> +    TSDebug(PLUGIN_NAME, "   Adding condition: %%{%s} with arg: %s\n", p.get_op().c_str(), p.get_arg().c_str());
>     c->initialize(p);
>     if (!c->set_hook(_hook)) {
>       TSError("%s: can't use this condition in this hook", PLUGIN_NAME);
> @@ -73,7 +73,7 @@ RuleSet::add_operator(Parser& p)
> 
>   if (NULL != o) {
>     // TODO: This should be extended to show both the "argument" and the "value" (if both are used)
> -    TSDebug(PLUGIN_NAME, "Adding operator: %s(%s)\n", p.get_op().c_str(), p.get_arg().c_str());
> +    TSDebug(PLUGIN_NAME, "   Adding operator: %s(%s)\n", p.get_op().c_str(), p.get_arg().c_str());
>     o->initialize(p);
>     if (!o->set_hook(_hook)) {
>       TSError("%s: can't use this operator in this hook", PLUGIN_NAME);
>