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 2010/08/16 15:27:24 UTC

svn commit: r985919 - in /trafficserver/traffic/trunk: iocore/cache/Cache.cc iocore/eventsystem/P_EventSystem.h librecords/I_RecCore.h librecords/P_RecCore.i librecords/P_RecDefs.h librecords/RecCore.cc librecords/RecProcess.cc proxy/ReverseProxy.cc

Author: zwoop
Date: Mon Aug 16 13:27:24 2010
New Revision: 985919

URL: http://svn.apache.org/viewvc?rev=985919&view=rev
Log:
TS-411: Eliminate extraneous stats thread.

Tested on: FC13

This merges the remaining stat into using the normal "call back"
mechanism used by all other raw stats. I also eliminated the
bytes_used stats per partition now, since it's not remotely functional.

Modified:
    trafficserver/traffic/trunk/iocore/cache/Cache.cc
    trafficserver/traffic/trunk/iocore/eventsystem/P_EventSystem.h
    trafficserver/traffic/trunk/librecords/I_RecCore.h
    trafficserver/traffic/trunk/librecords/P_RecCore.i
    trafficserver/traffic/trunk/librecords/P_RecDefs.h
    trafficserver/traffic/trunk/librecords/RecCore.cc
    trafficserver/traffic/trunk/librecords/RecProcess.cc
    trafficserver/traffic/trunk/proxy/ReverseProxy.cc

Modified: trafficserver/traffic/trunk/iocore/cache/Cache.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/Cache.cc?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/Cache.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/Cache.cc Mon Aug 16 13:27:24 2010
@@ -136,7 +136,7 @@ static void cplist_update();
 int cplist_reconfigure();
 static int create_partition(int partition_number, int size_in_blocks, int scheme, CachePart *cp);
 static void rebuild_host_table(Cache *cache);
-void register_cache_stats(RecRawStatBlock *rsb, const char *prefix);
+void register_cache_stats(RecRawStatBlock *rsb, const char *prefix, bool reg_bytes_used=true);
 
 Queue<CachePart> cp_list;
 int cp_list_len = 0;
@@ -168,16 +168,16 @@ cache_bytes_total(void)
 }
 
 int
-cache_stats_bytes_used_cb(const char *name,
-                          RecDataT data_type, RecData *data, RecRawStatBlock *rsb, int id, void *cookie)
+cache_stats_bytes_used_cb(const char *name, RecDataT data_type, RecData *data, RecRawStatBlock *rsb, int id)
 {
   NOWARN_UNUSED(name);
   NOWARN_UNUSED(data_type);
   NOWARN_UNUSED(data);
-  NOWARN_UNUSED(cookie);
   if (cacheProcessor.initialized == CACHE_INITIALIZED) {
     RecSetGlobalRawStatSum(rsb, id, cache_bytes_used());
   }
+  RecRawStatSyncSum(name, data_type, data, rsb, id);
+
   return 1;
 }
 
@@ -614,10 +614,8 @@ CacheProcessor::diskInitialized()
       for (; cp; cp = cp->link.next) {
         cp->part_rsb = RecAllocateRawStatBlock((int) cache_stat_count);
         char part_stat_str_prefix[256];
-        snprintf(part_stat_str_prefix, sizeof(part_stat_str_prefix), "proxy.process.cache.partition_%d",
-                 cp->part_number);
+        snprintf(part_stat_str_prefix, sizeof(part_stat_str_prefix), "proxy.process.cache.partition_%d", cp->part_number);
         register_cache_stats(cp->part_rsb, part_stat_str_prefix);
-
       }
     }
 
