You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by jp...@apache.org on 2012/12/03 05:43:58 UTC

[1/2] git commit: TS-1608: IpAllow should use standard config update

Updated Branches:
  refs/heads/master db0cfd442 -> c859379cc


TS-1608: IpAllow should use standard config update


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

Branch: refs/heads/master
Commit: c859379ccdf4b1651094ca2e71081b3e02d9a0a4
Parents: fe74191
Author: James Peach <jp...@apache.org>
Authored: Sun Dec 2 20:40:43 2012 -0800
Committer: James Peach <jp...@apache.org>
Committed: Sun Dec 2 20:40:43 2012 -0800

----------------------------------------------------------------------
 CHANGES                  |    3 +
 iocore/net/P_SSLConfig.h |   20 +--------
 mgmt/ProxyConfig.h       |   15 ++++++-
 proxy/IPAllow.cc         |   87 +++++++++++++++--------------------------
 proxy/IPAllow.h          |   20 ++++++----
 proxy/http/HttpAccept.cc |    3 +-
 6 files changed, 64 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/c859379c/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 30a2d56..3d925da 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,5 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 3.3.1
+
+  *) [TS-1608] IpAllow should use standard config update
+
   *) [TS-1580] Mutex leak plugged.
 
   *) [TS-1596] Added channel_stats plugin to experimental.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/c859379c/iocore/net/P_SSLConfig.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLConfig.h b/iocore/net/P_SSLConfig.h
index c942a8b..4519e87 100644
--- a/iocore/net/P_SSLConfig.h
+++ b/iocore/net/P_SSLConfig.h
@@ -33,22 +33,6 @@
 
 #include "ProxyConfig.h"
 
-namespace ssl { namespace detail {
-
-template <typename ClassType, typename ConfigType>
-struct scoped_config {
-  scoped_config() : ptr(ClassType::acquire()) {}
-  ~scoped_config() { ClassType::release(ptr); }
-
-  operator const ConfigType * () const { return ptr; }
-  const ConfigType * operator->() const { return ptr; }
-
-private:
-  ConfigType * ptr;
-};
-
-}}
-
 struct SSLCertLookup;
 
 /////////////////////////////////////////////////////////////
@@ -108,7 +92,7 @@ struct SSLConfig
   static SSLConfigParams * acquire();
   static void release(SSLConfigParams * params);
 
-  typedef ssl::detail::scoped_config<SSLConfig, SSLConfigParams> scoped_config;
+  typedef ConfigProcessor::scoped_config<SSLConfig, SSLConfigParams> scoped_config;
 
 private:
   static int configid;
@@ -121,7 +105,7 @@ struct SSLCertificateConfig
   static SSLCertLookup * acquire();
   static void release(SSLCertLookup * params);
 
-  typedef ssl::detail::scoped_config<SSLCertificateConfig, SSLCertLookup> scoped_config;
+  typedef ConfigProcessor::scoped_config<SSLCertificateConfig, SSLCertLookup> scoped_config;
 
 private:
   static int configid;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/c859379c/mgmt/ProxyConfig.h
----------------------------------------------------------------------
diff --git a/mgmt/ProxyConfig.h b/mgmt/ProxyConfig.h
index 6a3811d..23e4bba 100644
--- a/mgmt/ProxyConfig.h
+++ b/mgmt/ProxyConfig.h
@@ -49,10 +49,8 @@ void *config_string_alloc_cb(void *data, void *value);
 #define RegisterMgmtCallback(_signal,_fn,_data) \
 pmgmt->registerMgmtCallback(_signal,_fn,_data)
 
-
 #define MAX_CONFIGS  100
 
