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 2018/03/29 21:43:00 UTC

[trafficserver] branch master updated: Makes the global remap data structure ref-counted

This is an automated email from the ASF dual-hosted git repository.

zwoop pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/trafficserver.git


The following commit(s) were added to refs/heads/master by this push:
     new 12d305b  Makes the global remap data structure ref-counted
12d305b is described below

commit 12d305b8d465710e6fc7a036287f1fea733c22c0
Author: Leif Hedstrom <zw...@apache.org>
AuthorDate: Thu Mar 15 22:19:38 2018 -0400

    Makes the global remap data structure ref-counted
    
    This solves a number of issues, such as:
    
         1. Remap plugins won't have to refcount itself
         2. We can, I think, make remap plugins reloadable
    
    The code didn't turn out quite as I wanted; I had initially
    intended to store the leased table in the t_state (::State)
    object, but circular references here made that difficult
    without major refactoring. Instead, I hoisted it up to the
    HttpSM, and added a UrlRewrite* argument to a couple of
    functions that needs it in addition to the t_state.
    
    In addition, I eliminated the two macros around the ref-count
    increment / decrement, seems unecessary in general, and only
    used in a couple of places.
---
 iocore/eventsystem/P_Freer.h       | 11 +++++++++--
 lib/ts/Ptr.h                       |  3 ---
 proxy/ReverseProxy.cc              | 23 ++++++++++++++---------
 proxy/ReverseProxy.h               |  4 ++--
 proxy/http/HttpSM.cc               |  9 ++++++---
 proxy/http/HttpSM.h                |  5 +++++
 proxy/http/HttpTransact.cc         |  4 ++--
 proxy/http/HttpTransact.h          |  3 +--
 proxy/http/remap/RemapProcessor.cc | 34 +++++++++++++++++-----------------
 proxy/http/remap/RemapProcessor.h  |  4 ++--
 proxy/http/remap/UrlRewrite.h      | 23 +++++++++++++++++++++--
 proxy/logging/LogObject.cc         | 10 +++++-----
 12 files changed, 84 insertions(+), 49 deletions(-)

diff --git a/iocore/eventsystem/P_Freer.h b/iocore/eventsystem/P_Freer.h
index 837c31e..8562931 100644
--- a/iocore/eventsystem/P_Freer.h
+++ b/iocore/eventsystem/P_Freer.h
@@ -47,11 +47,18 @@ public: // Needed by WinNT compiler (compiler bug)
   DeleterContinuation(C *ap) : Continuation(new_ProxyMutex()), p(ap) { SET_HANDLER(&DeleterContinuation::dieEvent); }
 };
 