@@ -627,7 +625,6 @@ CacheProcessor::diskInitialized()
     for (i = 0; i < gndisks; i++) {
       CacheDisk *d = gdisks[i];
       if (is_debug_tag_set("cache_hosting")) {
-
         int j;
         Debug("cache_hosting", "Disk: %d: Part Blocks: %ld: Free space: %ld",
               i, d->header->num_diskpart_blks, d->free_space);
@@ -2528,26 +2525,29 @@ Cache::key_to_part(CacheKey *key, char *
     return host_rec->parts[0];
 }
 
-static void reg_int(const char *str, int stat, RecRawStatBlock *rsb, const char *prefix) {
+static void reg_int(const char *str, int stat, RecRawStatBlock *rsb, const char *prefix, RecRawStatSyncCb sync_cb=RecRawStatSyncSum) {
   char stat_str[256];
   snprintf(stat_str, sizeof(stat_str), "%s.%s", prefix, str);
-  RecRegisterRawStat(rsb, RECT_PROCESS,
-                     stat_str, RECD_INT, RECP_NON_PERSISTENT, stat, RecRawStatSyncSum);
+  RecRegisterRawStat(rsb, RECT_PROCESS, stat_str, RECD_INT, RECP_NON_PERSISTENT, stat, sync_cb);
   DOCACHE_CLEAR_DYN_STAT(stat)
 }
 #define REG_INT(_str, _stat) reg_int(_str, (int)_stat, rsb, prefix)
 
 // Register Stats
 void
-register_cache_stats(RecRawStatBlock *rsb, const char *prefix)
+register_cache_stats(RecRawStatBlock *rsb, const char *prefix, bool reg_bytes_used)
 {
   char stat_str[256];
 
-  REG_INT("bytes_used", cache_bytes_used_stat);
+  // TODO: At some point we should figure out how to calculate bytes_used for the per partition stats. Right now they will be zero.
+  if (reg_bytes_used) {
+    REG_INT("bytes_used", cache_bytes_used_stat);
+  }
+ 
+
   REG_INT("bytes_total", cache_bytes_total_stat);
   snprintf(stat_str, sizeof(stat_str), "%s.%s", prefix, "ram_cache.total_bytes");
-  RecRegisterRawStat(rsb, RECT_PROCESS,
-                     stat_str, RECD_INT, RECP_NULL, (int) cache_ram_cache_bytes_total_stat, RecRawStatSyncSum);
+  RecRegisterRawStat(rsb, RECT_PROCESS, stat_str, RECD_INT, RECP_NULL, (int) cache_ram_cache_bytes_total_stat, RecRawStatSyncSum);
   REG_INT("ram_cache.bytes_used", cache_ram_cache_bytes_stat);
   REG_INT("ram_cache.hits", cache_ram_cache_hits_stat);
   REG_INT("pread_count", cache_pread_count_stat);
@@ -2599,8 +2599,7 @@ ink_cache_init(ModuleVersion v)
   cache_rsb = RecAllocateRawStatBlock((int) cache_stat_count);
 
 
-  IOCORE_RegisterConfigInteger(RECT_CONFIG,
-                               "proxy.config.cache.min_average_object_size", 8000, RECU_DYNAMIC, RECC_NULL, NULL);
+  IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.min_average_object_size", 8000, RECU_DYNAMIC, RECC_NULL, NULL);
 
   IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.ram_cache.size", -1, RECU_DYNAMIC, RECC_NULL, NULL);
   IOCORE_EstablishStaticConfigInteger(cache_config_ram_cache_size, "proxy.config.cache.ram_cache.size");
@@ -2613,13 +2612,11 @@ ink_cache_init(ModuleVersion v)
   IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.ram_cache.compress_percent", 90, RECU_DYNAMIC, RECC_NULL, NULL);
   IOCORE_EstablishStaticConfigInt32(cache_config_ram_cache_compress_percent, "proxy.config.cache.ram_cache.compress_percent");
 
-  IOCORE_RegisterConfigInteger(RECT_CONFIG,
-                               "proxy.config.cache.limits.http.max_alts", 3, RECU_DYNAMIC, RECC_NULL, NULL);
+  IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.limits.http.max_alts", 3, RECU_DYNAMIC, RECC_NULL, NULL);
   IOCORE_EstablishStaticConfigInt32(cache_config_http_max_alts, "proxy.config.cache.limits.http.max_alts");
   Debug("cache_init", "proxy.config.cache.limits.http.max_alts = %d", cache_config_http_max_alts);
 
-  IOCORE_RegisterConfigInteger(RECT_CONFIG,
-                             "proxy.config.cache.ram_cache_cutoff", 1048576, RECU_DYNAMIC, RECC_NULL, NULL);
+  IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.ram_cache_cutoff", 1048576, RECU_DYNAMIC, RECC_NULL, NULL);
   IOCORE_EstablishStaticConfigInteger(cache_config_ram_cache_cutoff, "proxy.config.cache.ram_cache_cutoff");
   Debug("cache_init", "cache_config_ram_cache_cutoff = %lld = %lldMb",
         cache_config_ram_cache_cutoff, cache_config_ram_cache_cutoff / (1024 * 1024));
@@ -2719,14 +2716,13 @@ ink_cache_init(ModuleVersion v)
                               "proxy.config.cache.partition_filename",
                               "partition.config", RECU_RESTART_TS, RECC_NULL, NULL);
 
-
   IOCORE_RegisterConfigString(RECT_CONFIG,
                               "proxy.config.cache.hosting_filename", "hosting.config", RECU_DYNAMIC, RECC_NULL, NULL);
 
-
-  register_cache_stats(cache_rsb, "proxy.process.cache");
-  IOCORE_RegisterStatUpdateFunc("proxy.process.cache.bytes_used",
-                                cache_rsb, (int) cache_bytes_used_stat, cache_stats_bytes_used_cb, NULL);
+  // Special case here for cache.bytes_used, since it uses a different CB than the "normal" SUM. This
+  // is a little ugly, but it's for one single stat, and saves an entire thread (and unecessary callbacks).
+  reg_int("bytes_used", cache_bytes_used_stat, cache_rsb, "proxy.process.cache", cache_stats_bytes_used_cb);
+  register_cache_stats(cache_rsb, "proxy.process.cache", false);
 
   const char *err = NULL;
   if ((err = theCacheStore.read_config())) {

Modified: trafficserver/traffic/trunk/iocore/eventsystem/P_EventSystem.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/eventsystem/P_EventSystem.h?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/eventsystem/P_EventSystem.h (original)
+++ trafficserver/traffic/trunk/iocore/eventsystem/P_EventSystem.h Mon Aug 16 13:27:24 2010
@@ -70,15 +70,14 @@
 #define IOCORE_SignalManager               REC_SignalManager
 
 // configuration macros
-#define IOCORE_RegisterConfigInteger        REC_RegisterConfigInteger
-#define IOCORE_RegisterConfigString         REC_RegisterConfigString
+#define IOCORE_RegisterConfigInteger        RecRegisterConfigInt
+#define IOCORE_RegisterConfigString         RecRegisterConfigString
 #define IOCORE_ReadConfigInt32              REC_ReadConfigInt32
 #define IOCORE_ReadConfigInteger            REC_ReadConfigInteger
 #define IOCORE_ReadConfigFloat              REC_ReadConfigFloat
 #define IOCORE_ReadConfigStringAlloc        REC_ReadConfigStringAlloc
 #define IOCORE_ReadConfigString             REC_ReadConfigString
 #define IOCORE_RegisterConfigUpdateFunc     REC_RegisterConfigUpdateFunc
-#define IOCORE_RegisterStatUpdateFunc       RecRegisterRawStatUpdateFunc
 #define IOCORE_EstablishStaticConfigInteger REC_EstablishStaticConfigInteger
 #define IOCORE_EstablishStaticConfigInt32   REC_EstablishStaticConfigInt32
 #define IOCORE_EstablishStaticConfigInt32U  REC_EstablishStaticConfigInt32U

Modified: trafficserver/traffic/trunk/librecords/I_RecCore.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/librecords/I_RecCore.h?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/librecords/I_RecCore.h (original)
+++ trafficserver/traffic/trunk/librecords/I_RecCore.h Mon Aug 16 13:27:24 2010
@@ -82,7 +82,6 @@ int RecLinkConfigCounter(const char *nam
 int RecLinkConfigString(const char *name, RecString * rec_string);
 
 int RecRegisterConfigUpdateCb(const char *name, RecConfigUpdateCb update_cb, void *cookie);
-int RecRegisterStatUpdateFunc(const char *name, RecStatUpdateFunc update_func, void *cookie);
 int RecRegisterRawStatUpdateFunc(const char *name, RecRawStatBlock * rsb,
                                  int id, RecStatUpdateFunc update_func, void *cookie);
 
@@ -140,9 +139,6 @@ void RecSignalAlarm(int, const char *);
 // Backwards Compatibility Items (REC_ prefix)
 //-------------------------------------------------------------------------
 
-#define REC_RegisterConfigInteger RecRegisterConfigInt
-#define REC_RegisterConfigString  RecRegisterConfigString
-
 #define REC_ReadConfigInt32(_var,_config_var_name) do { \
   RecInt tmp = 0; \
   RecGetRecordInt(_config_var_name, (RecInt*) &tmp); \

Modified: trafficserver/traffic/trunk/librecords/P_RecCore.i
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/librecords/P_RecCore.i?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/librecords/P_RecCore.i (original)
+++ trafficserver/traffic/trunk/librecords/P_RecCore.i Mon Aug 16 13:27:24 2010
@@ -643,59 +643,6 @@ RecSyncStatsFile()
 }
 
 //-------------------------------------------------------------------------
-// RecExecStatUpdateFuncs()
-//
-//   Wrapper for RecExecRawStatUpdateFuncs(). Used for non-raw statistics.
-//
-//-------------------------------------------------------------------------
-
-int
-RecExecStatUpdateFuncs()
-{
-
-  return RecExecRawStatUpdateFuncs();
-
-}
-
-//-------------------------------------------------------------------------
-// RecExecRawStatUpdateFuncs()
-//
-//   Parameters 'rsb' and 'id' only applicabel to raw stats.
-//
-//   Note: 1. Although we support a list of update functions, currently,
-//            only the first function is invovled.
-//         2. The update function is responsible to set the value. For
-//            example: RecSetGlobalRawStatSum(rsb, id, ???);
-//
-//-------------------------------------------------------------------------
-
-int
-RecExecRawStatUpdateFuncs()
-{
-
-  RecRecord *r;
-  int i, num_records;
-  RecStatUpdateFuncList *cur_function = NULL;
-
-  num_records = g_num_records;
-  for (i = 0; i < num_records; i++) {
-    r = &(g_records[i]);
-    rec_mutex_acquire(&(r->lock));
-    if (REC_TYPE_IS_STAT(r->rec_type)) {
-      if (r->stat_meta.update_func_list) {
-        cur_function = r->stat_meta.update_func_list;
-        (*(cur_function->update_func)) (r->name, r->data_type, &(r->data),
-                                        cur_function->rsb, cur_function->id, cur_function->update_cookie);
-      }
-    }
-    rec_mutex_release(&(r->lock));
-  }
-
-  return REC_ERR_OKAY;
-
-}
-
-//-------------------------------------------------------------------------
 // RecReadConfigFile
 //-------------------------------------------------------------------------
 

Modified: trafficserver/traffic/trunk/librecords/P_RecDefs.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/librecords/P_RecDefs.h?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/librecords/P_RecDefs.h (original)
+++ trafficserver/traffic/trunk/librecords/P_RecDefs.h Mon Aug 16 13:27:24 2010
@@ -91,7 +91,6 @@ struct RecStatMeta
   RecRawStatBlock *sync_rsb;
   int sync_id;
   RecPersistT persist_type;
-  RecStatUpdateFuncList *update_func_list;
 };
 
 struct RecConfigMeta

Modified: trafficserver/traffic/trunk/librecords/RecCore.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/librecords/RecCore.cc?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/librecords/RecCore.cc (original)
+++ trafficserver/traffic/trunk/librecords/RecCore.cc Mon Aug 16 13:27:24 2010
@@ -363,62 +363,6 @@ RecRegisterConfigUpdateCb(const char *na
 
 }
 
-//-------------------------------------------------------------------------
-// RecRegisterStatUpdateFunc
-//-------------------------------------------------------------------------
-
-int
-RecRegisterStatUpdateFunc(char *name, RecStatUpdateFunc update_func, void *cookie)
-{
-
-  return RecRegisterRawStatUpdateFunc(name, NULL, 0, update_func, cookie);
-
-}
-
-int
-RecRegisterRawStatUpdateFunc(const char *name, RecRawStatBlock * rsb, int id, RecStatUpdateFunc update_func, void *cookie)
-{
-
-  int err = REC_ERR_FAIL;
-  RecRecord *r;
-
-  ink_rwlock_rdlock(&g_records_rwlock);
-
-  if (ink_hash_table_lookup(g_records_ht, name, (void **) &r)) {
-    rec_mutex_acquire(&(r->lock));
-    if (REC_TYPE_IS_STAT(r->rec_type)) {
-      RecStatUpdateFuncList *new_function = (RecStatUpdateFuncList *) xmalloc(sizeof(RecStatUpdateFuncList));
-      memset(new_function, 0, sizeof(RecStatUpdateFuncList));
-      new_function->rsb = rsb;
-      new_function->id = id;
-      new_function->update_func = update_func;
-      new_function->update_cookie = cookie;
-      new_function->next = NULL;
-
-      ink_debug_assert(new_function);
-      if (!r->stat_meta.update_func_list) {
-        r->stat_meta.update_func_list = new_function;
-      } else {
-        RecStatUpdateFuncList *cur_function = NULL;
-        RecStatUpdateFuncList *prev_function = NULL;
-        for (cur_function = r->stat_meta.update_func_list; cur_function; cur_function = cur_function->next) {
-          prev_function = cur_function;
-        }
-        ink_debug_assert(prev_function);
-        ink_debug_assert(!prev_function->next);
-        prev_function->next = new_function;
-      }
-      err = REC_ERR_OKAY;
-    }
-
-    rec_mutex_release(&(r->lock));
-  }
-
-  ink_rwlock_unlock(&g_records_rwlock);
-
-  return err;
-
-}
 
 //-------------------------------------------------------------------------
 // RecGetRecordXXX

Modified: trafficserver/traffic/trunk/librecords/RecProcess.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/librecords/RecProcess.cc?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/librecords/RecProcess.cc (original)
+++ trafficserver/traffic/trunk/librecords/RecProcess.cc Mon Aug 16 13:27:24 2010
@@ -264,28 +264,6 @@ struct sync_cont:public Continuation
 };
 
 //-------------------------------------------------------------------------
-// stat_sync_cont
-//-------------------------------------------------------------------------
-
-struct stat_sync_cont:public Continuation
-{
-  stat_sync_cont(ProxyMutex * m):Continuation(m)
-  {
-    SET_HANDLER(&stat_sync_cont::sync);
-  }
-  int sync(int event, Event * e)
-  {
-    REC_NOWARN_UNUSED(event);
-    REC_NOWARN_UNUSED(e);
-    while (true) {
-      RecExecRawStatUpdateFuncs();
-      sleep(REC_STAT_UPDATE_INTERVAL_SEC);
-    }
-    return EVENT_DONE;
-  }
-};
-
-//-------------------------------------------------------------------------
 // RecProcessInit
 //-------------------------------------------------------------------------
 
@@ -385,9 +363,6 @@ RecProcessStart()
   sync_cont *sc = NEW(new sync_cont(new_ProxyMutex()));
   eventProcessor.spawn_thread(sc);
 
-  stat_sync_cont *ssc = NEW(new stat_sync_cont(new_ProxyMutex()));
-  eventProcessor.spawn_thread(ssc);
-
   g_started = true;
 
   return REC_ERR_OKAY;
@@ -734,6 +709,8 @@ RecRegisterRawStatSyncCb(const char *nam
         r->stat_meta.sync_id = id;
         r->stat_meta.sync_cb = sync_cb;
         err = REC_ERR_OKAY;
+      } else {
+        ink_release_assert(false); // We shouldn't register CBs twice...
       }
     }
     rec_mutex_release(&(r->lock));

Modified: trafficserver/traffic/trunk/proxy/ReverseProxy.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/ReverseProxy.cc?rev=985919&r1=985918&r2=985919&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/ReverseProxy.cc (original)
+++ trafficserver/traffic/trunk/proxy/ReverseProxy.cc Mon Aug 16 13:27:24 2010
@@ -75,7 +75,7 @@ int
 init_reverse_proxy()
 {
   // Do this before we create the table, which reads the value.
-  REC_RegisterConfigInteger(RECT_CONFIG, backdoor_var, 0, RECU_RESTART_TS, RECC_NULL, NULL);
+  RecRegisterConfigInt(RECT_CONFIG, backdoor_var, 0, RECU_RESTART_TS, RECC_NULL, NULL);
 
   ink_assert(rewrite_table == NULL);
   reconfig_mutex = new_ProxyMutex();