-
 struct ConfigInfo
 {
   volatile int m_refcount;
@@ -68,6 +66,19 @@ class ConfigProcessor
 public:
   ConfigProcessor();
 
+  template <typename ClassType, typename ConfigType>
+  struct scoped_config {
+    scoped_config() : ptr(ClassType::acquire()) {}
+    ~scoped_config() { ClassType::release(ptr); }
+
+    operator bool() const { return ptr != 0; }
+    operator const ConfigType * () const { return ptr; }
+    const ConfigType * operator->() const { return ptr; }
+
+  private:
+    ConfigType * ptr;
+  };
+
   unsigned int set(unsigned int id, ConfigInfo * info);
   ConfigInfo *get(unsigned int id);
   void release(unsigned int id, ConfigInfo * data);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/c859379c/proxy/IPAllow.cc
----------------------------------------------------------------------
diff --git a/proxy/IPAllow.cc b/proxy/IPAllow.cc
index b5c2b1e..093cf73 100644
--- a/proxy/IPAllow.cc
+++ b/proxy/IPAllow.cc
@@ -39,66 +39,35 @@
 
 #include <sstream>
 
-#define IPAllowRegisterConfigUpdateFunc REC_RegisterConfigUpdateFunc
-#define IPAllowReadConfigStringAlloc REC_ReadConfigStringAlloc
-
 enum AclOp {
   ACL_OP_ALLOW, ///< Allow access.
   ACL_OP_DENY, ///< Deny access.
 };
 
-IpAllow* IpAllow::_instance = NULL;
-
 // Mask for all methods.
 // This can't be computed properly at process start, so it's delayed
 // until the instance is initialized.
 uint32_t IpAllow::ALL_METHOD_MASK;
 
-static Ptr<ProxyMutex> ip_reconfig_mutex;
+int IpAllow::configid = 0;
 
-//
-// struct IPAllow_FreerContinuation
-// Continuation to free old cache control lists after
-//  a timeout
-//
-struct IPAllow_FreerContinuation;
-typedef int (IPAllow_FreerContinuation::*IPAllow_FrContHandler) (int, void *);
-struct IPAllow_FreerContinuation: public Continuation
-{
-  IpAllow *p;
-  int freeEvent(int event, Event * e)
-  {
-    NOWARN_UNUSED(event);
-    NOWARN_UNUSED(e);
-    Debug("ip-allow", "Deleting old table");
-    delete p;
-    delete this;
-    return EVENT_DONE;
-  }
-  IPAllow_FreerContinuation(IpAllow * ap):Continuation(NULL), p(ap)
-  {
-    SET_HANDLER((IPAllow_FrContHandler) & IPAllow_FreerContinuation::freeEvent);
-  }
-};
+static Ptr<ProxyMutex> ip_reconfig_mutex;
 
-// struct IPAllow_UpdateContinuation
+// struct IpAllowUpdate
 //
 //   Used to read the ip_allow.conf file after the manager signals
 //      a change
 //
-struct IPAllow_UpdateContinuation: public Continuation
+struct IpAllowUpdate: public Continuation
 {
-  int file_update_handler(int etype, void *data)
-  {
-    NOWARN_UNUSED(etype);
-    NOWARN_UNUSED(data);
+  int update(int /* etype */, void * /* data */) {
     IpAllow::ReloadInstance();
     delete this;
-      return EVENT_DONE;
+    return EVENT_DONE;
   }
-  IPAllow_UpdateContinuation(ProxyMutex * m):Continuation(m)
-  {
-    SET_HANDLER(&IPAllow_UpdateContinuation::file_update_handler);
+
+  IpAllowUpdate(ProxyMutex * m):Continuation(m) {
+    SET_HANDLER(&IpAllowUpdate::update);
   }
 };
 
@@ -109,7 +78,7 @@ ipAllowFile_CB(const char *name, RecDataT data_type, RecData data, void *cookie)
   NOWARN_UNUSED(data_type);
   NOWARN_UNUSED(data);
   NOWARN_UNUSED(cookie);
-  eventProcessor.schedule_imm(NEW(new IPAllow_UpdateContinuation(ip_reconfig_mutex)), ET_CACHE);
+  eventProcessor.schedule_imm(NEW(new IpAllowUpdate(ip_reconfig_mutex)), ET_CACHE);
   return 0;
 }
 
@@ -117,33 +86,41 @@ ipAllowFile_CB(const char *name, RecDataT data_type, RecData data, void *cookie)
 //   Begin API functions
 //
 void
-IpAllow::InitInstance() {
+IpAllow::InitInstance()
+{
   // Should not have been initialized before
-  ink_assert(_instance == NULL);
+  ink_assert(IpAllow::configid == 0);
 
   ALL_METHOD_MASK = ~0;
 
   ip_reconfig_mutex = new_ProxyMutex();
-
-  _instance = NEW(new self("proxy.config.cache.ip_allow.filename", "IpAllow", "ip_allow"));
-  _instance->BuildTable();
-
-  IPAllowRegisterConfigUpdateFunc("proxy.config.cache.ip_allow.filename", ipAllowFile_CB, NULL);
+  REC_RegisterConfigUpdateFunc("proxy.config.cache.ip_allow.filename", ipAllowFile_CB, NULL);
+  ReloadInstance();
 }
 
 void
-IpAllow::ReloadInstance() {
+IpAllow::ReloadInstance()
+{
   self *new_table;
 
-  Debug("ip-allow", "ip_allow.config updated, reloading");
-
-  // Schedule the current table for deallocation in the future
-  eventProcessor.schedule_in(NEW(new IPAllow_FreerContinuation(_instance)), IP_ALLOW_TIMEOUT, ET_CACHE);
+  Note("ip_allow.config updated, reloading");
 
   new_table = NEW(new self("proxy.config.cache.ip_allow.filename", "IpAllow", "ip_allow"));
   new_table->BuildTable();
 
-  ink_atomic_swap(&_instance, new_table);
+  configid = configProcessor.set(configid, new_table);
+}
+
+IpAllow *
+IpAllow::acquire()
+{
+  return (IpAllow *)configProcessor.get(configid);
+}
+
+void
+IpAllow::release(IpAllow * lookup)
+{
+  configProcessor.release(configid, lookup);
 }
 
 //
@@ -165,7 +142,7 @@ IpAllow::IpAllow(
   config_file_var = ats_strdup(config_var);
   config_file_path[0] = '\0';
 
-  IPAllowReadConfigStringAlloc(config_file, (char *) config_file_var);
+  REC_ReadConfigStringAlloc(config_file, (char *) config_file_var);
   ink_release_assert(config_file != NULL);
   ink_filepath_make(config_file_path, sizeof(config_file_path), system_config_directory, config_file);
   ats_free(config_file);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/c859379c/proxy/IPAllow.h
----------------------------------------------------------------------
diff --git a/proxy/IPAllow.h b/proxy/IPAllow.h
index 80018e9..456f6aa 100644
--- a/proxy/IPAllow.h
+++ b/proxy/IPAllow.h
@@ -36,9 +36,10 @@
 #include "ts/IpMap.h"
 #include "vector"
 #include "ts/Vec.h"
+#include "ProxyConfig.h"
 
 // forward declare in name only so it can be a friend.
-struct IPAllow_UpdateContinuation;
+struct IpAllowUpdate;
 
 //
 // Timeout the IpAllowTable * this amount of time after the
@@ -63,9 +64,11 @@ struct AclRecord {
 
 /** Singleton class for access controls.
  */
-class IpAllow {
+class IpAllow : public ConfigInfo
+{
   friend int main(int, char**);
-  friend struct IPAllow_UpdateContinuation;
+  friend struct IpAllowUpdate;
+
 public:
   typedef IpAllow self; ///< Self reference type.
 
@@ -77,15 +80,18 @@ public:
   uint32_t match(sockaddr const* ip) const;
 
   /// @return The global instance.
-  static self* instance();
+  static IpAllow * acquire();
+  static void release(IpAllow * params);
 
   static bool CheckMask(uint32_t, int);
   /// @return A mask that permits all methods.
   static uint32_t AllMethodMask() {
     return ALL_METHOD_MASK;
   }
-private:
 
+  typedef ConfigProcessor::scoped_config<IpAllow, IpAllow> scoped_config;
+
+private:
   static void InitInstance();
   static void ReloadInstance();
   static uint32_t MethodIdxToMask(int);
@@ -98,11 +104,9 @@ private:
   Vec<AclRecord> _acls;
   static uint32_t ALL_METHOD_MASK;
 
-  static self* _instance;
+  static int configid;
 };
 
-inline IpAllow* IpAllow::instance() { return _instance; }
-
 inline uint32_t IpAllow::MethodIdxToMask(int idx) { return 1 << (idx - HTTP_WKSIDX_CONNECT); }
 
 inline uint32_t

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/c859379c/proxy/http/HttpAccept.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpAccept.cc b/proxy/http/HttpAccept.cc
index 88c04d4..6a7d599 100644
--- a/proxy/http/HttpAccept.cc
+++ b/proxy/http/HttpAccept.cc
@@ -43,12 +43,13 @@ HttpAccept::mainEvent(int event, void *data)
     sockaddr const* client_ip = netvc->get_remote_addr();
     uint32_t acl_method_mask = 0;
     ip_port_text_buffer ipb;
+    IpAllow::scoped_config ipallow;
 
     // The backdoor port is now only bound to "localhost", so no
     // reason to check for if it's incoming from "localhost" or not.
     if (backdoor) {
       acl_method_mask = IpAllow::AllMethodMask();
-    } else if (IpAllow::instance() && ((acl_method_mask = IpAllow::instance()->match(client_ip)) == 0)) {
+    } else if (ipallow && ((acl_method_mask = ipallow->match(client_ip)) == 0)) {
       Warning("client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
       netvc->do_io_close();