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 2015/05/11 23:39:39 UTC

trafficserver git commit: [TS-3562] background_fetch: refactor some code

Repository: trafficserver
Updated Branches:
  refs/heads/master b7aafbe36 -> 632571d42


[TS-3562] background_fetch: refactor some code

elimiante the std::map, for rules, split things out into manageable
sub-modules, overall cleanup.


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

Branch: refs/heads/master
Commit: 632571d42e4bff00bd55d536794ff1be14029f94
Parents: b7aafbe
Author: Leif Hedstrom <zw...@apache.org>
Authored: Mon May 11 14:56:29 2015 -0600
Committer: Leif Hedstrom <zw...@apache.org>
Committed: Mon May 11 14:57:24 2015 -0600

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../experimental/background_fetch/Makefile.am   |   2 +-
 .../background_fetch/background_fetch.cc        | 667 ++++---------------
 .../experimental/background_fetch/configs.cc    | 124 ++++
 plugins/experimental/background_fetch/configs.h |  90 +++
 .../experimental/background_fetch/headers.cc    | 129 ++++
 plugins/experimental/background_fetch/headers.h |  32 +
 plugins/experimental/background_fetch/rules.cc  | 175 +++++
 plugins/experimental/background_fetch/rules.h   |  70 ++
 9 files changed, 745 insertions(+), 547 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 554d1c9..4e91d6f 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,9 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 6.0.0
 
+  *) [TS-3562] background_fetch: refactor some code, elimiante the std::map
+   for rules, split things out into manageable sub-modules.
+
   *) [TS-3592] Set cacheurl default config location to config directory.
 
   *) [TS-3590]: Mark H2 fetch streams as non-internal requests

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/Makefile.am
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/Makefile.am b/plugins/experimental/background_fetch/Makefile.am
index fef8d44..f841cf3 100644
--- a/plugins/experimental/background_fetch/Makefile.am
+++ b/plugins/experimental/background_fetch/Makefile.am
@@ -17,5 +17,5 @@
 include $(top_srcdir)/build/plugins.mk
 
 pkglib_LTLIBRARIES = background_fetch.la
-background_fetch_la_SOURCES = background_fetch.cc
+background_fetch_la_SOURCES = background_fetch.cc headers.cc rules.cc configs.cc
 background_fetch_la_LDFLAGS = $(TS_PLUGIN_LDFLAGS)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/background_fetch.cc
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/background_fetch.cc b/plugins/experimental/background_fetch/background_fetch.cc
index ae6b04c..e5cdb6b 100644
--- a/plugins/experimental/background_fetch/background_fetch.cc
+++ b/plugins/experimental/background_fetch/background_fetch.cc
@@ -21,319 +21,60 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 */
-
 #include <stdlib.h>
 #include <stdio.h>
 #include <string.h>
 #include <stdarg.h>
 #include <getopt.h>
-#include <netinet/in.h>
-#include <arpa/inet.h>
+
+#include <string>
 
 #include "ts/ts.h"
 #include "ts/remap.h"
-#include "ink_atomic.h"
-#include "ink_defs.h"
-
-#include <string>
-#include <set>
-#include <map>
+#include "headers.h"
+#include "rules.h"
+#include "configs.h"
 
 // Some wonkiness around compiler version and the unordered map (hash)
 #if HAVE_UNORDERED_MAP
 #include <unordered_map>
 typedef std::unordered_map<std::string, bool> OutstandingRequests;
 #else
+#include <map>
 typedef std::map<std::string, bool> OutstandingRequests;
 #endif
 
-// Constants
-const char PLUGIN_NAME[] = "background_fetch";
-
-// Types
-typedef struct {
-  bool exclude;
-  std::string ruleField;
-  std::string ruleValue;
-} BgFetchRuleStruct;
+// Global config, if we don't have a remap specific config.
+static BgFetchConfig *gConfig;
 
-typedef std::map<uint32_t, BgFetchRuleStruct> BgFetchRuleMap;
 
