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 2014/05/16 02:05:10 UTC

git commit: TS-2788: proxy.config.alarm.bin should be reloadable

Repository: trafficserver
Updated Branches:
  refs/heads/master 51e16b2fb -> 50b872cf6


TS-2788: proxy.config.alarm.bin should be reloadable

Stop caching the value of proxy.config.alarm.bin and friends, so
it is reloadable.

Add a helper API, RecConfigReadBinDir(), to consistently handle
proxy.config.bin_path. Update all the places that uses
proxy.config.bin_path to use the new API.

Update the alarms configuration documentation. Remove various
unnecessary buffers and string caches.


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

Branch: refs/heads/master
Commit: 50b872cf6921215c98c63c178f6a42601b1b7fbb
Parents: 51e16b2
Author: James Peach <jp...@apache.org>
Authored: Wed May 7 15:58:15 2014 -0700
Committer: James Peach <jp...@apache.org>
Committed: Thu May 15 17:04:47 2014 -0700

----------------------------------------------------------------------
 CHANGES                                         |  2 +
 .../configuration/records.config.en.rst         | 34 ++++---
 lib/records/I_RecCore.h                         |  4 +
 lib/records/RecCore.cc                          | 17 ++++
 mgmt/Alarms.cc                                  | 94 +++++++++-----------
 mgmt/Alarms.h                                   |  5 +-
 mgmt/LocalManager.cc                            | 47 +++++-----
 mgmt/LocalManager.h                             |  3 -
 8 files changed, 112 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 215a2a5..af2576d 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.0.0
 
+  *) [TS-2788] Make proxy.config.alarm.bin reloadable.
+
   *) [TS-2811] Error logged in regex_remap when lowercase_substitutions option is used
 
   *) [TS-2791] SPDY POST transactions failing with ERR_CLIENT_ABORT

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/doc/reference/configuration/records.config.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/configuration/records.config.en.rst b/doc/reference/configuration/records.config.en.rst
index 2855034..c81cd6e 100644
--- a/doc/reference/configuration/records.config.en.rst
+++ b/doc/reference/configuration/records.config.en.rst
@@ -175,14 +175,6 @@ System Variables
    installation prefix. The ``$TS_ROOT`` environment variable can
    be used alter the installation prefix at run time.
 
-.. ts:cv:: CONFIG proxy.config.alarm_email STRING
-   :reloadable:
-
-   The email address to which Traffic Server sends alarm messages.
-
-During a custom Traffic Server installation, you can specify the email address;
-otherwise, Traffic Server uses the Traffic Server user account name as the default value for this variable.
-
 .. ts:cv:: CONFIG proxy.config.syslog_facility STRING LOG_DAEMON
 
    The facility used to record system log files. Refer to :ref:`understanding-traffic-server-log-files`.
@@ -415,14 +407,34 @@ Process Manager
 Alarm Configuration
 ===================
 
+.. ts:cv:: CONFIG proxy.config.alarm_email STRING
+   :reloadable:
+
+   The address to which the alarm script should send email.
+
 .. ts:cv:: CONFIG proxy.config.alarm.bin STRING example_alarm_bin.sh
+   :reloadable:
 
-   Name of the script file that can execute certain actions when an alarm is signaled. The default file is a sample script named
-   ``example_alarm_bin.sh`` located in the ``bin`` directory. You must edit the script to suit your needs.
+   Name of the script file that can execute certain actions when
+   an alarm is signaled. The script is invoked with up to 4 arguments:
+
+       - the alarm message
+       - the value of :ts:cv:`proxy.config.product_name`
+       - the value of :ts:cv:`proxy.config.admin.admin_user`
+       - the value of :ts:cv:`proxy.config.alarm_email`
 
 .. ts:cv:: CONFIG proxy.config.alarm.abs_path STRING NULL