+// This can be useful for two things (or both):
+//    1. Make sure to schedule a delete on an ET_TASK thread
+//    2. Delay the delete (this should be used sparingly)
 template <class C>
 TS_INLINE void
 new_Deleter(C *ap, ink_hrtime t)
 {
-  eventProcessor.schedule_in(new DeleterContinuation<C>(ap), t, ET_TASK);
+  if (t > 0) {
+    eventProcessor.schedule_in(new DeleterContinuation<C>(ap), t, ET_TASK);
+  } else {
+    eventProcessor.schedule_imm(new DeleterContinuation<C>(ap), ET_TASK);
+  }
 }
 
 template <class C> struct FreeCallContinuation : public Continuation {
@@ -108,7 +115,7 @@ template <class C> struct DereferContinuation : public Continuation {
   dieEvent(int, Event *)
   {
     p->refcount_dec();
-    if (REF_COUNT_OBJ_REFCOUNT_DEC(p) == 0) {
+    if (p->refcount_dec() == 0) {
       delete p;
     }
 
diff --git a/lib/ts/Ptr.h b/lib/ts/Ptr.h
index 3c3c9fd..687f053 100644
--- a/lib/ts/Ptr.h
+++ b/lib/ts/Ptr.h
@@ -90,9 +90,6 @@ private:
   int m_refcount;
 };
 
-#define REF_COUNT_OBJ_REFCOUNT_INC(_x) (_x)->refcount_inc()
-#define REF_COUNT_OBJ_REFCOUNT_DEC(_x) (_x)->refcount_dec()
-
 ////////////////////////////////////////////////////////////////////////
 //
 // class Ptr
diff --git a/proxy/ReverseProxy.cc b/proxy/ReverseProxy.cc
index ad41a96..f1220e1 100644
--- a/proxy/ReverseProxy.cc
+++ b/proxy/ReverseProxy.cc
@@ -42,9 +42,6 @@
 #include "UrlRewrite.h"
 #include "UrlMapping.h"
 
-/** Time till we free the old stuff after a reconfiguration. */
-#define URL_REWRITE_TIMEOUT (HRTIME_SECOND * 300)
-
 // Global Ptrs
 static Ptr<ProxyMutex> reconfig_mutex;
 UrlRewrite *rewrite_table = nullptr;
@@ -79,6 +76,10 @@ init_reverse_proxy()
   REC_RegisterConfigUpdateFunc("proxy.config.reverse_proxy.enabled", url_rewrite_CB, (void *)REVERSE_CHANGED);
   REC_RegisterConfigUpdateFunc("proxy.config.admin.synthetic_port", url_rewrite_CB, (void *)SYNTH_PORT_CHANGED);
   REC_RegisterConfigUpdateFunc("proxy.config.http.referer_default_redirect", url_rewrite_CB, (void *)HTTP_DEFAULT_REDIRECT_CHANGED);
+
+  // Hold at least one lease, until we reload the configuration
+  rewrite_table->acquire();
+
   return 0;
 }
 
@@ -87,15 +88,15 @@ init_reverse_proxy()
    according to the rules in remap.config.
 */
 mapping_type
-request_url_remap_redirect(HTTPHdr *request_header, URL *redirect_url)
+request_url_remap_redirect(HTTPHdr *request_header, URL *redirect_url, UrlRewrite *table)
 {
-  return rewrite_table ? rewrite_table->Remap_redirect(request_header, redirect_url) : NONE;
+  return table ? table->Remap_redirect(request_header, redirect_url) : NONE;
 }
 
 bool
-response_url_remap(HTTPHdr *response_header)
+response_url_remap(HTTPHdr *response_header, UrlRewrite *table)
 {
-  return rewrite_table ? rewrite_table->ReverseMap(response_header) : false;
+  return table ? table->ReverseMap(response_header) : false;
 }
 
 //
@@ -134,14 +135,18 @@ reloadUrlRewrite()
   Debug("url_rewrite", "remap.config updated, reloading...");
   newTable = new UrlRewrite();
   if (newTable->is_valid()) {
-    new_Deleter(rewrite_table, URL_REWRITE_TIMEOUT);
     static const char *msg = "remap.config done reloading!";
-    ink_atomic_swap(&rewrite_table, newTable);
+
+    // Hold at least one lease, until we reload the configuration
+    newTable->acquire();
+
+    ink_atomic_swap(&rewrite_table, newTable)->release(); // Swap configurations, and release the old one
     Debug("url_rewrite", "%s", msg);
     Note("%s", msg);
     return true;
   } else {
     static const char *msg = "failed to reload remap.config, not replacing!";
+
     delete newTable;
     Debug("url_rewrite", "%s", msg);
     Warning("%s", msg);
diff --git a/proxy/ReverseProxy.h b/proxy/ReverseProxy.h
index cbeb539..187dcd6 100644
--- a/proxy/ReverseProxy.h
+++ b/proxy/ReverseProxy.h
@@ -53,8 +53,8 @@ extern remap_plugin_info *remap_pi_list;
 // API Functions
 int init_reverse_proxy();
 
-mapping_type request_url_remap_redirect(HTTPHdr *request_header, URL *redirect_url);
-bool response_url_remap(HTTPHdr *response_header);
+mapping_type request_url_remap_redirect(HTTPHdr *request_header, URL *redirect_url, UrlRewrite *table);
+bool response_url_remap(HTTPHdr *response_header, UrlRewrite *table);
 
 // Reload Functions
 bool reloadUrlRewrite();
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index a61e036..347bb64 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -268,6 +268,7 @@ HttpSM::cleanup()
   // t_state.content_control.cleanup();
 
   HttpConfig::release(t_state.http_config_param);
+  m_remap->release();
 
   mutex.clear();
   tunnel.mutex.clear();
@@ -309,6 +310,8 @@ HttpSM::init()
   t_state.state_machine    = this;
 
   t_state.http_config_param = HttpConfig::acquire();
+  // Acquire a lease on the global remap / rewrite table (stupid global name ...)
+  m_remap = rewrite_table->acquire();
 
   // Simply point to the global config for the time being, no need to copy this
   // entire struct if nothing is going to change it.
@@ -3929,7 +3932,7 @@ HttpSM::state_remap_request(int event, void * /* data ATS_UNUSED */)
   case EVENT_REMAP_COMPLETE: {
     pending_action = nullptr;
     SMDebug("url_rewrite", "completed processor-based remapping request for [%" PRId64 "]", sm_id);
-    t_state.url_remap_success = remapProcessor.finish_remap(&t_state);
+    t_state.url_remap_success = remapProcessor.finish_remap(&t_state, m_remap);
     call_transact_and_set_next_state(nullptr);
     break;
   }
@@ -3949,7 +3952,7 @@ HttpSM::do_remap_request(bool run_inline)
   SMDebug("url_rewrite", "Starting a possible remapping for request [%" PRId64 "]", sm_id);
   bool ret = false;
   if (t_state.cop_test_page == false) {
-    ret = remapProcessor.setup_for_remap(&t_state);
+    ret = remapProcessor.setup_for_remap(&t_state, m_remap);
   }
 
   // Preserve effective url before remap
@@ -7206,7 +7209,7 @@ HttpSM::set_next_state()
     if (!remapProcessor.using_separate_thread()) {
       do_remap_request(true); /* run inline */
       SMDebug("url_rewrite", "completed inline remapping request for [%" PRId64 "]", sm_id);
-      t_state.url_remap_success = remapProcessor.finish_remap(&t_state);
+      t_state.url_remap_success = remapProcessor.finish_remap(&t_state, m_remap);
       if (t_state.next_action == HttpTransact::SM_ACTION_SEND_ERROR_CACHE_NOOP && t_state.transact_return_point == nullptr) {
         // It appears that we can now set the next_action to error and transact_return_point to nullptr when
         // going through do_remap_request presumably due to a plugin setting an error.  In that case, it seems
diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
index 94349e5..b47cc13 100644
--- a/proxy/http/HttpSM.h
+++ b/proxy/http/HttpSM.h
@@ -37,6 +37,7 @@
 #include "P_EventSystem.h"
 #include "HttpCacheSM.h"
 #include "HttpTransact.h"
+#include "UrlRewrite.h"
 #include "HttpTunnel.h"
 #include "InkAPIInternal.h"
 #include "../ProxyClientTransaction.h"
@@ -328,6 +329,10 @@ public:
 
   HttpTransact::State t_state;
 
+  // This unfortunately can't go into the t_state, beacuse of circular dependencies. We could perhaps refactor
+  // this, with a lot of work, but this is easier for now.
+  UrlRewrite *m_remap;
+
   // _postbuf api
   int64_t postbuf_reader_avail();
   int64_t postbuf_buffer_avail();
diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
index 874578b..7297888 100644
--- a/proxy/http/HttpTransact.cc
+++ b/proxy/http/HttpTransact.cc
@@ -1095,7 +1095,7 @@ HttpTransact::handleIfRedirect(State *s)
   int answer;
   URL redirect_url;
 
-  answer = request_url_remap_redirect(&s->hdr_info.client_request, &redirect_url);
+  answer = request_url_remap_redirect(&s->hdr_info.client_request, &redirect_url, s->state_machine->m_remap);
   if ((answer == PERMANENT_REDIRECT) || (answer == TEMPORARY_REDIRECT)) {
     int remap_redirect_len;
 
@@ -7761,7 +7761,7 @@ HttpTransact::build_response(State *s, HTTPHdr *base_response, HTTPHdr *outgoing
 
   // process reverse mappings on the location header
   // TS-1364: do this regardless of response code
-  response_url_remap(outgoing_response);
+  response_url_remap(outgoing_response, s->state_machine->m_remap);
 
   if (s->http_config_param->enable_http_stats) {
     HttpTransactHeaders::generate_and_set_squid_codes(outgoing_response, s->via_string, &s->squid_codes);
diff --git a/proxy/http/HttpTransact.h b/proxy/http/HttpTransact.h
index 0c1a8ae..240c473 100644
--- a/proxy/http/HttpTransact.h
+++ b/proxy/http/HttpTransact.h
@@ -36,11 +36,10 @@
 #include "ProxyConfig.h"
 #include "Transform.h"
 #include "Milestones.h"
-//#include "HttpAuthParams.h"
 #include "api/ts/remap.h"
 #include "RemapPluginInfo.h"
 #include "UrlMapping.h"
-#include <records/I_RecHttp.h>
+#include "records/I_RecHttp.h"
 
 #define MAX_DNS_LOOKUPS 2
 
diff --git a/proxy/http/remap/RemapProcessor.cc b/proxy/http/remap/RemapProcessor.cc
index 8e76989..8619941 100644
--- a/proxy/http/remap/RemapProcessor.cc
+++ b/proxy/http/remap/RemapProcessor.cc
@@ -38,13 +38,13 @@ RemapProcessor::start(int num_threads, size_t stacksize)
 
 /**
   Most of this comes from UrlRewrite::Remap(). Generally, all this does
-  is set "map" to the appropriate entry from the global rewrite_table
+  is set "map" to the appropriate entry from the HttpSM's leased m_remap
   such that we will then have access to the correct url_mapping inside
   perform_remap.
 
 */
 bool
-RemapProcessor::setup_for_remap(HttpTransact::State *s)
+RemapProcessor::setup_for_remap(HttpTransact::State *s, UrlRewrite *table)
 {
   Debug("url_rewrite", "setting up for remap: %p", s);
   URL *request_url        = nullptr;
@@ -56,13 +56,13 @@ RemapProcessor::setup_for_remap(HttpTransact::State *s)
   int request_port;
   bool proxy_request = false;
 
-  s->reverse_proxy = rewrite_table->reverse_proxy;
+  s->reverse_proxy = table->reverse_proxy;
   s->url_map.set(s->hdr_info.client_request.m_heap);
 
   ink_assert(redirect_url != nullptr);
 
-  if (unlikely((rewrite_table->num_rules_forward == 0) && (rewrite_table->num_rules_forward_with_recv_port == 0))) {
-    ink_assert(rewrite_table->forward_mappings.empty() && rewrite_table->forward_mappings_with_recv_port.empty());
+  if (unlikely((table->num_rules_forward == 0) && (table->num_rules_forward_with_recv_port == 0))) {
+    ink_assert(table->forward_mappings.empty() && table->forward_mappings_with_recv_port.empty());
     Debug("url_rewrite", "[lookup] No forward mappings found; Skipping...");
     return false;
   }
@@ -86,29 +86,29 @@ RemapProcessor::setup_for_remap(HttpTransact::State *s)
 
   Debug("url_rewrite", "[lookup] attempting %s lookup", proxy_request ? "proxy" : "normal");
 
-  if (rewrite_table->num_rules_forward_with_recv_port) {
+  if (table->num_rules_forward_with_recv_port) {
     Debug("url_rewrite", "[lookup] forward mappings with recv port found; Using recv port %d", s->client_info.dst_addr.port());
-    if (rewrite_table->forwardMappingWithRecvPortLookup(request_url, s->client_info.dst_addr.port(), request_host, request_host_len,
-                                                        s->url_map)) {
+    if (table->forwardMappingWithRecvPortLookup(request_url, s->client_info.dst_addr.port(), request_host, request_host_len,
+                                                s->url_map)) {
       Debug("url_rewrite", "Found forward mapping with recv port");
       mapping_found = true;
-    } else if (rewrite_table->num_rules_forward == 0) {
-      ink_assert(rewrite_table->forward_mappings.empty());
+    } else if (table->num_rules_forward == 0) {
+      ink_assert(table->forward_mappings.empty());
       Debug("url_rewrite", "No forward mappings left");
       return false;
     }
   }
 
   if (!mapping_found) {
-    mapping_found = rewrite_table->forwardMappingLookup(request_url, request_port, request_host, request_host_len, s->url_map);
+    mapping_found = table->forwardMappingLookup(request_url, request_port, request_host, request_host_len, s->url_map);
   }
 
   // If no rules match and we have a host, check empty host rules since
   // they function as default rules for server requests.
   // If there's no host, we've already done this.
-  if (!mapping_found && rewrite_table->nohost_rules && request_host_len) {
+  if (!mapping_found && table->nohost_rules && request_host_len) {
     Debug("url_rewrite", "[lookup] nothing matched");
-    mapping_found = rewrite_table->forwardMappingLookup(request_url, 0, "", 0, s->url_map);
+    mapping_found = table->forwardMappingLookup(request_url, 0, "", 0, s->url_map);
   }
 
   if (!proxy_request) { // do extra checks on a server request
@@ -138,7 +138,7 @@ RemapProcessor::setup_for_remap(HttpTransact::State *s)
 }
 
 bool
-RemapProcessor::finish_remap(HttpTransact::State *s)
+RemapProcessor::finish_remap(HttpTransact::State *s, UrlRewrite *table)
 {
   url_mapping *map        = nullptr;
   HTTPHdr *request_header = &s->hdr_info.client_request;
@@ -156,7 +156,7 @@ RemapProcessor::finish_remap(HttpTransact::State *s)
     return false;
   }
   // Do fast ACL filtering (it is safe to check map here)
-  rewrite_table->PerformACLFiltering(s, map);
+  table->PerformACLFiltering(s, map);
 
   // Check referer filtering rules
   if ((s->filter_mask & URL_REMAP_FILTER_REFERER) != 0 && (ri = map->referer_list) != nullptr) {
@@ -220,11 +220,11 @@ RemapProcessor::finish_remap(HttpTransact::State *s)
           *redirect_url                                  = ats_strdup(tmp_redirect_buf);
         }
       } else {
-        *redirect_url = ats_strdup(rewrite_table->http_default_redirect_url);
+        *redirect_url = ats_strdup(table->http_default_redirect_url);
       }
 
       if (*redirect_url == nullptr) {
-        *redirect_url = ats_strdup(map->filter_redirect_url ? map->filter_redirect_url : rewrite_table->http_default_redirect_url);
+        *redirect_url = ats_strdup(map->filter_redirect_url ? map->filter_redirect_url : table->http_default_redirect_url);
       }
       if (HTTP_STATUS_NONE == s->http_return_code)
         s->http_return_code = HTTP_STATUS_MOVED_TEMPORARILY;
diff --git a/proxy/http/remap/RemapProcessor.h b/proxy/http/remap/RemapProcessor.h
index d956677..760410a 100644
--- a/proxy/http/remap/RemapProcessor.h
+++ b/proxy/http/remap/RemapProcessor.h
@@ -45,8 +45,8 @@ class RemapProcessor : public Processor
 public:
   RemapProcessor() : ET_REMAP(0), _use_separate_remap_thread(false) {}
   ~RemapProcessor() {}
-  bool setup_for_remap(HttpTransact::State *s);
-  bool finish_remap(HttpTransact::State *s);
+  bool setup_for_remap(HttpTransact::State *s, UrlRewrite *table);
+  bool finish_remap(HttpTransact::State *s, UrlRewrite *table);
 
   Action *perform_remap(Continuation *cont, HttpTransact::State *s);
   int start(int num_threads, size_t stacksize);
diff --git a/proxy/http/remap/UrlRewrite.h b/proxy/http/remap/UrlRewrite.h
index d7a37ae..9850a21 100644
--- a/proxy/http/remap/UrlRewrite.h
+++ b/proxy/http/remap/UrlRewrite.h
@@ -51,7 +51,7 @@ enum mapping_type {
 /**
  *
 **/
-class UrlRewrite
+class UrlRewrite : public RefCountObj
 {
 public:
   UrlRewrite();
@@ -62,12 +62,31 @@ public:
   bool ReverseMap(HTTPHdr *response_header);
   void SetReverseFlag(int flag);
   void Print();
+
+  // The UrlRewrite object is-a RefCountObj, but this is a convenience to make it clear that we
+  // don't delete() these objects directly, but via the release() method only.
+  UrlRewrite *
+  acquire()
+  {
+    this->refcount_inc();
+    return this;
+  }
+
+  void
+  release()
+  {
+    if (0 == this->refcount_dec()) {
+      // Delete this on an ET_TASK thread, which avoids doing potentially slow things on an ET_NET thread.
+      Debug("url_rewrite", "Deleting old configuration immediately");
+      new_Deleter(this, 0);
+    }
+  }
+
   bool
   is_valid() const
   {
     return _valid;
   };
-  //  private:
 
   static const int MAX_REGEX_SUBS = 10;
 
diff --git a/proxy/logging/LogObject.cc b/proxy/logging/LogObject.cc
index 8d2643d..b4cac5e 100644
--- a/proxy/logging/LogObject.cc
+++ b/proxy/logging/LogObject.cc
@@ -860,13 +860,13 @@ LogObjectManager::LogObjectManager()
 LogObjectManager::~LogObjectManager()
 {
   for (unsigned i = 0; i < _objects.size(); ++i) {
-    if (REF_COUNT_OBJ_REFCOUNT_DEC(_objects[i]) == 0) {
+    if (_objects[i]->refcount_dec() == 0) {
       delete _objects[i];
     }
   }
 
   for (unsigned i = 0; i < _APIobjects.size(); ++i) {
-    if (REF_COUNT_OBJ_REFCOUNT_DEC(_APIobjects[i]) == 0) {
+    if (_APIobjects[i]->refcount_dec() == 0) {
       delete _APIobjects[i];
     }
   }
@@ -894,7 +894,7 @@ LogObjectManager::_manage_object(LogObject *log_object, bool is_api_object, int
       {
         // no conflicts, add object to the list of managed objects
         //
-        REF_COUNT_OBJ_REFCOUNT_INC(log_object);
+        log_object->refcount_inc();
         if (is_api_object) {
           _APIobjects.push_back(log_object);
         } else {
@@ -1222,10 +1222,10 @@ LogObjectManager::transfer_objects(LogObjectManager &old_mgr)
       if (*new_obj == *old_obj) {
         Debug("log-config-transfer", "keeping existing object %s", old_obj->get_base_filename());
 
-        REF_COUNT_OBJ_REFCOUNT_INC(old_obj);
+        old_obj->refcount_inc();
         this->_objects[j] = old_obj;
 
-        if (REF_COUNT_OBJ_REFCOUNT_DEC(new_obj) == 0) {
+        if (new_obj->refcount_dec() == 0) {
           delete new_obj;
         }
         ++num_kept_objects;

-- 
To stop receiving notification emails like this one, please contact
zwoop@apache.org.