-class BgFetchRulesConfig
+///////////////////////////////////////////////////////////////////////////
+// Hold the global ackground fetch state. This is currently shared across all
+// configurations, as a singleton. ToDo: Would it ever make sense to do this
+// per remap rule? Probably not.
+class BgFetchState
 {
 public:
-  BgFetchRulesConfig() : _ref_count(0), _cont(NULL), _bgFetchRuleMap(NULL){};
-
-  void
-  incrRefCount()
-  {
-    ink_atomic_increment(&_ref_count, 1);
-  }
-
-  void
-  decrRefCount()
+  static BgFetchState &
+  getInstance()
   {
-    if (1 >= ink_atomic_decrement(&_ref_count, 1))
-      delete this;
+    static BgFetchState _instance;
+    return _instance;
   }
 
-  void
-  setRuleMap(BgFetchRuleMap *ruleMap)
-  {
-    _bgFetchRuleMap = ruleMap;
-  }
-
-  BgFetchRuleMap *
-  getRuleMap() const
-  {
-    return _bgFetchRuleMap;
-  }
+  ~BgFetchState() { TSMutexDestroy(_lock); }
 
   void
-  setCont(TSCont cont)
-  {
-    _cont = cont;
-  }
-
-  TSCont
-  getCont() const
+  createLog(const char *log_name)
   {
-    return _cont;
-  }
-
-private:
-  ~BgFetchRulesConfig()
-  {
-    if (_bgFetchRuleMap) {
-      delete _bgFetchRuleMap;
-    }
-    if (_cont) {
-      TSContDestroy(_cont);
-    }
-  }
-
-  volatile int _ref_count;
-  TSCont _cont;
-  BgFetchRuleMap *_bgFetchRuleMap;
-};
-
-// Global config, if we don't have a remap specific config.
-static BgFetchRuleMap gBgFetchRuleMap;
-
-
-///////////////////////////////////////////////////////////////////////////
-// Read a config file
-//
-static bool
-read_config(char *config_file, BgFetchRuleMap *ri)
-{
-  char file_path[1024];
-  TSFile file;
-
-  if (config_file == NULL) {
-    TSError("%s: invalid config file", PLUGIN_NAME);
-    return false;
-  }
-
-  TSDebug(PLUGIN_NAME, "trying to open config file in this path: %s", config_file);
-
-  file = TSfopen(config_file, "r");
-  if (file == NULL) {
-    TSDebug(PLUGIN_NAME, "Failed to open config file %s, trying rel path", config_file);
-    snprintf(file_path, sizeof(file_path), "%s/%s", TSInstallDirGet(), config_file);
-    file = TSfopen(file_path, "r");
-    if (file == NULL) {
-      TSError("%s: invalid config file", PLUGIN_NAME);
-      return false;
-    }
-  }
-
-  BgFetchRuleMap *bgFetchRuleMapP = ri;
-  char buffer[8192];
-  uint32_t index = 0;
-
-  memset(buffer, 0, sizeof(buffer));
-  while (TSfgets(file, buffer, sizeof(buffer) - 1) != NULL) {
-    char *eol = 0;
-    // make sure line was not bigger than buffer
-    if ((eol = strchr(buffer, '\n')) == NULL && (eol = strstr(buffer, "\r\n")) == NULL) {
-      TSError("%s: exclusion line too long, did not get a good line in cfg, skipping, line: %s", PLUGIN_NAME, buffer);
-      memset(buffer, 0, sizeof(buffer));
-      continue;
-    }
-    // make sure line has something useful on it
-    if (eol - buffer < 2 || buffer[0] == '#') {
-      memset(buffer, 0, sizeof(buffer));
-      continue;
-    }
-
-    char *savePtr = NULL;
-    char *cfg = strtok_r(buffer, "\n\r\n", &savePtr);
-
-    if (cfg != NULL) {
-      TSDebug(PLUGIN_NAME, "setting background_fetch exclusion criterion based on string: %s", cfg);
-      char *cfg_type = strtok_r(buffer, " ", &savePtr);
-      char *cfg_name = NULL;
-      char *cfg_value = NULL;
-      bool exclude = false;
-
-      if (cfg_type) {
-        if (!strcmp(cfg_type, "exclude")) {
-          exclude = true;
-        } else if (strcmp(cfg_type, "include")) {
-          TSError("%s: invalid specifier %s, skipping config line", PLUGIN_NAME, cfg_type);
-          memset(buffer, 0, sizeof(buffer));
-          continue;
-        }
-        cfg_name = strtok_r(NULL, " ", &savePtr);
-        if (cfg_name) {
-          cfg_value = strtok_r(NULL, " ", &savePtr);
-          if (cfg_value) {
-            if (!strcmp(cfg_name, "Content-Length")) {
-              if ((cfg_value[0] != '<') && (cfg_value[0] != '>')) {
-                TSError("%s: invalid content-len condition %s, skipping config value", PLUGIN_NAME, cfg_value);
-                memset(buffer, 0, sizeof(buffer));
-                continue;
-              }
-            }
-            TSDebug(PLUGIN_NAME, "adding background_fetch exclusion rule %d for %s: %s", exclude, cfg_name, cfg_value);
-            BgFetchRuleStruct ruleS = {exclude, cfg_name, cfg_value};
-            bgFetchRuleMapP->insert(std::make_pair(index++, ruleS));
-          } else {
-            TSError("%s: invalid value %s, skipping config line", PLUGIN_NAME, cfg_name);
-          }
-        }
-      }
-      memset(buffer, 0, sizeof(buffer));
-    }
-  }
-  TSfclose(file);
-
-  TSDebug(PLUGIN_NAME, "Done parsing config");
-
-  return true;
-}
-
-
-///////////////////////////////////////////////////////////////////////////
-// Remove a header (fully) from an TSMLoc / TSMBuffer. Return the number
-// of fields (header values) we removed.
-int
-remove_header(TSMBuffer bufp, TSMLoc hdr_loc, const char *header, int len)
-{
-  TSMLoc field = TSMimeHdrFieldFind(bufp, hdr_loc, header, len);
-  int cnt = 0;
-
-  while (field) {
-    TSMLoc tmp = TSMimeHdrFieldNextDup(bufp, hdr_loc, field);
-
-    ++cnt;
-    TSMimeHdrFieldDestroy(bufp, hdr_loc, field);
-    TSHandleMLocRelease(bufp, hdr_loc, field);
-    field = tmp;
-  }
-
-  return cnt;
-}
-
-
-///////////////////////////////////////////////////////////////////////////
-// Set a header to a specific value. This will avoid going to through a
-// remove / add sequence in case of an existing header.
-// but clean.
-bool
-set_header(TSMBuffer bufp, TSMLoc hdr_loc, const char *header, int len, const char *val, int val_len)
-{
-  if (!bufp || !hdr_loc || !header || len <= 0 || !val || val_len <= 0) {
-    return false;
-  }
-
-  bool ret = false;
-  TSMLoc field_loc = TSMimeHdrFieldFind(bufp, hdr_loc, header, len);
-
-  if (!field_loc) {
-    // No existing header, so create one
-    if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(bufp, hdr_loc, header, len, &field_loc)) {
-      if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(bufp, hdr_loc, field_loc, -1, val, val_len)) {
-        TSMimeHdrFieldAppend(bufp, hdr_loc, field_loc);
-        ret = true;
-      }
-      TSHandleMLocRelease(bufp, hdr_loc, field_loc);
-    }
-  } else {
-    TSMLoc tmp = NULL;
-    bool first = true;
-
-    while (field_loc) {
-      if (first) {
-        first = false;
-        if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(bufp, hdr_loc, field_loc, -1, val, val_len)) {
-          ret = true;
-        }
-      } else {
-        TSMimeHdrFieldDestroy(bufp, hdr_loc, field_loc);
-      }
-      tmp = TSMimeHdrFieldNextDup(bufp, hdr_loc, field_loc);
-      TSHandleMLocRelease(bufp, hdr_loc, field_loc);
-      field_loc = tmp;
-    }
+    TSDebug(PLUGIN_NAME, "Creating log name %s\n", log_name);
+    TSAssert(TS_SUCCESS == TSTextLogObjectCreate(log_name, TS_LOG_MODE_ADD_TIMESTAMP, &_log));
   }
 
-  return ret;
-}
-
-
-///////////////////////////////////////////////////////////////////////////
-// Dump a header on stderr, useful together with TSDebug().
-void
-dump_headers(TSMBuffer bufp, TSMLoc hdr_loc)
-{
-  TSIOBuffer output_buffer;
-  TSIOBufferReader reader;
-  TSIOBufferBlock block;
-  const char *block_start;
-  int64_t block_avail;
-
-  output_buffer = TSIOBufferCreate();
-  reader = TSIOBufferReaderAlloc(output_buffer);
-
-  /* This will print  just MIMEFields and not the http request line */
-  TSMimeHdrPrint(bufp, hdr_loc, output_buffer);
-
-  /* We need to loop over all the buffer blocks, there can be more than 1 */
-  block = TSIOBufferReaderStart(reader);
-  do {
-    block_start = TSIOBufferBlockReadStart(block, reader, &block_avail);
-    if (block_avail > 0) {
-      TSDebug(PLUGIN_NAME, "Headers are:\n%.*s", static_cast<int>(block_avail), block_start);
-    }
-    TSIOBufferReaderConsume(reader, block_avail);
-    block = TSIOBufferReaderStart(reader);
-  } while (block && block_avail != 0);
-
-  /* Free up the TSIOBuffer that we used to print out the header */
-  TSIOBufferReaderFree(reader);
-  TSIOBufferDestroy(output_buffer);
-}
-
-
-///////////////////////////////////////////////////////////////////////////
-// Struct to hold configurations and state. This can be global, or per
-// remap rule. This also holds the list of currently outstanding URLs,
-// such that we can avoid sending more than one background fill per URL at
-// any given time.
-class BGFetchConfig
-{
-public:
-  BGFetchConfig() : log(NULL) { _lock = TSMutexCreate(); }
-
-  ~BGFetchConfig() { TSMutexDestroy(_lock); }
-
-  void
-  create_log(const char *log_name)
+  TSTextLogObject
+  getLog()
   {
-    TSDebug(PLUGIN_NAME, "Creating log name %s\n", log_name);
-    TSAssert(TS_SUCCESS == TSTextLogObjectCreate(log_name, TS_LOG_MODE_ADD_TIMESTAMP, &log));
+    return _log;
   }
 
   bool
@@ -350,7 +91,7 @@ public:
     }
     TSMutexUnlock(_lock);
 
-    TSDebug(PLUGIN_NAME, "BGFetchConfig.acquire(): ret = %d, url = %s\n", ret, url.c_str());
+    TSDebug(PLUGIN_NAME, "BgFetchState.acquire(): ret = %d, url = %s\n", ret, url.c_str());
 
     return ret;
   }
@@ -372,30 +113,26 @@ public:
     return ret;
   }
 