+   :reloadable:
+
+   The absolute path to the directory containing the alarm script.
+   If this is not set, the script will be located relative to
+   :ts:cv:`proxy.config.bin_path`.
+
+.. ts:cv:: CONFIG proxy.config.alarm.script_runtime INT 5
+   :reloadable:
 
-   The full path to the script file that sends email to alert someone about Traffic Server problems.
+   The number of seconds that Traffic Server allows the alarm script
+   to run before aborting it.
 
 HTTP Engine
 ===========

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/lib/records/I_RecCore.h
----------------------------------------------------------------------
diff --git a/lib/records/I_RecCore.h b/lib/records/I_RecCore.h
index 38b2a66..9df6522 100644
--- a/lib/records/I_RecCore.h
+++ b/lib/records/I_RecCore.h
@@ -57,6 +57,10 @@ char * RecConfigReadSnapshotDir();
 // MUST release the result with ats_free().
 char * RecConfigReadLogDir();
 
+// Return a copy of the system's bin directory, taking proxy.config.bin_path into account. The caller MUST
+// release the result with ats_free().
+char * RecConfigReadBinDir();
+
 // Return a copy of a configuration file that is relative to sysconfdir. The relative path to the configuration
 // file is specified in the configuration variable named by "file_variable". If the configuration variable has no
 // value, NULL is returned. The caller MUST release the result with ats_free().

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/lib/records/RecCore.cc
----------------------------------------------------------------------
diff --git a/lib/records/RecCore.cc b/lib/records/RecCore.cc
index 9582312..91466dc 100644
--- a/lib/records/RecCore.cc
+++ b/lib/records/RecCore.cc
@@ -1098,6 +1098,23 @@ RecConfigReadLogDir()
 }
 
 //-------------------------------------------------------------------------
+// RecConfigReadBinDir
+//-------------------------------------------------------------------------
+char *
+RecConfigReadBinDir()
+{
+  char buf[PATH_NAME_MAX + 1];
+
+  buf[0] = '\0';
+  RecGetRecordString("proxy.config.bin_path", buf, PATH_NAME_MAX);
+  if (strlen(buf) > 0) {
+    return Layout::get()->relative(buf);
+  } else {
+    return ats_strdup(Layout::get()->bindir);
+  }
+}
+
+//-------------------------------------------------------------------------
 // RecConfigReadSnapshotDir.
 //-------------------------------------------------------------------------
 char *

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/mgmt/Alarms.cc
----------------------------------------------------------------------
diff --git a/mgmt/Alarms.cc b/mgmt/Alarms.cc
index 16c4149..2ec2f89 100644
--- a/mgmt/Alarms.cc
+++ b/mgmt/Alarms.cc
@@ -64,26 +64,29 @@ const char *alarmText[] = {
 
 const int alarmTextNum = sizeof(alarmText) / sizeof(char *);
 
-Alarms::Alarms()
+// Return the alarm script directory. Use proxy.config.alarm.abs_path if it is
+// set, falling back to proxy.config.bin_path otherwise.
+static char *
+alarm_script_dir()
 {
-  bool found;
+  char * path;
+
+  path = REC_readString("proxy.config.alarm.abs_path", NULL);
+  if (path && *path) {
+    return path;
+  }
 
+  return RecConfigReadBinDir();
+}
+
+Alarms::Alarms()
+{
   cur_cb = 0;
   cblist = ink_hash_table_create(InkHashTableKeyType_String);
   local_alarms = ink_hash_table_create(InkHashTableKeyType_String);
   remote_alarms = ink_hash_table_create(InkHashTableKeyType_String);
   ink_mutex_init(&mutex, "alarms-mutex");
-  alarm_bin = REC_readString("proxy.config.alarm.bin", &found);
-  ink_assert(found);
-  alarm_bin_path = REC_readString("proxy.config.alarm.abs_path", &found);
-  ink_assert(found);
-  if (!alarm_bin_path) {
-    alarm_bin_path = REC_readString("proxy.config.bin_path", &found);
-    ink_assert(found);
-  }
   alarmOEMcount = minOEMkey;
-
-  return;
 }                               /* End Alarms::Alarms */
 
 
@@ -93,7 +96,6 @@ Alarms::~Alarms()
   ink_hash_table_destroy_and_xfree_values(local_alarms);
   ink_hash_table_destroy_and_xfree_values(remote_alarms);
   ink_mutex_destroy(&mutex);
-  return;
 }                               /* End Alarms::Alarms */
 
 
@@ -107,7 +109,6 @@ Alarms::registerCallback(AlarmCallbackFunc func)
   Debug("alarm", "[Alarms::registerCallback] Registering Alarms callback\n");
   ink_hash_table_insert(cblist, cb_buf, (void *) func);
   ink_mutex_release(&mutex);
-  return;
 }                               /* End Alarms::registerCallback */
 
 
@@ -235,20 +236,19 @@ Alarms::signalAlarm(alarm_t a, const char *desc, const char *ip)
 
   Debug("alarm", "[Alarms::signalAlarm] Sending Alarm: '%s'", desc);
 
-  if (!desc)
+  if (!desc) {
     desc = (char *) getAlarmText(a);
+  }
 
   /*
    * Exec alarm bin for priority alarms everytime, regardless if they are
    * potentially duplicates. However, only exec this for you own alarms,
    * don't want every node in the cluster reporting the same alarm.
    */
-  if (priority == 1 && alarm_bin && alarm_bin_path && !ip) {
+  if (priority == 1 && !ip) {
     execAlarmBin(desc);
   }
 
-
-
   ink_mutex_acquire(&mutex);
   if (!ip) {
     // if an OEM alarm, then must create the unique key alarm type;
@@ -351,12 +351,12 @@ Alarms::signalAlarm(alarm_t a, const char *desc, const char *ip)
     Debug("alarm", "[Alarms::signalAlarm] invoke callback for %d", a);
     (*(func)) (a, ip, desc);
   }
+
   /* Priority 2 alarms get signalled if they are the first unsolved occurence. */
-  if (priority == 2 && alarm_bin && alarm_bin_path && !ip) {
+  if (priority == 2 && !ip) {
     execAlarmBin(desc);
   }
 
-  return;
 }                               /* End Alarms::signalAlarm */
 
 
@@ -495,27 +495,27 @@ Alarms::checkSystemNAlert()
 void
 Alarms::execAlarmBin(const char *desc)
 {
-  char cmd_line[1024];
-  char *alarm_email_from_name = 0;
-  char *alarm_email_from_addr = 0;
-  char *alarm_email_to_addr = 0;
-  bool found;
+  xptr<char> bindir(alarm_script_dir());
+  char cmd_line[MAXPATHLEN];
+
+  xptr<char> alarm_bin(REC_readString("proxy.config.alarm.bin", NULL));
+  xptr<char> alarm_email_from_name;
+  xptr<char> alarm_email_from_addr;
+  xptr<char> alarm_email_to_addr;
 
-  // get email info
-  alarm_email_from_name = REC_readString("proxy.config.product_name", &found);
-  if (!found)
-    alarm_email_from_name = 0;
-  alarm_email_from_addr = REC_readString("proxy.config.admin.admin_user", &found);
-  if (!found)
-    alarm_email_from_addr = 0;
-  alarm_email_to_addr = REC_readString("proxy.config.alarm_email", &found);
-  if (!found)
-    alarm_email_to_addr = 0;
-
-  int status;
   pid_t pid;
 
-  ink_filepath_make(cmd_line, sizeof(cmd_line), alarm_bin_path, alarm_bin);
+  // If there's no alarm script configured, don't even bother.
+  if (!alarm_bin || *alarm_bin == '\0') {
+    return;
+  }
+
+  // get email info
+  alarm_email_from_name = REC_readString("proxy.config.product_name", NULL);
+  alarm_email_from_addr = REC_readString("proxy.config.admin.admin_user", NULL);
+  alarm_email_to_addr = REC_readString("proxy.config.alarm_email", NULL);
+
+  ink_filepath_make(cmd_line, sizeof(cmd_line), bindir, alarm_bin);
 
 #ifdef POSIX_THREAD
   if ((pid = fork()) < 0)
@@ -525,11 +525,12 @@ Alarms::execAlarmBin(const char *desc)
   {
     mgmt_elog(stderr, errno, "[Alarms::execAlarmBin] Unable to fork1 process\n");
   } else if (pid > 0) {         /* Parent */
-    // INKqa11769
+    int status;
     bool script_done = false;
-    time_t timeout = (time_t) REC_readInteger("proxy.config.alarm.script_runtime", &found);
-    if (!found)
+    time_t timeout = (time_t) REC_readInteger("proxy.config.alarm.script_runtime", NULL);
+    if (!timeout) {
       timeout = 5;              // default time = 5 secs
+    }
     time_t time_delta = 0;
     time_t first_time = time(0);
     while (time_delta <= timeout) {
@@ -552,19 +553,12 @@ Alarms::execAlarmBin(const char *desc)
   } else {
     int res;
     if (alarm_email_from_name && alarm_email_from_addr && alarm_email_to_addr) {
-      res = execl(cmd_line, alarm_bin, desc, alarm_email_from_name, alarm_email_from_addr, alarm_email_to_addr, (char*)NULL);
+      res = execl(cmd_line, (const char *)alarm_bin, desc, (const char *)alarm_email_from_name, (const char *)alarm_email_from_addr, (const char *)alarm_email_to_addr, (char*)NULL);
     } else {
-      res = execl(cmd_line, alarm_bin, desc, (char*)NULL);
+      res = execl(cmd_line, (const char *)alarm_bin, desc, (char*)NULL);
     }
     _exit(res);
   }
-
-
-
-  // free memory
-  ats_free(alarm_email_from_name);
-  ats_free(alarm_email_from_addr);
-  ats_free(alarm_email_to_addr);
 }
 
 //

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/mgmt/Alarms.h
----------------------------------------------------------------------
diff --git a/mgmt/Alarms.h b/mgmt/Alarms.h
index 23bc81c..568d582 100644
--- a/mgmt/Alarms.h
+++ b/mgmt/Alarms.h
@@ -111,7 +111,7 @@ public:
   void registerCallback(AlarmCallbackFunc func);
   bool isCurrentAlarm(alarm_t a, char *ip = NULL);
 
-  void signalAlarm(alarm_t t, const char *desc = NULL, const char *ip = NULL);
+  void signalAlarm(alarm_t t, const char * desc, const char * ip = NULL);
   void resolveAlarm(alarm_t a, char *ip = NULL);
 
   void constructAlarmMessage(char *ip, char *message, int max);
@@ -132,9 +132,6 @@ private:
   int cur_cb;
   ink_mutex mutex;
 
-  char *alarm_bin;
-  char *alarm_bin_path;
-
   InkHashTable *cblist;
   InkHashTable *local_alarms;
   InkHashTable *remote_alarms;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/mgmt/LocalManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/LocalManager.cc b/mgmt/LocalManager.cc
index ce4b85a..42ddd79 100644
--- a/mgmt/LocalManager.cc
+++ b/mgmt/LocalManager.cc
@@ -202,6 +202,7 @@ LocalManager::LocalManager(bool proxy_on)
 {
   bool found;
   xptr<char> rundir(RecConfigReadRuntimeDir());
+  xptr<char> bindir(RecConfigReadBinDir());
 
 #ifdef MGMT_USE_SYSLOG
   syslog_facility = 0;
@@ -221,8 +222,6 @@ LocalManager::LocalManager(bool proxy_on)
     mgmt_sync_key = MGMT_SEMID_DEFAULT;
   }
 
-  ink_strlcpy(pserver_path, rundir, sizeof(pserver_path));
-
   virt_map = NULL;
 
   RecInt http_enabled = REC_readInteger("proxy.config.http.enabled", &found);
@@ -266,30 +265,21 @@ LocalManager::LocalManager(bool proxy_on)
   }
 #endif
 
-  bin_path = REC_readString("proxy.config.bin_path", &found);
   process_server_timeout_secs = REC_readInteger("proxy.config.lm.pserver_timeout_secs", &found);
   process_server_timeout_msecs = REC_readInteger("proxy.config.lm.pserver_timeout_msecs", &found);
   proxy_name = REC_readString("proxy.config.proxy_name", &found);
   proxy_binary = REC_readString("proxy.config.proxy_binary", &found);
   proxy_options = REC_readString("proxy.config.proxy_binary_opts", &found);
   env_prep = REC_readString("proxy.config.env_prep", &found);
-  // Calculate configured bin_path from the prefix
-  char *absolute_bin_path = Layout::get()->relative(bin_path);
-  ats_free(bin_path);
-  bin_path = absolute_bin_path;
+
   // Calculate proxy_binary from the absolute bin_path
-  absolute_proxy_binary = Layout::relative_to(absolute_bin_path, proxy_binary);
+  absolute_proxy_binary = Layout::relative_to(bindir, proxy_binary);
 
+  // coverity[fs_check_call]
   if (access(absolute_proxy_binary, R_OK | X_OK) == -1) {
-    // Try 'Layout::bindir' directory
-    ats_free(absolute_proxy_binary);
-    absolute_proxy_binary = Layout::relative_to(Layout::get()->bindir, proxy_binary);
-    // coverity[fs_check_call]
-    if (access(absolute_proxy_binary, R_OK | X_OK) == -1) {
-        mgmt_elog(0, "[LocalManager::LocalManager] Unable to access() '%s': %d, %s\n",
-                absolute_proxy_binary, errno, strerror(errno));
-        mgmt_fatal(0, "[LocalManager::LocalManager] please set bin path 'proxy.config.bin_path' \n");
-    }
+      mgmt_elog(0, "[LocalManager::LocalManager] Unable to access() '%s': %d, %s\n",
+              absolute_proxy_binary, errno, strerror(errno));
+      mgmt_fatal(0, "[LocalManager::LocalManager] please set bin path 'proxy.config.bin_path' \n");
   }
 
   internal_ticker = 0;
@@ -316,6 +306,7 @@ LocalManager::initAlarm()
 void
 LocalManager::initCCom(int mcport, char *addr, int rsport)
 {
+  xptr<char> rundir(RecConfigReadRuntimeDir());
   bool found;
   IpEndpoint cluster_ip;    // ip addr of the cluster interface
   ip_text_buffer clusterAddrStr;         // cluster ip addr as a String
@@ -361,7 +352,7 @@ LocalManager::initCCom(int mcport, char *addr, int rsport)
   ink_strlcat(envBuf, clusterAddrStr, envBuf_size);
   ink_release_assert(putenv(envBuf) == 0);
 
-  ccom = new ClusterCom(ats_ip4_addr_cast(&cluster_ip), hostname, mcport, addr, rsport, pserver_path);
+  ccom = new ClusterCom(ats_ip4_addr_cast(&cluster_ip), hostname, mcport, addr, rsport, rundir);
   virt_map = new VMap(intrName, ats_ip4_addr_cast(&cluster_ip), &lmgmt->ccom->mutex);
   virt_map->downAddrs();        // Just to be safe
   ccom->establishChannels();
@@ -377,7 +368,8 @@ LocalManager::initCCom(int mcport, char *addr, int rsport)
 void
 LocalManager::initMgmtProcessServer()
 {
-  char fpath[1024];
+  xptr<char> rundir(RecConfigReadRuntimeDir());
+  char fpath[MAXPATHLEN];
   int servlen, one = 1;
   struct sockaddr_un serv_addr;
 
@@ -387,7 +379,8 @@ LocalManager::initMgmtProcessServer()
   }
 #endif
 
-  snprintf(fpath, sizeof(fpath), "%s/%s", pserver_path, LM_CONNECTION_SERVER);
+  ink_filepath_make(fpath, sizeof(fpath), rundir, LM_CONNECTION_SERVER);
+
   unlink(fpath);
   if ((process_server_sockfd = socket(AF_UNIX, SOCK_STREAM, 0)) < 0) {
     mgmt_fatal(stderr, errno, "[LocalManager::initMgmtProcessServer] Unable to open socket exiting\n");
@@ -532,7 +525,7 @@ LocalManager::pollMgmtProcessServer()
 
           if (lmgmt->run_proxy) {
             mgmt_elog(0, "[Alarms::signalAlarm] Server Process was reset\n");
-            lmgmt->alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_PROCESS_DIED);
+            lmgmt->alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_PROCESS_DIED, NULL);
           } else {
             mgmt_log("[TrafficManager] Server process shutdown\n");
           }
@@ -565,7 +558,7 @@ LocalManager::handleMgmtMsgFromProcesses(MgmtMessageHdr * mh)
   switch (mh->msg_id) {
   case MGMT_SIGNAL_PID:
     watched_process_pid = *((pid_t *) data_raw);
-    lmgmt->alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_PROCESS_BORN);
+    lmgmt->alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_PROCESS_BORN, NULL);
     proxy_running++;
     proxy_launch_pid = -1;
     proxy_launch_outstanding = false;