-  TSTextLogObject log;
-
 private:
+  BgFetchState() : _log(NULL), _lock(TSMutexCreate()) {}
+
+  BgFetchState(BgFetchState const &);   // Don't Implement
+  void operator=(BgFetchState const &); // Don't implement
+
+  TSTextLogObject _log;
   OutstandingRequests _urls;
   TSMutex _lock;
 };
 
-BGFetchConfig *gConfig;
-
 
 //////////////////////////////////////////////////////////////////////////////
-// Hold and manage some state for the background fetch continuation
+// Hold and manage some state for the TXN background fetch continuation.
 // This is necessary, because the TXN is likely to not be available
 // during the time we fetch from origin.
-static int cont_bg_fetch(TSCont contp, TSEvent event, void *edata);
+struct BgFetchData {
+  BgFetchData() : hdr_loc(TS_NULL_MLOC), url_loc(TS_NULL_MLOC), vc(NULL), _bytes(0), _cont(NULL) { mbuf = TSMBufferCreate(); }
 
-struct BGFetchData {
-  BGFetchData(BGFetchConfig *cfg = gConfig)
-    : hdr_loc(TS_NULL_MLOC), url_loc(TS_NULL_MLOC), vc(NULL), _bytes(0), _cont(NULL), _config(cfg)
-  {
-    mbuf = TSMBufferCreate();
-  }
-
-  ~BGFetchData()
+  ~BgFetchData()
   {
     TSHandleMLocRelease(mbuf, TS_NULL_MLOC, hdr_loc);
     TSHandleMLocRelease(mbuf, TS_NULL_MLOC, url_loc);
@@ -403,14 +140,14 @@ struct BGFetchData {
     TSMBufferDestroy(mbuf);
 
     if (vc) {
-      TSError("%s: Destroyed BGFetchDATA while VC was alive", PLUGIN_NAME);
+      TSError("%s: Destroyed BgFetchDATA while VC was alive", PLUGIN_NAME);
       TSVConnClose(vc);
       vc = NULL;
     }
 
     // If we got schedule, also clean that up
     if (_cont) {
-      release_url();
+      releaseUrl();
 
       TSContDestroy(_cont);
       _cont = NULL;
@@ -422,23 +159,24 @@ struct BGFetchData {
   }
 
   bool
-  acquire_url() const
+  acquireUrl() const
   {
-    return _config->acquire(_url);
+    return BgFetchState::getInstance().acquire(_url);
   }
   bool
-  release_url() const
+  releaseUrl() const
   {
-    return _config->release(_url);
+    return BgFetchState::getInstance().release(_url);
   }
 
   const char *
-  get_url() const
+  getUrl() const
   {
     return _url.c_str();
   }
+
   void
-  add_bytes(int64_t b)
+  addBytes(int64_t b)
   {
     _bytes += b;
   }
@@ -462,7 +200,6 @@ private:
   std::string _url;
   int64_t _bytes;
   TSCont _cont;
-  BGFetchConfig *_config;
 };
 
 
@@ -476,7 +213,7 @@ private:
 // Upon succesful completion, the struct should be ready to start a
 // background fetch.
 bool
-BGFetchData::initialize(TSMBuffer request, TSMLoc req_hdr, TSHttpTxn txnp)
+BgFetchData::initialize(TSMBuffer request, TSMLoc req_hdr, TSHttpTxn txnp)
 {
   TSAssert(TS_NULL_MLOC == hdr_loc);
   TSAssert(TS_NULL_MLOC == url_loc);
@@ -530,10 +267,11 @@ BGFetchData::initialize(TSMBuffer request, TSMLoc req_hdr, TSHttpTxn txnp)
   return false;
 }
 
+static int cont_bg_fetch(TSCont contp, TSEvent event, void *edata);
 
 // Create, setup and schedule the background fetch continuation.
 void
-BGFetchData::schedule()
+BgFetchData::schedule()
 {
   TSAssert(NULL == _cont);
 
@@ -551,15 +289,14 @@ BGFetchData::schedule()
   TSContSchedule(_cont, 0, TS_THREAD_POOL_NET);
 }
 
-
 // Log format is:
 //    remap-tag bytes status url
 void
-BGFetchData::log(TSEvent event) const
+BgFetchData::log(TSEvent event) const
 {
-  BGFetchConfig *conf = _config ? _config : gConfig;
+  TSTextLogObject log = BgFetchState::getInstance().getLog();
 
-  if (conf->log) {
+  if (log || TSIsDebugTagSet(PLUGIN_NAME)) {
     const char *status;
 
     switch (event) {
@@ -581,7 +318,10 @@ BGFetchData::log(TSEvent event) const
     }
 
     // ToDo: Also deal with per-remap tagging
-    TSTextLogObjectWrite(conf->log, "%s %" PRId64 " %s %s", "-", _bytes, status, _url.c_str());
+    TSDebug(PLUGIN_NAME, "%s %" PRId64 " %s %s", "-", _bytes, status, _url.c_str());
+    if (log) {
+      TSTextLogObjectWrite(log, "%s %" PRId64 " %s %s", "-", _bytes, status, _url.c_str());
+    }
   }
 }
 
@@ -593,7 +333,7 @@ BGFetchData::log(TSEvent event) const
 static int
 cont_bg_fetch(TSCont contp, TSEvent event, void * /* edata ATS_UNUSED */)
 {
-  BGFetchData *data = static_cast<BGFetchData *>(TSContDataGet(contp));
+  BgFetchData *data = static_cast<BgFetchData *>(TSContDataGet(contp));
   int64_t avail;
 
   switch (event) {
@@ -617,7 +357,7 @@ cont_bg_fetch(TSCont contp, TSEvent event, void * /* edata ATS_UNUSED */)
         TSError("%s: Unknown address family %d", PLUGIN_NAME, sockaddress->sa_family);
         break;
       }
-      TSDebug(PLUGIN_NAME, "Starting bg fetch on: %s", data->get_url());
+      TSDebug(PLUGIN_NAME, "Starting bg fetch on: %s", data->getUrl());
       dump_headers(data->mbuf, data->hdr_loc);
     }
 
@@ -636,12 +376,14 @@ cont_bg_fetch(TSCont contp, TSEvent event, void * /* edata ATS_UNUSED */)
     }
 
   case TS_EVENT_VCONN_WRITE_COMPLETE:
+    // TSVConnShutdown(data->vc, 0, 1);
+    // TSVIOReenable(data->w_vio);
     TSDebug(PLUGIN_NAME, "Write Complete");
     break;
 
   case TS_EVENT_VCONN_READ_READY:
     avail = TSIOBufferReaderAvail(data->resp_io_buf_reader);
-    data->add_bytes(avail);
+    data->addBytes(avail);
     TSIOBufferReaderConsume(data->resp_io_buf_reader, avail);
     TSVIONDoneSet(data->r_vio, TSVIONDoneGet(data->r_vio) + avail);
     TSVIOReenable(data->r_vio);
@@ -660,13 +402,13 @@ cont_bg_fetch(TSCont contp, TSEvent event, void * /* edata ATS_UNUSED */)
 
     TSDebug(PLUGIN_NAME, "Closing down background transaction, event= %s(%d)", TSHttpEventNameLookup(event), event);
     avail = TSIOBufferReaderAvail(data->resp_io_buf_reader);
-    data->add_bytes(avail);
+    data->addBytes(avail);
     TSIOBufferReaderConsume(data->resp_io_buf_reader, avail);
     TSVIONDoneSet(data->r_vio, TSVIONDoneGet(data->r_vio) + avail);
-
     data->log(event);
-    data->vc = NULL;
+
     // Close, release and cleanup
+    data->vc = NULL;
     delete data;
     break;
 
@@ -700,10 +442,10 @@ cont_check_cacheable(TSCont contp, TSEvent /* event ATS_UNUSED */, void *edata)
 
       TSDebug(PLUGIN_NAME, "Testing: request / response is cacheable?");
       if (cacheable) {
-        BGFetchData *data = new BGFetchData();
+        BgFetchData *data = new BgFetchData();
 
         // Initialize the data structure (can fail) and acquire a privileged lock on the URL
-        if (data->initialize(request, req_hdr, txnp) && data->acquire_url()) {
+        if (data->initialize(request, req_hdr, txnp) && data->acquireUrl()) {
           data->schedule();
         } else {
           delete data; // Not sure why this would happen, but ok.
@@ -723,156 +465,6 @@ cont_check_cacheable(TSCont contp, TSEvent /* event ATS_UNUSED */, void *edata)
   return 0;
 }
 
-///////////////////////////////////////////////////////////////////////////
-// Check if a header excludes us from running the background fetch
-//
-static bool
-check_client_ip_configured(TSHttpTxn txnp, const char *cfg_ip)
-{
-  const sockaddr *client_ip = TSHttpTxnClientAddrGet(txnp);
-  char ip_buf[INET6_ADDRSTRLEN];
-
-  if (AF_INET == client_ip->sa_family) {
-    inet_ntop(AF_INET, &(reinterpret_cast<const sockaddr_in *>(client_ip)->sin_addr), ip_buf, INET_ADDRSTRLEN);
-  } else if (AF_INET6 == client_ip->sa_family) {
-    inet_ntop(AF_INET6, &(reinterpret_cast<const sockaddr_in6 *>(client_ip)->sin6_addr), ip_buf, INET6_ADDRSTRLEN);
-  } else {
-    TSError("%s: unknown family %d", PLUGIN_NAME, client_ip->sa_family);
-    return false;
-  }
-
-  TSDebug(PLUGIN_NAME, "cfg_ip %s, client_ip %s", cfg_ip, ip_buf);
-
-  if ((strlen(cfg_ip) == strlen(ip_buf)) && !strcmp(cfg_ip, ip_buf)) {
-    TSDebug(PLUGIN_NAME, "bg fetch for ip %s, configured ip %s", ip_buf, cfg_ip);
-    return true;
-  }
-
-  return false;
-}
-
-static bool
-check_content_length(const uint32_t len, const char *cfg_val)
-{
-  uint32_t cfg_cont_len = atoi(&cfg_val[1]);
-
-  if (cfg_val[0] == '<') {
-    return (len <= cfg_cont_len);
-  } else if (cfg_val[0] == '>') {
-    return (len >= cfg_cont_len);
-  } else {
-    TSError("%s: invalid content length condition %c", PLUGIN_NAME, cfg_val[0]);
-    return false;
-  }
-}
-
-///////////////////////////////////////////////////////////////////////////
-// Check if a header excludes us from running the background fetch
-//
-static bool
-check_field_configured(TSHttpTxn txnp, const char *field_name, const char *cfg_val)
-{
-  // check for client-ip first
-  if (!strcmp(field_name, "Client-IP")) {
-    if (!strcmp(cfg_val, "*")) {
-      TSDebug(PLUGIN_NAME, "Found client_ip wild card");
-      return true;
-    }
-    if (check_client_ip_configured(txnp, cfg_val)) {
-      TSDebug(PLUGIN_NAME, "Found client_ip match");
-      return true;
-    }
-  }
-
-  bool hdr_found = false;
-
-  TSMBuffer hdr_bufp;
-  TSMLoc resp_hdrs;
-
-  if (!strcmp(field_name, "Content-Length")) {
-    if (TS_SUCCESS == TSHttpTxnServerRespGet(txnp, &hdr_bufp, &resp_hdrs)) {
-      TSMLoc loc = TSMimeHdrFieldFind(hdr_bufp, resp_hdrs, field_name, -1);
-      if (TS_NULL_MLOC != loc) {
-        uint32_t content_len = TSMimeHdrFieldValueUintGet(hdr_bufp, resp_hdrs, loc, 0 /* index */);
-        if (check_content_length(content_len, cfg_val)) {
-          TSDebug(PLUGIN_NAME, "Found content-length match");
-          hdr_found = true;
-        }
-        TSHandleMLocRelease(hdr_bufp, resp_hdrs, loc);
-      } else {
-        TSDebug(PLUGIN_NAME, "No content-length field in resp");
-      }
-    } else {
-      TSError("%s: Failed to get resp headers", PLUGIN_NAME);
-    }
-    TSHandleMLocRelease(hdr_bufp, TS_NULL_MLOC, resp_hdrs);
-    return hdr_found;
-  }
-
-  TSMLoc req_hdrs;
-  TSReturnCode ret = TSHttpTxnClientReqGet(txnp, &hdr_bufp, &req_hdrs);
-
-  if (ret != TS_SUCCESS) {
-    // something wrong..
-    TSError("%s: Failed to get req headers", PLUGIN_NAME);
-    return false;
-  }
-
-  TSMLoc loc = TSMimeHdrFieldFind(hdr_bufp, req_hdrs, field_name, -1);
-
-  if (TS_NULL_MLOC != loc) {
-    if (!strcmp(cfg_val, "*")) {
-      TSDebug(PLUGIN_NAME, "Found %s wild card", field_name);
-      hdr_found = true;
-    } else {
-      int val_len = 0;
-      const char *val_str = TSMimeHdrFieldValueStringGet(hdr_bufp, req_hdrs, loc, 0, &val_len);
-
-      if (!val_str || val_len <= 0) {
-        TSDebug(PLUGIN_NAME, "invalid field");
-      } else {
-        TSDebug(PLUGIN_NAME, "comparing with %s", cfg_val);
-        if (NULL != strstr(val_str, cfg_val)) {
-          hdr_found = true;
-        }
-      }
-    }
-    TSHandleMLocRelease(hdr_bufp, req_hdrs, loc);
-  } else {
-    TSDebug(PLUGIN_NAME, "no field %s in request header", field_name);
-  }
-
-  TSHandleMLocRelease(hdr_bufp, TS_NULL_MLOC, req_hdrs);
-  return hdr_found;
-}
-
-///////////////////////////////////////////////////////////////////////////
-// Check the configuration (either per remap, or global), and decide if
-// this request is allowed to trigger a background fetch.
-//
-static bool
-is_background_fetch_allowed(TSHttpTxn txnp, BgFetchRuleMap *ri)
-{
-  TSDebug(PLUGIN_NAME, "Testing: request is internal?");
-  if (TSHttpTxnIsInternal(txnp) == TS_SUCCESS) {
-    return false;
-  }
-
-  bool allow_bg_fetch = true;
-
-  for (BgFetchRuleMap::iterator it = ri->begin(); it != ri->end(); ++it) {
-    BgFetchRuleStruct sRule = it->second;
-    const char *ruleField = sRule.ruleField.c_str();
-    if (check_field_configured(txnp, ruleField, sRule.ruleValue.c_str())) {
-      TSDebug(PLUGIN_NAME, "found field match %s, exclude %d", ruleField, sRule.exclude);
-      allow_bg_fetch = !sRule.exclude;
-      break;
-    }
-  }
-
-  return allow_bg_fetch;
-}
-
 
 //////////////////////////////////////////////////////////////////////////////
 // Main "plugin", which is a global READ_RESPONSE_HDR hook. Before
@@ -890,46 +482,45 @@ cont_handle_response(TSCont contp, TSEvent event, void *edata)
 {
   // ToDo: If we want to support per-remap configurations, we have to pass along the data here
   TSHttpTxn txnp = static_cast<TSHttpTxn>(edata);
-  BgFetchRulesConfig *rc = static_cast<BgFetchRulesConfig *>(TSContDataGet(contp));
-  BgFetchRuleMap *ri = rc->getRuleMap();
-
-  if (ri == NULL) {
-    // something wrong..
-    TSError("%s: can't get BgFetchRule Map", PLUGIN_NAME);
-    ri = &gBgFetchRuleMap;
-  }
+  BgFetchConfig *config = static_cast<BgFetchConfig *>(TSContDataGet(contp));
 
-  switch (event) {
-  case TS_EVENT_HTTP_READ_RESPONSE_HDR:
-    if (is_background_fetch_allowed(txnp, ri)) {
-      TSMBuffer response;
-      TSMLoc resp_hdr;
-
-      if (TS_SUCCESS == TSHttpTxnServerRespGet(txnp, &response, &resp_hdr)) {
-        // ToDo: Check the MIME type first, to see if it's a type we care about.
-        // ToDo: Such MIME types should probably be per remap rule.
-
-        // 2. Only deal with 206 responses from Origin
-        TSDebug(PLUGIN_NAME, "Testing: response is 206?");
-        if (TS_HTTP_STATUS_PARTIAL_CONTENT == TSHttpHdrStatusGet(response, resp_hdr)) {
-          // Everything looks good so far, add a TXN hook for SEND_RESPONSE_HDR
-          TSCont contp = TSContCreate(cont_check_cacheable, NULL);
-
-          TSHttpTxnHookAdd(txnp, TS_HTTP_SEND_RESPONSE_HDR_HOOK, contp);
+  if (NULL == config) {
+    // something seriously wrong..
+    TSError("%s: can't get configurations", PLUGIN_NAME);
+  } else {
+    switch (event) {
+    case TS_EVENT_HTTP_READ_RESPONSE_HDR:
+      if (config->getRules()->bgFetchAllowed(txnp)) {
+        TSMBuffer response;
+        TSMLoc resp_hdr;
+
+        if (TS_SUCCESS == TSHttpTxnServerRespGet(txnp, &response, &resp_hdr)) {
+          // ToDo: Check the MIME type first, to see if it's a type we care about.
+          // ToDo: Such MIME types should probably be per remap rule.
+
+          // Only deal with 206 responses from Origin
+          TSDebug(PLUGIN_NAME, "Testing: response is 206?");
+          if (TS_HTTP_STATUS_PARTIAL_CONTENT == TSHttpHdrStatusGet(response, resp_hdr)) {
+            // Everything looks good so far, add a TXN hook for SEND_RESPONSE_HDR
+            TSCont contp = TSContCreate(cont_check_cacheable, NULL);
+
+            TSHttpTxnHookAdd(txnp, TS_HTTP_SEND_RESPONSE_HDR_HOOK, contp);
+          }
+          // Release the response MLoc
+          TSHandleMLocRelease(response, TS_NULL_MLOC, resp_hdr);
         }
-        // Release the response MLoc
-        TSHandleMLocRelease(response, TS_NULL_MLOC, resp_hdr);
       }
+      break;
+    case TS_EVENT_HTTP_TXN_CLOSE:
+      config->release(); // Release the configuration lease when txn closes
+      break;
+    default:
+      TSError("%s: unknown event for this plugin", PLUGIN_NAME);
+      TSDebug(PLUGIN_NAME, "unknown event for this plugin");
+      break;
     }
-    break;
-  case TS_EVENT_HTTP_TXN_CLOSE:
-    rc->decrRefCount();
-    break;
-  default:
-    TSError("%s: unknown event for this plugin", PLUGIN_NAME);
-    TSDebug(PLUGIN_NAME, "unknown event for this plugin");
-    break;
   }
+
   // Reenable and continue with the state machine.
   TSHttpTxnReenable(txnp, TS_EVENT_HTTP_CONTINUE);
   return 0;
@@ -954,19 +545,22 @@ TSPluginInit(int argc, const char *argv[])
     TSError("%s: plugin registration failed.\n", PLUGIN_NAME);
   }
 
-  gConfig = new BGFetchConfig();
-  optind = 1;
+  TSCont cont = TSContCreate(cont_handle_response, NULL);
+
+  gConfig = new BgFetchConfig(cont);
+  gConfig->acquire(); // Inc refcount, although this global config should never go out of scope
 
+  optind = 1;
   while (true) {
     int opt = getopt_long(argc, (char *const *)argv, "lc", longopt, NULL);
 
     switch (opt) {
     case 'l':
-      gConfig->create_log(optarg);
+      BgFetchState::getInstance().createLog(optarg);
       break;
     case 'c':
       TSDebug(PLUGIN_NAME, "config file %s..", optarg);
-      read_config(optarg, &gBgFetchRuleMap);
+      gConfig->readConfig(optarg);
       break;
     }
 
@@ -976,12 +570,6 @@ TSPluginInit(int argc, const char *argv[])
   }
 
   TSDebug(PLUGIN_NAME, "Initialized");
-
-  BgFetchRulesConfig *rulesConf = new BgFetchRulesConfig();
-  TSCont cont = TSContCreate(cont_handle_response, NULL);
-  rulesConf->setRuleMap(&gBgFetchRuleMap);
-  rulesConf->setCont(cont);
-  TSContDataSet(cont, static_cast<void *>(rulesConf));
   TSHttpHookAdd(TS_HTTP_READ_RESPONSE_HDR_HOOK, cont);
 }
 
@@ -1017,33 +605,18 @@ TSRemapInit(TSRemapInterface *api_info, char *errbuf, int errbuf_size)
 TSReturnCode
 TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf */, int /* errbuf_size */)
 {
-  BgFetchRuleMap *ri = new BgFetchRuleMap();
-  if (ri == NULL) {
-    TSError("%s:Unable to create remap instance", PLUGIN_NAME);
-    return TS_ERROR;
-  }
+  TSCont cont = TSContCreate(cont_handle_response, NULL);
+  BgFetchConfig *config = new BgFetchConfig(cont);
 
-  if (gConfig == NULL) {
-    TSDebug(PLUGIN_NAME, "creating gConfig");
-    gConfig = new BGFetchConfig();
-  }
+  config->acquire(); // Inc refcount
 
-  char *fileName = NULL;
+  // Parse the optional rules, wihch becomes a linked list of BgFetchRule's.
   if (argc > 2) {
-    fileName = argv[2];
-    TSDebug(PLUGIN_NAME, "config file %s", fileName);
+    TSDebug(PLUGIN_NAME, "config file %s", argv[2]);
+    config->readConfig(argv[2]);
   }
 
-  read_config(fileName, ri);
-
-  BgFetchRulesConfig *rulesConf = new BgFetchRulesConfig();
-  TSCont cont = TSContCreate(cont_handle_response, NULL);
-  rulesConf->setRuleMap(ri);
-  rulesConf->setCont(cont);
-  rulesConf->incrRefCount();
-  TSContDataSet(cont, static_cast<void *>(rulesConf));
-
-  *ih = (void *)rulesConf;
+  *ih = (void *)config;
 
   return TS_SUCCESS;
 }
@@ -1051,8 +624,9 @@ TSRemapNewInstance(int argc, char *argv[], void **ih, char * /* errbuf */, int /
 void
 TSRemapDeleteInstance(void *ih)
 {
-  BgFetchRulesConfig *ri = static_cast<BgFetchRulesConfig *>(ih);
-  ri->decrRefCount();
+  BgFetchConfig *config = static_cast<BgFetchConfig *>(ih);
+
+  config->release();
 }
 
 
@@ -1066,12 +640,13 @@ TSRemapDoRemap(void *ih, TSHttpTxn txnp, TSRemapRequestInfo * /* rri */)
     return TSREMAP_NO_REMAP;
   }
 
-  TSDebug(PLUGIN_NAME, "background fetch TSRemapDoRemap...");
+  BgFetchConfig *config = static_cast<BgFetchConfig *>(ih);
+
+  config->acquire(); // Inc refcount
+  TSHttpTxnHookAdd(txnp, TS_HTTP_READ_RESPONSE_HDR_HOOK, config->getCont());
+  TSHttpTxnHookAdd(txnp, TS_HTTP_TXN_CLOSE_HOOK, config->getCont());
 
-  BgFetchRulesConfig *rulesConf = static_cast<BgFetchRulesConfig *>(ih);
-  rulesConf->incrRefCount();
-  TSHttpTxnHookAdd(txnp, TS_HTTP_READ_RESPONSE_HDR_HOOK, rulesConf->getCont());
-  TSHttpTxnHookAdd(txnp, TS_HTTP_TXN_CLOSE_HOOK, rulesConf->getCont());
+  TSDebug(PLUGIN_NAME, "background fetch TSRemapDoRemap...");
 
   return TSREMAP_NO_REMAP;
 }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/configs.cc
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/configs.cc b/plugins/experimental/background_fetch/configs.cc
new file mode 100644
index 0000000..7e1b033
--- /dev/null
+++ b/plugins/experimental/background_fetch/configs.cc
@@ -0,0 +1,124 @@
+/** @file
+
+    Plugin to perform background fetches of certain content that would
+    otherwise not be cached. For example, Range: requests / responses.
+
+    @section license License
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+*/
+
+#include "configs.h"
+
+
+// Read a config file, populare the linked list (chain the BgFetchRule's)
+bool
+BgFetchConfig::readConfig(const char *config_file)
+{
+  char file_path[4096];
+  TSFile file;
+
+  if (NULL == config_file) {
+    TSError("%s: invalid config file", PLUGIN_NAME);
+    return false;
+  }
+
+  TSDebug(PLUGIN_NAME, "trying to open config file in this path: %s", config_file);
+
+  file = TSfopen(config_file, "r");
+  if (NULL == file) {
+    TSDebug(PLUGIN_NAME, "Failed to open config file %s, trying rel path", config_file);
+    snprintf(file_path, sizeof(file_path), "%s/%s", TSInstallDirGet(), config_file);
+    file = TSfopen(file_path, "r");
+    if (NULL == file) {
+      TSError("%s: invalid config file", PLUGIN_NAME);
+      return false;
+    }
+  }
+
+  BgFetchRule *cur = NULL;
+  char buffer[8192];
+
+  memset(buffer, 0, sizeof(buffer));
+  while (TSfgets(file, buffer, sizeof(buffer) - 1) != NULL) {
+    char *eol = 0;
+
+    // make sure line was not bigger than buffer
+    if (NULL == (eol = strchr(buffer, '\n')) && NULL == (eol = strstr(buffer, "\r\n"))) {
+      TSError("%s: exclusion line too long, did not get a good line in cfg, skipping, line: %s", PLUGIN_NAME, buffer);
+      memset(buffer, 0, sizeof(buffer));
+      continue;
+    }
+    // make sure line has something useful on it
+    if (eol - buffer < 2 || buffer[0] == '#') {
+      memset(buffer, 0, sizeof(buffer));
+      continue;
+    }
+
+    char *savePtr = NULL;
+    char *cfg = strtok_r(buffer, "\n\r\n", &savePtr);
+
+    if (NULL != cfg) {
+      TSDebug(PLUGIN_NAME, "setting background_fetch exclusion criterion based on string: %s", cfg);
+      char *cfg_type = strtok_r(buffer, " ", &savePtr);
+      char *cfg_name = NULL;
+      char *cfg_value = NULL;
+      bool exclude = false;
+
+      if (cfg_type) {
+        if (!strcmp(cfg_type, "exclude")) {
+          exclude = true;
+        } else if (strcmp(cfg_type, "include")) {
+          TSError("%s: invalid specifier %s, skipping config line", PLUGIN_NAME, cfg_type);
+          memset(buffer, 0, sizeof(buffer));
+          continue;
+        }
+        cfg_name = strtok_r(NULL, " ", &savePtr);
+        if (cfg_name) {
+          cfg_value = strtok_r(NULL, " ", &savePtr);
+          if (cfg_value) {
+            if (!strcmp(cfg_name, "Content-Length")) {
+              if ((cfg_value[0] != '<') && (cfg_value[0] != '>')) {
+                TSError("%s: invalid content-len condition %s, skipping config value", PLUGIN_NAME, cfg_value);
+                memset(buffer, 0, sizeof(buffer));
+                continue;
+              }
+            }
+            BgFetchRule *r = new BgFetchRule(exclude, cfg_name, cfg_value);
+
+            if (NULL == _rules) {
+              _rules = r;
+            } else {
+              cur->chain(r);
+            }
+            cur = r;
+
+            TSDebug(PLUGIN_NAME, "adding background_fetch exclusion rule %d for %s: %s", exclude, cfg_name, cfg_value);
+          } else {
+            TSError("%s: invalid value %s, skipping config line", PLUGIN_NAME, cfg_name);
+          }
+        }
+      }
+      memset(buffer, 0, sizeof(buffer));
+    }
+  }
+
+  TSfclose(file);
+  TSDebug(PLUGIN_NAME, "Done parsing config");
+
+  return true;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/configs.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/configs.h b/plugins/experimental/background_fetch/configs.h
new file mode 100644
index 0000000..4f5ff33
--- /dev/null
+++ b/plugins/experimental/background_fetch/configs.h
@@ -0,0 +1,90 @@
+/** @file
+
+    Plugin to perform background fetches of certain content that would
+    otherwise not be cached. For example, Range: requests / responses.
+
+    @section license License
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+*/
+
+#ifndef CONFIGS_H_DEBFCE23_D6E9_40C2_AAA5_32B32586A3DA
+#define CONFIGS_H_DEBFCE23_D6E9_40C2_AAA5_32B32586A3DA
+
+#include <stdlib.h>
+
+#include "rules.h"
+#include "ink_atomic.h"
+
+// Constants
+const char PLUGIN_NAME[] = "background_fetch";
+
+
+///////////////////////////////////////////////////////////////////////////
+// This holds one complete background fetch rule, which is also ref-counted.
+//
+class BgFetchConfig
+{
+public:
+  BgFetchConfig(TSCont cont) : _cont(cont), _rules(NULL), _ref_count(0) { TSContDataSet(cont, static_cast<void *>(this)); }
+
+  void
+  acquire()
+  {
+    ink_atomic_increment(&_ref_count, 1);
+  }
+
+  void
+  release()
+  {
+    TSDebug(PLUGIN_NAME, "ref_count is %d", _ref_count);
+    if (1 >= ink_atomic_decrement(&_ref_count, 1)) {
+      TSDebug(PLUGIN_NAME, "configuration deleted, due to ref-counting");
+      delete this;
+    }
+  }
+
+  BgFetchRule *
+  getRules() const
+  {
+    return _rules;
+  }
+
+  TSCont
+  getCont() const
+  {
+    return _cont;
+  }
+
+  // This parses and populates the BgFetchRule linked list (_rules).
+  bool readConfig(const char *file_name);
+
+private:
+  ~BgFetchConfig()
+  {
+    delete _rules;
+    if (_cont) {
+      TSContDestroy(_cont);
+    }
+  }
+
+  TSCont _cont;
+  BgFetchRule *_rules;
+  volatile int _ref_count;
+};
+
+#endif /* CONFIGS_H_DEBFCE23_D6E9_40C2_AAA5_32B32586A3DA */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/headers.cc
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/headers.cc b/plugins/experimental/background_fetch/headers.cc
new file mode 100644
index 0000000..8c33aba
--- /dev/null
+++ b/plugins/experimental/background_fetch/headers.cc
@@ -0,0 +1,129 @@
+/** @file
+
+    Plugin to perform background fetches of certain content that would
+    otherwise not be cached. For example, Range: requests / responses.
+
+    @section license License
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+*/
+#include <stdlib.h>
+
+#include "configs.h"
+#include "headers.h"
+
+
+///////////////////////////////////////////////////////////////////////////
+// Remove a header (fully) from an TSMLoc / TSMBuffer. Return the number
+// of fields (header values) we removed.
+int
+remove_header(TSMBuffer bufp, TSMLoc hdr_loc, const char *header, int len)
+{
+  TSMLoc field = TSMimeHdrFieldFind(bufp, hdr_loc, header, len);
+  int cnt = 0;
+
+  while (field) {
+    TSMLoc tmp = TSMimeHdrFieldNextDup(bufp, hdr_loc, field);
+
+    ++cnt;
+    TSMimeHdrFieldDestroy(bufp, hdr_loc, field);
+    TSHandleMLocRelease(bufp, hdr_loc, field);
+    field = tmp;
+  }
+
+  return cnt;
+}
+
+
+///////////////////////////////////////////////////////////////////////////
+// Set a header to a specific value. This will avoid going to through a
+// remove / add sequence in case of an existing header.
+// but clean.
+bool
+set_header(TSMBuffer bufp, TSMLoc hdr_loc, const char *header, int len, const char *val, int val_len)
+{
+  if (!bufp || !hdr_loc || !header || len <= 0 || !val || val_len <= 0) {
+    return false;
+  }
+
+  bool ret = false;
+  TSMLoc field_loc = TSMimeHdrFieldFind(bufp, hdr_loc, header, len);
+
+  if (!field_loc) {
+    // No existing header, so create one
+    if (TS_SUCCESS == TSMimeHdrFieldCreateNamed(bufp, hdr_loc, header, len, &field_loc)) {
+      if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(bufp, hdr_loc, field_loc, -1, val, val_len)) {
+        TSMimeHdrFieldAppend(bufp, hdr_loc, field_loc);
+        ret = true;
+      }
+      TSHandleMLocRelease(bufp, hdr_loc, field_loc);
+    }
+  } else {
+    TSMLoc tmp = NULL;
+    bool first = true;
+
+    while (field_loc) {
+      if (first) {
+        first = false;
+        if (TS_SUCCESS == TSMimeHdrFieldValueStringSet(bufp, hdr_loc, field_loc, -1, val, val_len)) {
+          ret = true;
+        }
+      } else {
+        TSMimeHdrFieldDestroy(bufp, hdr_loc, field_loc);
+      }
+      tmp = TSMimeHdrFieldNextDup(bufp, hdr_loc, field_loc);
+      TSHandleMLocRelease(bufp, hdr_loc, field_loc);
+      field_loc = tmp;
+    }
+  }
+
+  return ret;
+}
+
+
+///////////////////////////////////////////////////////////////////////////
+// Dump a header on stderr, useful together with TSDebug().
+void
+dump_headers(TSMBuffer bufp, TSMLoc hdr_loc)
+{
+  TSIOBuffer output_buffer;
+  TSIOBufferReader reader;
+  TSIOBufferBlock block;
+  const char *block_start;
+  int64_t block_avail;
+
+  output_buffer = TSIOBufferCreate();
+  reader = TSIOBufferReaderAlloc(output_buffer);
+
+  /* This will print  just MIMEFields and not the http request line */
+  TSMimeHdrPrint(bufp, hdr_loc, output_buffer);
+
+  /* We need to loop over all the buffer blocks, there can be more than 1 */
+  block = TSIOBufferReaderStart(reader);
+  do {
+    block_start = TSIOBufferBlockReadStart(block, reader, &block_avail);
+    if (block_avail > 0) {
+      TSDebug(PLUGIN_NAME, "Headers are:\n%.*s", static_cast<int>(block_avail), block_start);
+    }
+    TSIOBufferReaderConsume(reader, block_avail);
+    block = TSIOBufferReaderStart(reader);
+  } while (block && block_avail != 0);
+
+  /* Free up the TSIOBuffer that we used to print out the header */
+  TSIOBufferReaderFree(reader);
+  TSIOBufferDestroy(output_buffer);
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/headers.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/headers.h b/plugins/experimental/background_fetch/headers.h
new file mode 100644
index 0000000..f773928
--- /dev/null
+++ b/plugins/experimental/background_fetch/headers.h
@@ -0,0 +1,32 @@
+/** @file
+
+    Plugin to perform background fetches of certain content that would
+    otherwise not be cached. For example, Range: requests / responses.
+
+    @section license License
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+*/
+
+#ifndef HEADERS_H_9E78B01C_90CA_4E2D_9346_B17740751B9F
+#define HEADERS_H_9E78B01C_90CA_4E2D_9346_B17740751B9F
+
+int remove_header(TSMBuffer bufp, TSMLoc hdr_loc, const char *header, int len);
+bool set_header(TSMBuffer bufp, TSMLoc hdr_loc, const char *header, int len, const char *val, int val_len);
+void dump_headers(TSMBuffer bufp, TSMLoc hdr_loc);
+
+#endif /* HEADERS_H_9E78B01C_90CA_4E2D_9346_B17740751B9F */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/rules.cc
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/rules.cc b/plugins/experimental/background_fetch/rules.cc
new file mode 100644
index 0000000..ad3cada
--- /dev/null
+++ b/plugins/experimental/background_fetch/rules.cc
@@ -0,0 +1,175 @@
+/** @file
+
+    Plugin to perform background fetches of certain content that would
+    otherwise not be cached. For example, Range: requests / responses.
+
+    @section license License
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+*/
+
+#include <stdlib.h>
+
+#include "configs.h"
+#include "rules.h"
+
+///////////////////////////////////////////////////////////////////////////
+// These are little helper functions for the main rules evaluator.
+//
+static bool
+check_client_ip_configured(TSHttpTxn txnp, const char *cfg_ip)
+{
+  const sockaddr *client_ip = TSHttpTxnClientAddrGet(txnp);
+  char ip_buf[INET6_ADDRSTRLEN];
+
+  if (AF_INET == client_ip->sa_family) {
+    inet_ntop(AF_INET, &(reinterpret_cast<const sockaddr_in *>(client_ip)->sin_addr), ip_buf, INET_ADDRSTRLEN);
+  } else if (AF_INET6 == client_ip->sa_family) {
+    inet_ntop(AF_INET6, &(reinterpret_cast<const sockaddr_in6 *>(client_ip)->sin6_addr), ip_buf, INET6_ADDRSTRLEN);
+  } else {
+    TSError("%s: unknown family %d", PLUGIN_NAME, client_ip->sa_family);
+    return false;
+  }
+
+  TSDebug(PLUGIN_NAME, "cfg_ip %s, client_ip %s", cfg_ip, ip_buf);
+
+  if ((strlen(cfg_ip) == strlen(ip_buf)) && !strcmp(cfg_ip, ip_buf)) {
+    TSDebug(PLUGIN_NAME, "bg fetch for ip %s, configured ip %s", ip_buf, cfg_ip);
+    return true;
+  }
+
+  return false;
+}
+
+static bool
+check_content_length(const uint32_t len, const char *cfg_val)
+{
+  uint32_t cfg_cont_len = atoi(&cfg_val[1]);
+
+  if (cfg_val[0] == '<') {
+    return (len <= cfg_cont_len);
+  } else if (cfg_val[0] == '>') {
+    return (len >= cfg_cont_len);
+  } else {
+    TSError("%s: invalid content length condition %c", PLUGIN_NAME, cfg_val[0]);
+    return false;
+  }
+}
+
+///////////////////////////////////////////////////////////////////////////
+// Check if a header excludes us from running the background fetch
+//
+static bool
+check_field_configured(TSHttpTxn txnp, const char *field_name, const char *cfg_val)
+{
+  // check for client-ip first
+  if (!strcmp(field_name, "Client-IP")) {
+    if (!strcmp(cfg_val, "*")) {
+      TSDebug(PLUGIN_NAME, "Found client_ip wild card");
+      return true;
+    }
+    if (check_client_ip_configured(txnp, cfg_val)) {
+      TSDebug(PLUGIN_NAME, "Found client_ip match");
+      return true;
+    }
+  }
+
+  bool hdr_found = false;
+  TSMBuffer hdr_bufp;
+  TSMLoc hdr_loc;
+
+  // Check response headers. ToDo: This doesn't check e.g. Content-Type :-/.
+  if (!strcmp(field_name, "Content-Length")) {
+    if (TS_SUCCESS == TSHttpTxnServerRespGet(txnp, &hdr_bufp, &hdr_loc)) {
+      TSMLoc loc = TSMimeHdrFieldFind(hdr_bufp, hdr_loc, field_name, -1);
+
+      if (TS_NULL_MLOC != loc) {
+        unsigned int content_len = TSMimeHdrFieldValueUintGet(hdr_bufp, hdr_loc, loc, 0 /* index */);
+
+        if (check_content_length(content_len, cfg_val)) {
+          TSDebug(PLUGIN_NAME, "Found content-length match");
+          hdr_found = true;
+        }
+        TSHandleMLocRelease(hdr_bufp, hdr_loc, loc);
+      } else {
+        TSDebug(PLUGIN_NAME, "No content-length field in resp");
+      }
+      TSHandleMLocRelease(hdr_bufp, TS_NULL_MLOC, hdr_loc);
+    } else {
+      TSError("%s: Failed to get resp headers", PLUGIN_NAME);
+    }
+    return hdr_found;
+  }
+
+  // Check request headers
+  if (TS_SUCCESS == TSHttpTxnClientReqGet(txnp, &hdr_bufp, &hdr_loc)) {
+    TSMLoc loc = TSMimeHdrFieldFind(hdr_bufp, hdr_loc, field_name, -1);
+
+    if (TS_NULL_MLOC != loc) {
+      if (!strcmp(cfg_val, "*")) {
+        TSDebug(PLUGIN_NAME, "Found %s wild card", field_name);
+        hdr_found = true;
+      } else {
+        int val_len = 0;
+        const char *val_str = TSMimeHdrFieldValueStringGet(hdr_bufp, hdr_loc, loc, 0, &val_len);
+
+        if (!val_str || val_len <= 0) {
+          TSDebug(PLUGIN_NAME, "invalid field");
+        } else {
+          TSDebug(PLUGIN_NAME, "comparing with %s", cfg_val);
+          if (NULL != strstr(val_str, cfg_val)) {
+            hdr_found = true;
+          }
+        }
+      }
+      TSHandleMLocRelease(hdr_bufp, hdr_loc, loc);
+    } else {
+      TSDebug(PLUGIN_NAME, "no field %s in request header", field_name);
+    }
+    TSHandleMLocRelease(hdr_bufp, TS_NULL_MLOC, hdr_loc);
+  } else {
+    TSError("%s: Failed to get resp headers", PLUGIN_NAME);
+  }
+
+  return hdr_found;
+}
+
+///////////////////////////////////////////////////////////////////////////
+// Check the configuration (either per remap, or global), and decide if
+// this request is allowed to trigger a background fetch.
+//
+bool
+BgFetchRule::bgFetchAllowed(TSHttpTxn txnp) const
+{
+  TSDebug(PLUGIN_NAME, "Testing: request is internal?");
+  if (TSHttpTxnIsInternal(txnp) == TS_SUCCESS) {
+    return false;
+  }
+
+  bool allow_bg_fetch = true;
+
+  // We could do this recursively, but following the linked list is probably more efficient.
+  for (const BgFetchRule *r = this; NULL != r; r = r->_next) {
+    if (check_field_configured(txnp, r->_field, r->_value)) {
+      TSDebug(PLUGIN_NAME, "found field match %s, exclude %d", r->_field, (int)r->_exclude);
+      allow_bg_fetch = !r->_exclude;
+      break;
+    }
+  }
+
+  return allow_bg_fetch;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/632571d4/plugins/experimental/background_fetch/rules.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/background_fetch/rules.h b/plugins/experimental/background_fetch/rules.h
new file mode 100644
index 0000000..7ccc0fd
--- /dev/null
+++ b/plugins/experimental/background_fetch/rules.h
@@ -0,0 +1,70 @@
+/** @file
+
+    Plugin to perform background fetches of certain content that would
+    otherwise not be cached. For example, Range: requests / responses.
+
+    @section license License
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+*/
+
+#ifndef RULES_H_E39522EE_1258_49B4_8FA4_7DAB7A6FC4DA
+#define RULES_H_E39522EE_1258_49B4_8FA4_7DAB7A6FC4DA
+
+#include <stdlib.h>
+#include <netinet/in.h>
+#include <arpa/inet.h>
+
+#include "ts/ts.h"
+
+///////////////////////////////////////////////////////////////////////////
+// This is a linked list of rule entries. This gets stored and parsed with the
+// BgFetchConfig object.
+//
+class BgFetchRule
+{
+public:
+  BgFetchRule(bool exc, const char *field, const char *value)
+    : _exclude(exc), _field(TSstrdup(field)), _value(TSstrdup(value)), _next(NULL)
+  {
+  }
+
+  ~BgFetchRule()
+  {
+    delete _field;
+    delete _value;
+    delete _next;
+  }
+
+  // For chaining the linked list
+  void
+  chain(BgFetchRule *n)
+  {
+    _next = n;
+  }
+
+  // Main evaluation entry point.
+  bool bgFetchAllowed(TSHttpTxn txnp) const;
+
+private:
+  bool _exclude;
+  const char *_field;
+  const char *_value;
+  BgFetchRule *_next; // For the linked list
+};
+
+#endif /* RULES_H_E39522EE_1258_49B4_8FA4_7DAB7A6FC4DA */