@@ -772,7 +765,7 @@ LocalManager::sendMgmtMsgToProcesses(MgmtMessageHdr * mh)
             mgmt_elog(stderr, 0, "[LocalManager::pollMgmtProcessServer] " "Server Process has been terminated\n");
             if (lmgmt->run_proxy) {
               mgmt_elog(0, "[Alarms::signalAlarm] Server Process was reset\n");
-              lmgmt->alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_PROCESS_DIED);
+              lmgmt->alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_PROCESS_DIED, NULL);
             } else {
               mgmt_log("[TrafficManager] Server process shutdown\n");
             }
@@ -927,10 +920,12 @@ LocalManager::startProxy()
       waitpid(pid, &estatus, 0);
     } else {
       int res;
-      char env_prep_bin[1024];
 
-      snprintf(env_prep_bin, sizeof(env_prep_bin), "%s/%s", bin_path, env_prep);
-      res = execl(env_prep_bin, env_prep, (char*)NULL);
+      char env_prep_bin[MAXPATHLEN];
+      xptr<char> bindir(RecConfigReadBinDir());
+
+      ink_filepath_make(env_prep_bin, sizeof(env_prep_bin), bindir, env_prep);
+      res = execl(env_prep_bin, env_prep_bin, (char*)NULL);
       _exit(res);
     }
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/50b872cf/mgmt/LocalManager.h
----------------------------------------------------------------------
diff --git a/mgmt/LocalManager.h b/mgmt/LocalManager.h
index 8370a02..08a0104 100644
--- a/mgmt/LocalManager.h
+++ b/mgmt/LocalManager.h
@@ -57,7 +57,6 @@ public:
     delete alarm_keeper;
     delete virt_map;
     delete ccom;
-    ats_free(bin_path);
     ats_free(absolute_proxy_binary);
     ats_free(proxy_name);
     ats_free(proxy_binary);
@@ -123,8 +122,6 @@ public:
   int process_server_timeout_secs;
   int process_server_timeout_msecs;
 
-  char pserver_path[PATH_NAME_MAX];
-  char *bin_path;
   char *absolute_proxy_binary;
   char *proxy_name;
   char *proxy_binary;