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/11/14 01:33:50 UTC

[1/2] trafficserver git commit: TS-3192: implement proxy.config.config_dir

Repository: trafficserver
Updated Branches:
  refs/heads/master edff64d19 -> 435e75661


TS-3192: implement proxy.config.config_dir

proxy.config.config_dir is supposed to nominate the configuration
directory. However, it has never been implemented making it difficult
to bootstrap a Traffic Server instance into a configuration environment
that was not compiled in.

Add the RecConfigReadConfigDir() and RecConfigReadConfigPath()
management APIs to deal with correctly allowing proxy.config.config_dir
to override the compiled in layout. Apply these everywhere that
uses sysconfdir so that we get consistent behavior.

Separate default records configuration for traffic_cop. We don't
want traffic_cop to pull in all of libmgmt, but we need the default
records values so that we can correctly process environment overrides.
Split RecordsConfig.cc into data and API components so that traffic_cop
can get the raw data. Update traffic_cop to initialize its defaults
before loading records.config.

Merge environment veriable overrides into LibRecordsConfigInit()
and make sure we always call this. This solves initializatino
problems in early startup when traffic_server needs to know
configuration paths before it can sync with traffic_manager. Without
this we can only use TS_ROOT and the compiled-in layout.


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

Branch: refs/heads/master
Commit: d513cf7412d20670fce512b1e910ec5aff83e13e
Parents: edff64d
Author: James Peach <jp...@apache.org>
Authored: Tue Nov 11 10:29:01 2014 -0800
Committer: James Peach <jp...@apache.org>
Committed: Thu Nov 13 16:33:14 2014 -0800

----------------------------------------------------------------------
 CHANGES                                  |   2 +
 ci/tsqa/functions                        |   2 +-
 cmd/traffic_cop/Makefile.am              |   1 +
 cmd/traffic_cop/traffic_cop.cc           | 120 ++++++++++++++-----
 cmd/traffic_crashlog/traffic_crashlog.cc |   1 -
 cmd/traffic_layout/traffic_layout.cc     |  21 +---
 cmd/traffic_manager/traffic_manager.cc   |  74 ++----------
 iocore/cluster/ClusterMachine.cc         |   4 +-
 iocore/cluster/P_ClusterMachine.h        |   2 +-
 iocore/net/SSLConfig.cc                  |   9 +-
 iocore/net/Socks.cc                      |  15 +--
 lib/records/I_RecCore.h                  |   4 +
 lib/records/RecCore.cc                   |  61 ++++++----
 lib/records/RecUtils.cc                  |   2 +-
 mgmt/LocalManager.cc                     |  18 ++-
 mgmt/Makefile.am                         |  15 ++-
 mgmt/ProcessManager.cc                   |  11 +-
 mgmt/ProcessManager.h                    |   1 -
 mgmt/RecordsConfig.cc                    | 145 ++---------------------
 mgmt/RecordsConfig.h                     |   6 +-
 mgmt/RecordsConfigUtils.cc               | 160 ++++++++++++++++++++++++++
 mgmt/Rollback.cc                         |  20 ++--
 mgmt/api/CoreAPI.cc                      |  45 ++------
 mgmt/cluster/VMap.cc                     |   4 +-
 mgmt/cluster/VMap.h                      |   2 +-
 mgmt/web2/WebIntrMain.cc                 |   2 +-
 proxy/InkAPI.cc                          |   9 +-
 proxy/Main.cc                            |  12 +-
 proxy/Plugin.cc                          |   7 +-
 proxy/Update.cc                          |   2 +-
 proxy/config/records.config.default.in   |   8 --
 proxy/http/remap/RemapConfig.cc          |   4 +-
 proxy/http/remap/UrlRewrite.cc           |  15 +--
 proxy/logging/LogConfig.cc               |  27 +----
 proxy/logging/LogConfig.h                |   2 -
 proxy/logging/LogStandalone.cc           |   7 +-
 36 files changed, 410 insertions(+), 430 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index dabd166..4ea3804 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.2.0
 
+  *) [TS-3192] Implement proxy.config.config_dir.
+
   *) [TS-3195] Improved crash logging.
 
   *) [TS-3194] Remove unused proxy.config.plugin.plugin_mgmt_dir configuration.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/ci/tsqa/functions
----------------------------------------------------------------------
diff --git a/ci/tsqa/functions b/ci/tsqa/functions
index 60783b1..f839108 100644
--- a/ci/tsqa/functions
+++ b/ci/tsqa/functions
@@ -254,7 +254,7 @@ bootstrap() {
   done
 
   # Copy config across
-  cp -r $(tsxs -q SYSCONFDIR)/*.config $TSQA_ROOT/$sysconfdir
+  cp -r $(tsxs -q SYSCONFDIR)/*.{config,xml} $TSQA_ROOT/$sysconfdir
 
   # Delete any config variables we are about to set.
   sed -i.orig \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/cmd/traffic_cop/Makefile.am
----------------------------------------------------------------------
diff --git a/cmd/traffic_cop/Makefile.am b/cmd/traffic_cop/Makefile.am
index 35296ca..9176948 100644
--- a/cmd/traffic_cop/Makefile.am
+++ b/cmd/traffic_cop/Makefile.am
@@ -33,6 +33,7 @@ traffic_cop_SOURCES = \
 traffic_cop_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
 
 traffic_cop_LDADD = \
+  $(top_builddir)/mgmt/libmgmt_c.la \
   $(top_builddir)/mgmt/api/libtsmgmt.la \
   $(top_builddir)/lib/ts/libtsutil.la \
   $(top_builddir)/lib/records/librecords_cop.a \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/cmd/traffic_cop/traffic_cop.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_cop/traffic_cop.cc b/cmd/traffic_cop/traffic_cop.cc
index a195f56..ee65d63 100644
--- a/cmd/traffic_cop/traffic_cop.cc
+++ b/cmd/traffic_cop/traffic_cop.cc
@@ -26,6 +26,7 @@
 #include "I_Version.h"
 #include "I_RecCore.h"
 #include "mgmtapi.h"
+#include "RecordsConfig.h"
 #include "ClusterCom.h"
 #include "ink_cap.h"
 
@@ -91,7 +92,6 @@ static char server_binary[PATH_NAME_MAX] = "traffic_server";
 static char manager_options[OPTIONS_LEN_MAX] = "";
 
 static char log_file[PATH_NAME_MAX] = "traffic.out";
-static char bin_path[PATH_NAME_MAX] = "bin";
 
 static int autoconf_port = 8083;
 static int rs_port = 8088;
@@ -142,9 +142,13 @@ static void get_admin_user(void);
 
 struct ConfigValue
 {
+  ConfigValue()
+    : config_type(RECT_NULL), data_type(RECD_NULL) {
+  }
+
   ConfigValue(RecT _t, RecDataT _d, const std::string& _v)
     : config_type(_t), data_type(_d), data_value(_v) {
-    }
+  }
 
   RecT        config_type;
   RecDataT    data_type;
@@ -465,7 +469,16 @@ transient_error(int error, int wait_ms)
 static void
 config_register_variable(RecT rec_type, RecDataT data_type, const char * name, const char * value, bool /* inc_version */)
 {
-  configTable.insert(std::make_pair(std::string(name), ConfigValue(rec_type, data_type, value)));
+  configTable[std::string(name)] = ConfigValue(rec_type, data_type, value);
+}
+
+static void
+config_register_default(const RecordElement * record, void *)
+{
+  if (record->type == RECT_CONFIG || record->type == RECT_LOCAL) {
+    const char * value = record->value ? record->value : ""; // splooch NULL values so std::string can swallow them
+    configTable[std::string(record->name)] = ConfigValue(record->type, record->value_type, value);
+  }
 }
 
 static void
@@ -518,7 +531,7 @@ ConfigIntFatalError:
   exit(1);
 }
 
-static const char *
+static char *
 config_read_runtime_dir()
 {
   char state_dir[PATH_NAME_MAX + 1];
@@ -532,15 +545,60 @@ config_read_runtime_dir()
   }
 }
 
+static char *
+config_read_sysconfig_dir()
+{
+  char sysconfig_dir[PATH_NAME_MAX + 1];
+
+  sysconfig_dir[0] = '\0';
+  config_read_string("proxy.config.config_dir", sysconfig_dir, sizeof(sysconfig_dir), true);
+  if (strlen(sysconfig_dir) > 0) {
+    return Layout::get()->relative(sysconfig_dir);
+  } else {
+    return ats_strdup(Layout::get()->sysconfdir);
+  }
+}
+
+static char *
+config_read_bin_dir()
+{
+  char bindir[PATH_NAME_MAX + 1];
+
+  bindir[0] = '\0';
+  config_read_string("proxy.config.bin_path", bindir, sizeof(bindir), true);
+  cop_log(COP_DEBUG, "binpath is %s\n", bindir);
+  if (strlen(bindir) > 0) {
+    return Layout::get()->relative(bindir);
+  } else {
+    return ats_strdup(Layout::get()->bindir);
+  }
+}
+
+static char *
+config_read_log_dir()
+{
+  char logdir[PATH_NAME_MAX + 1];
+
+  logdir[0] = '\0';
+  config_read_string("proxy.config.log.logfile_dir", logdir, sizeof(logdir), true);
+  if (strlen(logdir) > 0) {
+    return Layout::get()->relative(logdir);
+  } else {
+    return ats_strdup(Layout::get()->logdir);
+  }
+}
+
 static void
 config_reload_records()
 {
   struct stat stat_buf;
   static time_t last_mod = 0;
-  char log_dir[PATH_NAME_MAX];
   char log_filename[PATH_NAME_MAX];
   int tmp_int;
 
+  ats_scoped_str bindir;
+  ats_scoped_str logdir;
+
   cop_log_trace("Entering %s()\n", __func__);
   // coverity[fs_check_call]
   if (stat(config_file, &stat_buf) == -1) {
@@ -555,6 +613,7 @@ config_reload_records()
   }
 
   configTable.clear();
+  RecordsConfigIterate(config_register_default, NULL);
 
   if (RecConfigFileParse(config_file, config_register_variable, false) != REC_ERR_OKAY) {
     cop_log(COP_FATAL, "could not parse \"%s\"\n", config_file);
@@ -565,28 +624,23 @@ config_reload_records()
   config_read_string("proxy.config.proxy_binary", server_binary, sizeof(server_binary), true);
   get_admin_user();
 
-  config_read_string("proxy.config.bin_path", bin_path, sizeof(bin_path), true);
-  Layout::get()->relative(bin_path, sizeof(bin_path), bin_path);
-  if (access(bin_path, R_OK) == -1) {
-    ink_strlcpy(bin_path, Layout::get()->bindir, sizeof(bin_path));
-    if (access(bin_path, R_OK) == -1) {
-      cop_log(COP_FATAL, "could not access() \"%s\"\n", bin_path);
-      cop_log(COP_FATAL, "please set 'proxy.config.bin_path' \n");
-      exit(1);
-    }
+  bindir = config_read_bin_dir();
+  if (access(bindir, R_OK) == -1) {
+    cop_log(COP_FATAL, "could not access() \"%s\"\n", (const char *)bindir);
+    cop_log(COP_FATAL, "please set 'proxy.config.bin_path' \n");
+    exit(1);
   }
-  config_read_string("proxy.config.log.logfile_dir", log_dir, sizeof(log_dir));
-  Layout::get()->relative(log_dir, sizeof(log_dir), log_dir);
-  if (access(log_dir, W_OK) == -1) {
-    ink_strlcpy(log_dir, Layout::get()->logdir, sizeof(log_dir));
-    if (access(log_dir, W_OK) == -1) {
-      cop_log(COP_FATAL, "could not access() \"%s\"\n", log_dir);
-      cop_log(COP_FATAL, "please set 'proxy.config.log.logfile_dir' \n");
-      exit(1);
-    }
+
+  logdir = config_read_log_dir();
+  if (access(logdir, W_OK) == -1) {
+    cop_log(COP_FATAL, "could not access() \"%s\"\n", (const char *)logdir);
+    cop_log(COP_FATAL, "please set 'proxy.config.log.logfile_dir' \n");
+    exit(1);
   }
+
   config_read_string("proxy.config.output.logfile", log_filename, sizeof(log_filename));
-  Layout::relative_to(log_file, sizeof(log_file), log_dir, log_filename);
+  Layout::relative_to(log_file, sizeof(log_file), logdir, log_filename);
+
   config_read_int("proxy.config.process_manager.mgmt_port", &http_backdoor_port, true);
   config_read_int("proxy.config.admin.autoconf_port", &autoconf_port, true);
   config_read_int("proxy.config.cluster.rsport", &rs_port, true);
@@ -664,6 +718,8 @@ spawn_manager()
   int err;
   int key;
 
+  ats_scoped_str bindir(config_read_bin_dir());
+
   cop_log_trace("Entering spawn_manager()\n");
   // Clean up shared memory segments.
   if (sem_id > 0) {
@@ -688,7 +744,7 @@ spawn_manager()
     }
   }
 
-  Layout::relative_to(prog, sizeof(prog), bin_path, manager_binary);
+  Layout::relative_to(prog, sizeof(prog), bindir, manager_binary);
   if (access(prog, R_OK | X_OK) == -1) {
     cop_log(COP_FATAL, "unable to access() manager binary \"%s\" [%d '%s']\n", prog, errno, strerror(errno));
     exit(1);
@@ -1590,7 +1646,8 @@ check(void *arg)
 
     // We do this after the first round of checks, since the first "check" will spawn traffic_manager
     if (!mgmt_init) {
-      TSInit(Layout::get()->runtimedir, static_cast<TSInitOptionT>(TS_MGMT_OPT_NO_EVENTS));
+      ats_scoped_str runtimedir(config_read_runtime_dir());
+      TSInit(runtimedir, static_cast<TSInitOptionT>(TS_MGMT_OPT_NO_EVENTS));
       mgmt_init = true;
     }
   }
@@ -1710,13 +1767,13 @@ static void
 init_config_file()
 {
   struct stat info;
-  const char * config_dir;
+  ats_scoped_str config_dir;
 
   cop_log_trace("Entering init_config_file()\n");
 
-  config_dir = Layout::get()->sysconfdir;
+  config_dir = config_read_sysconfig_dir();
   if (stat(config_dir, &info) < 0) {
-    cop_log(COP_FATAL, "unable to locate config directory '%s'\n",config_dir);
+    cop_log(COP_FATAL, "unable to locate config directory '%s'\n", (const char *)config_dir);
     cop_log(COP_FATAL, " please try setting correct root path in env variable TS_ROOT \n");
     exit(1);
   }
@@ -1725,7 +1782,8 @@ init_config_file()
   if (stat(config_file, &info) < 0) {
     Layout::relative_to(config_file, sizeof(config_file), config_dir, "records.config");
     if (stat(config_file, &info) < 0) {
-      cop_log(COP_FATAL, "unable to locate \"%s/records.config\" or \"%s/records.config.shadow\"\n", config_dir, config_dir);
+      cop_log(COP_FATAL, "unable to locate \"%s/records.config\" or \"%s/records.config.shadow\"\n",
+          (const char *)config_dir, (const char *)config_dir);
       exit(1);
     }
   }
@@ -1739,7 +1797,9 @@ init()
 
   cop_log_trace("Entering init()\n");
 
+  // Start up the records store and load the defaults so that we can locate our configuration.
   RecConfigFileInit();
+  RecordsConfigIterate(config_register_default, NULL);
 
   init_signals();
   init_syslog();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/cmd/traffic_crashlog/traffic_crashlog.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_crashlog/traffic_crashlog.cc b/cmd/traffic_crashlog/traffic_crashlog.cc
index 0005c81..5b43886 100644
--- a/cmd/traffic_crashlog/traffic_crashlog.cc
+++ b/cmd/traffic_crashlog/traffic_crashlog.cc
@@ -113,7 +113,6 @@ main(int /* argc ATS_UNUSED */, char **argv)
   Layout::create();
   RecProcessInit(RECM_STAND_ALONE, NULL /* diags */);
   LibRecordsConfigInit();
-  RecordsConfigOverrideFromEnvironment();
 
   if (syslog_mode) {
     RecString name;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/cmd/traffic_layout/traffic_layout.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_layout/traffic_layout.cc b/cmd/traffic_layout/traffic_layout.cc
index d8b7875..b410f34 100644
--- a/cmd/traffic_layout/traffic_layout.cc
+++ b/cmd/traffic_layout/traffic_layout.cc
@@ -40,17 +40,10 @@ printvar(const char * name, char * val)
   ats_free(val);
 }
 
-static void
-printl(const char * name, char * val)
-{
-  printf("%s: %s\n", name, val);
-}
-
 int
 main(int /* argc ATS_UNUSED */, char **argv)
 {
   AppVersionInfo appVersionInfo;
-  char path[PATH_NAME_MAX + 1];
 
   appVersionInfo.setup(PACKAGE_NAME, "traffic_layout", PACKAGE_VERSION,
           __DATE__, __TIME__, BUILD_MACHINE, BUILD_PERSON, "");
@@ -61,11 +54,10 @@ main(int /* argc ATS_UNUSED */, char **argv)
   Layout::create();
   RecProcessInit(RECM_STAND_ALONE, NULL /* diags */);
   LibRecordsConfigInit();
-  RecordsConfigOverrideFromEnvironment();
 
-  printl("PREFIX", Layout::get()->prefix);
+  printf("%s: %s\n", "PREFIX", Layout::get()->prefix);
   printvar("BINDIR", RecConfigReadBinDir());
-  printl("SYSCONFDIR", Layout::get()->sysconfdir);
+  printvar("SYSCONFDIR", RecConfigReadConfigDir());
   printvar("LIBDIR", Layout::get()->libdir);
   printvar("LOGDIR", RecConfigReadLogDir());
   printvar("RUNTIMEDIR", RecConfigReadRuntimeDir());
@@ -73,14 +65,9 @@ main(int /* argc ATS_UNUSED */, char **argv)
   printvar("INCLUDEDIR", Layout::get()->includedir);
   printvar("SNAPSHOTDIR", RecConfigReadSnapshotDir());
 
-  Layout::get()->relative_to(path, sizeof(path), Layout::get()->sysconfdir, "records.config");
-  printl("records.config", path);
-
+  printvar("records.config", RecConfigReadConfigPath(NULL, REC_CONFIG_FILE));
   printvar("remap.config", RecConfigReadConfigPath("proxy.config.url_remap.filename"));
-
-  Layout::get()->relative_to(path, sizeof(path), Layout::get()->sysconfdir, "plugin.config");
-  printl("plugin.config", path);
-
+  printvar("plugin.config", RecConfigReadConfigPath(NULL, "plugin.config"));
   printvar("ssl_multicert.config", RecConfigReadConfigPath("proxy.config.ssl.server.multicert.filename"));
   printvar("storage.config", RecConfigReadConfigPath("proxy.config.cache.storage_filename"));
   printvar("hosting.config", RecConfigReadConfigPath("proxy.config.cache.hosting_filename"));

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/cmd/traffic_manager/traffic_manager.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_manager/traffic_manager.cc b/cmd/traffic_manager/traffic_manager.cc
index 11227fb..9489544 100644
--- a/cmd/traffic_manager/traffic_manager.cc
+++ b/cmd/traffic_manager/traffic_manager.cc
@@ -67,8 +67,6 @@ static void printUsage(void);
 extern "C" int getpwnam_r(const char *name, struct passwd *result, char *buffer, size_t buflen, struct passwd **resptr);
 #endif
 
-static void extractConfigInfo(char *mgmt_path, const char *recs_conf, char *userName, int *fds_throttle);
-
 static StatProcessor *statProcessor;   // Statistics Processors
 static AppVersionInfo appVersionInfo;  // Build info for this application
 
@@ -248,15 +246,16 @@ static void
 init_dirs()
 {
   ats_scoped_str rundir(RecConfigReadRuntimeDir());
+  ats_scoped_str sysconfdir(RecConfigReadConfigDir());
 
-  if (access(Layout::get()->sysconfdir, R_OK) == -1) {
-    mgmt_elog(0, "unable to access() config dir '%s': %d, %s\n", Layout::get()->sysconfdir, errno, strerror(errno));
+  if (access(sysconfdir, R_OK) == -1) {
+    mgmt_elog(0, "unable to access() config directory '%s': %d, %s\n", (const char *)sysconfdir, errno, strerror(errno));
     mgmt_elog(0, "please set the 'TS_ROOT' environment variable\n");
     _exit(1);
   }
 
   if (access(rundir, R_OK) == -1) {
-    mgmt_elog(0, "unable to access() local state dir '%s': %d, %s\n", (const char *)rundir, errno, strerror(errno));
+    mgmt_elog(0, "unable to access() local state directory '%s': %d, %s\n", (const char *)rundir, errno, strerror(errno));
     mgmt_elog(0, "please set 'proxy.config.local_state_dir'\n");
     _exit(1);
   }
@@ -373,7 +372,7 @@ main(int argc, char **argv)
   char *envVar = NULL, *group_addr = NULL, *tsArgs = NULL;
   bool log_to_syslog = true;
   char userToRunAs[80];
-  int  fds_throttle = -1;
+  RecInt fds_throttle = -1;
   time_t ticker;
   ink_thread webThrId;
 
@@ -518,12 +517,14 @@ main(int argc, char **argv)
 
   RecLocalInit();
   LibRecordsConfigInit();
-  RecordsConfigOverrideFromEnvironment();
 
   init_dirs();// setup critical directories, needs LibRecords
 
-  // Get the config info we need while we are still root
-  extractConfigInfo(mgmt_path, recs_conf, userToRunAs, &fds_throttle);
+  if (RecGetRecordString("proxy.config.admin.user_id", userToRunAs, sizeof(userToRunAs)) != TS_ERR_OKAY || strlen(userToRunAs) == 0) {
+    mgmt_fatal(stderr, 0, "proxy.config.admin.user_id is not set\n");
+  }
+
+  RecGetRecordInt("proxy.config.net.connections_throttle", &fds_throttle);
 
   set_process_limits(fds_throttle); // as root
   runAsUser(userToRunAs);
@@ -1069,58 +1070,3 @@ runAsUser(const char * userName)
 
   }
 }                               /* End runAsUser() */
-
-//  void extractConfigInfo(...)
-//
-//  We need to get certain records.config values while we are
-//   root.  We can not use LMRecords to get them because the constructor
-//   for LMRecords creates the mgmt DBM and we do not want that to
-//   be owned as root.  This function extracts that info from
-//   records.config
-//
-//
-void
-extractConfigInfo(char *mgmt_path, const char *recs_conf, char *userName, int *fds_throttle)
-{
-  char file[1024];
-  bool useridFound = false;
-  bool throttleFound = false;
-
-  /* Figure out what user we should run as */
-  if (mgmt_path && recs_conf) {
-    FILE *fin;
-    snprintf(file, sizeof(file), "%s/%s.shadow", mgmt_path, recs_conf);
-    if (!(fin = fopen(file, "r"))) {
-      ink_filepath_make(file, sizeof(file), mgmt_path, recs_conf);
-      if (!(fin = fopen(file, "r"))) {
-        mgmt_elog(stderr, errno, "[extractConfigInfo] Unable to open config file(%s)\n", file);
-        _exit(1);
-      }
-    }
-    // Get 'user id' and 'network connections throttle limit'
-    while (((!useridFound) || (!throttleFound)) && fgets(file, 1024, fin)) {
-      if (strstr(file, "CONFIG proxy.config.admin.user_id STRING")) {
-        //coverity[secure_coding]
-        if ((sscanf(file, "CONFIG proxy.config.admin.user_id STRING %1023s\n", userName) == 1) &&
-            strcmp(userName, "NULL") != 0) {
-          useridFound = true;
-        }
-      } else if (strstr(file, "CONFIG proxy.config.net.connections_throttle INT")) {
-        if ((sscanf(file, "CONFIG proxy.config.net.connections_throttle INT %d\n", fds_throttle) == 1)) {
-          throttleFound = true;
-        }
-      }
-
-    }
-    fclose(fin);
-  } else {
-    mgmt_elog(stderr, 0, "[extractConfigInfo] Fatal Error: unable to access records file\n");
-    _exit(1);
-  }
-
-  if (useridFound == false) {
-    mgmt_elog(stderr, 0, "[extractConfigInfo] Fatal Error: proxy.config.admin.user_id is not set\n");
-    _exit(1);
-  }
-
-}                               /* End extractConfigInfo() */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/iocore/cluster/ClusterMachine.cc
----------------------------------------------------------------------
diff --git a/iocore/cluster/ClusterMachine.cc b/iocore/cluster/ClusterMachine.cc
index 7889c41..0756579 100644
--- a/iocore/cluster/ClusterMachine.cc
+++ b/iocore/cluster/ClusterMachine.cc
@@ -208,13 +208,13 @@ free_MachineList(MachineList * l)
 }
 
 MachineList *
-read_MachineList(char *filename, int afd)
+read_MachineList(const char *filename, int afd)
 {
   char line[256];
   int n = -1, i = 0, ln = 0;
   MachineList *l = NULL;
   ink_assert(filename || (afd != -1));
-  ats_scoped_str path(Layout::get()->relative_to(Layout::get()->sysconfdir, filename));
+  ats_scoped_str path(RecConfigReadConfigPath(NULL, filename));
 
   int fd = ((afd != -1) ? afd : open(path, O_RDONLY));
   if (fd >= 0) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/iocore/cluster/P_ClusterMachine.h
----------------------------------------------------------------------
diff --git a/iocore/cluster/P_ClusterMachine.h b/iocore/cluster/P_ClusterMachine.h
index 848a2b2..4415d5d 100644
--- a/iocore/cluster/P_ClusterMachine.h
+++ b/iocore/cluster/P_ClusterMachine.h
@@ -105,7 +105,7 @@ struct MachineList
   }
 };
 
-MachineList *read_MachineList(char *filename, int test_fd = -1);
+MachineList *read_MachineList(const char *filename, int test_fd = -1);
 void free_MachineList(MachineList * l);
 
 struct clusterConfigFile

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/iocore/net/SSLConfig.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLConfig.cc b/iocore/net/SSLConfig.cc
index 6a72914..cbfee9c 100644
--- a/iocore/net/SSLConfig.cc
+++ b/iocore/net/SSLConfig.cc
@@ -120,9 +120,9 @@ set_paths_helper(const char *path, const char *filename, char **final_path, char
 {
   if (final_path) {
     if (path && path[0] != '/') {
-      *final_path = Layout::get()->relative_to(Layout::get()->prefix, path);
+      *final_path = RecConfigReadPrefixPath(NULL, path);
     } else if (!path || path[0] == '\0'){
-      *final_path = ats_strdup(Layout::get()->sysconfdir);
+      *final_path = RecConfigReadConfigDir();
     } else {
       *final_path = ats_strdup(path);
     }
@@ -145,7 +145,6 @@ SSLConfigParams::initialize()
   char *ssl_client_private_key_filename = NULL;
   char *ssl_client_private_key_path = NULL;
   char *clientCACertRelativePath = NULL;
-  char *multicert_config_file = NULL;
   char *ssl_server_ca_cert_filename = NULL;
   char *ssl_client_ca_cert_filename = NULL;
 
@@ -239,9 +238,7 @@ SSLConfigParams::initialize()
   set_paths_helper(serverCertRelativePath, NULL, &serverCertPathOnly, NULL);
   ats_free(serverCertRelativePath);
 
-  REC_ReadConfigStringAlloc(multicert_config_file, "proxy.config.ssl.server.multicert.filename");
-  set_paths_helper(Layout::get()->sysconfdir, multicert_config_file, NULL, &configFilePath);
-  ats_free(multicert_config_file);
+  configFilePath = RecConfigReadConfigPath("proxy.config.ssl.server.multicert.filename");
 
   REC_ReadConfigStringAlloc(ssl_server_private_key_path, "proxy.config.ssl.server.private_key.path");
   set_paths_helper(ssl_server_private_key_path, NULL, &serverKeyPathOnly, NULL);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/iocore/net/Socks.cc
----------------------------------------------------------------------
diff --git a/iocore/net/Socks.cc b/iocore/net/Socks.cc
index 073cf51..cfdd214 100644
--- a/iocore/net/Socks.cc
+++ b/iocore/net/Socks.cc
@@ -463,8 +463,7 @@ void
 loadSocksConfiguration(socks_conf_struct * socks_conf_stuff)
 {
   int socks_config_fd = -1;
-  char config_pathname[PATH_NAME_MAX + 1];
-  char *socks_config_file = NULL;
+  ats_scoped_str config_pathname;
 #ifdef SOCKS_WITH_TS
   char *tmp;
 #endif
@@ -504,22 +503,18 @@ loadSocksConfiguration(socks_conf_struct * socks_conf_stuff)
   SocksServerConfig::startup();
 #endif
 
-  socks_config_file = REC_ConfigReadString("proxy.config.socks.socks_config_file");
+  config_pathname = RecConfigReadConfigPath("proxy.config.socks.socks_config_file");
+  Debug("Socks", "Socks Config File: %s", (const char *)config_pathname);
 
-  if (!socks_config_file) {
+  if (!config_pathname) {
     Error("SOCKS Config: could not read config file name. SOCKS Turned off");
     goto error;
   }
 
-  Layout::relative_to(config_pathname, sizeof(config_pathname),
-                      Layout::get()->sysconfdir, socks_config_file);
-  ats_free(socks_config_file);
-  Debug("Socks", "Socks Config File: %s", config_pathname);
-
   socks_config_fd =::open(config_pathname, O_RDONLY);
 
   if (socks_config_fd < 0) {
-    Error("SOCKS Config: could not open config file '%s'. SOCKS Turned off", config_pathname);
+    Error("SOCKS Config: could not open config file '%s'. SOCKS Turned off", (const char *)config_pathname);
     goto error;
   }
 #ifdef SOCKS_WITH_TS

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/lib/records/I_RecCore.h
----------------------------------------------------------------------
diff --git a/lib/records/I_RecCore.h b/lib/records/I_RecCore.h
index 06cc46d..a8131cd 100644
--- a/lib/records/I_RecCore.h
+++ b/lib/records/I_RecCore.h
@@ -45,6 +45,10 @@ typedef void (*RecConfigEntryCallback)(RecT rec_type, RecDataT data_type, const
 void RecConfigFileInit(void);
 int RecConfigFileParse(const char * path, RecConfigEntryCallback handler, bool inc_version);
 
+// Return a copy of the system's configuration directory, taking proxy.config.config_dir into account. The
+// caller MUST release the result with ats_free().
+char * RecConfigReadConfigDir();
+
 // Return a copy of the system's local state directory, taking proxy.config.local_state_dir into account. The
 // caller MUST release the result with ats_free().
 char * RecConfigReadRuntimeDir();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/lib/records/RecCore.cc
----------------------------------------------------------------------
diff --git a/lib/records/RecCore.cc b/lib/records/RecCore.cc
index 329538e..ad1856b 100644
--- a/lib/records/RecCore.cc
+++ b/lib/records/RecCore.cc
@@ -188,10 +188,10 @@ RecCoreInit(RecModeT mode_type, Diags *_diags)
     // ./etc/trafficserver/records.config
     // ./records.config
     bool file_exists = true;
-    g_rec_config_fpath = Layout::relative_to(Layout::get()->sysconfdir, REC_CONFIG_FILE REC_SHADOW_EXT);
+    g_rec_config_fpath = RecConfigReadConfigPath(NULL, REC_CONFIG_FILE REC_SHADOW_EXT);
     if (RecFileExists(g_rec_config_fpath) == REC_ERR_FAIL) {
       ats_free((char *)g_rec_config_fpath);
-      g_rec_config_fpath = Layout::relative_to(Layout::get()->sysconfdir, REC_CONFIG_FILE);
+      g_rec_config_fpath = RecConfigReadConfigPath(NULL, REC_CONFIG_FILE);
       if (RecFileExists(g_rec_config_fpath) == REC_ERR_FAIL) {
         RecLog(DL_Warning, "Could not find '%s', system will run with defaults\n", REC_CONFIG_FILE);
         file_exists = false;
@@ -1092,6 +1092,23 @@ REC_readString(const char *name, bool * found, bool lock)
 }
 
 //-------------------------------------------------------------------------
+// RecConfigReadConfigDir
+//-------------------------------------------------------------------------
+char *
+RecConfigReadConfigDir()
+{
+  char buf[PATH_NAME_MAX + 1];
+
+  buf[0] = '\0';
+  RecGetRecordString("proxy.config.config_dir", buf, PATH_NAME_MAX);
+  if (strlen(buf) > 0) {
+    return Layout::get()->relative(buf);
+  } else {
+    return ats_strdup(Layout::get()->sysconfdir);
+  }
+}
+
+//-------------------------------------------------------------------------
 // RecConfigReadRuntimeDir
 //-------------------------------------------------------------------------
 char *
@@ -1148,15 +1165,7 @@ RecConfigReadBinDir()
 char *
 RecConfigReadSnapshotDir()
 {
-  char buf[PATH_NAME_MAX + 1];
-
-  buf[0] = '\0';
-  RecGetRecordString("proxy.config.snapshot_dir", buf, PATH_NAME_MAX);
-  if (strlen(buf) > 0) {
-    return Layout::get()->relative_to(Layout::get()->sysconfdir, buf);
-  } else {
-    return Layout::get()->relative_to(Layout::get()->sysconfdir, "snapshots");
-  }
+  return RecConfigReadConfigPath("proxy.config.snapshot_dir", "snapshots");
 }
 
 //-------------------------------------------------------------------------
@@ -1165,16 +1174,22 @@ RecConfigReadSnapshotDir()
 char *
 RecConfigReadConfigPath(const char * file_variable, const char * default_value)
 {
-  char buf[PATH_NAME_MAX + 1];
+  ats_scoped_str sysconfdir(RecConfigReadConfigDir());
 
-  buf[0] = '\0';
-  RecGetRecordString(file_variable, buf, PATH_NAME_MAX);
-  if (strlen(buf) > 0) {
-    return Layout::get()->relative_to(Layout::get()->sysconfdir, buf);
+  // If the file name is in a configuration variable, look it up first ...
+  if (file_variable) {
+    char buf[PATH_NAME_MAX + 1];
+
+    buf[0] = '\0';
+    RecGetRecordString(file_variable, buf, PATH_NAME_MAX);
+    if (strlen(buf) > 0) {
+      return Layout::get()->relative_to(sysconfdir, buf);
+    }
   }
 
+  // Otherwise take the default ...
   if (default_value) {
-    return Layout::get()->relative_to(Layout::get()->sysconfdir, default_value);
+    return Layout::get()->relative_to(sysconfdir, default_value);
   }
 
   return NULL;
@@ -1188,12 +1203,16 @@ RecConfigReadPrefixPath(const char * file_variable, const char * default_value)
 {
   char buf[PATH_NAME_MAX + 1];
 
-  buf[0] = '\0';
-  RecGetRecordString(file_variable, buf, PATH_NAME_MAX);
-  if (strlen(buf) > 0) {
-    return Layout::get()->relative_to(Layout::get()->prefix, buf);
+  // If the file name is in a configuration variable, look it up first ...
+  if (file_variable) {
+    buf[0] = '\0';
+    RecGetRecordString(file_variable, buf, PATH_NAME_MAX);
+    if (strlen(buf) > 0) {
+      return Layout::get()->relative_to(Layout::get()->prefix, buf);
+    }
   }
 
+  // Otherwise take the default ...
   if (default_value) {
     return Layout::get()->relative_to(Layout::get()->prefix, default_value);
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/lib/records/RecUtils.cc
----------------------------------------------------------------------
diff --git a/lib/records/RecUtils.cc b/lib/records/RecUtils.cc
index 97feec0..4dddc98 100644
--- a/lib/records/RecUtils.cc
+++ b/lib/records/RecUtils.cc
@@ -422,7 +422,7 @@ RecDataSetFromString(RecDataT data_type, RecData * data_dst, const char *data_st
     data_src.rec_float = atof(data_string);
     break;
   case RECD_STRING:
-    if (strcmp((data_string), "NULL") == 0) {
+    if (data_string && strcmp((data_string), "NULL") == 0) {
       data_src.rec_string = NULL;
     } else {
       // It's OK to cast away the const here, because RecDataSet will copy the string.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/LocalManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/LocalManager.cc b/mgmt/LocalManager.cc
index 4806673..92b7890 100644
--- a/mgmt/LocalManager.cc
+++ b/mgmt/LocalManager.cc
@@ -204,6 +204,7 @@ LocalManager::LocalManager(bool proxy_on)
   bool found;
   ats_scoped_str rundir(RecConfigReadRuntimeDir());
   ats_scoped_str bindir(RecConfigReadBinDir());
+  ats_scoped_str sysconfdir(RecConfigReadConfigDir());
 
   syslog_facility = 0;
 
@@ -233,9 +234,8 @@ LocalManager::LocalManager(bool proxy_on)
   // Get the default IP binding values.
   RecHttpLoadIp("proxy.local.incoming_ip_to_bind", m_inbound_ip4, m_inbound_ip6);
 
-  if (access(Layout::get()->sysconfdir, R_OK) == -1) {
-    mgmt_elog(0, "[LocalManager::LocalManager] unable to access() directory '%s': %d, %s\n",
-	Layout::get()->sysconfdir, errno, strerror(errno));
+  if (access(sysconfdir, R_OK) == -1) {
+    mgmt_elog(0, "[LocalManager::LocalManager] unable to access() directory '%s': %d, %s\n", (const char *)sysconfdir, errno, strerror(errno));
     mgmt_fatal(0, "[LocalManager::LocalManager] please set the 'TS_ROOT' environment variable\n");
   }
 
@@ -247,15 +247,13 @@ LocalManager::LocalManager(bool proxy_on)
     mgmt_log("[LocalManager::LocalManager] WCCP identifying address set to %s.\n", static_cast<char*>(wccp_addr_str));
   }
 
-  ats_scoped_str wccp_config_str(REC_readString("proxy.config.wccp.services", &found));
-  if (found && wccp_config_str && *wccp_config_str) {
+  ats_scoped_str wccp_config_str(RecConfigReadConfigPath("proxy.config.wccp.services"));
+  if (wccp_config_str && strlen(wccp_config_str) > 0) {
     bool located = true;
-    if (access(wccp_config_str, R_OK) == -1) {
-      wccp_config_str = Layout::relative_to(Layout::get()->sysconfdir, wccp_config_str);
-      if (access(wccp_config_str, R_OK) == -1 ) {
-        located = false;
-      }
+    if (access(wccp_config_str, R_OK) == -1 ) {
+      located = false;
     }
+
     if (located) {
       wccp_cache.loadServicesFromFile(wccp_config_str);
     } else { // not located

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/Makefile.am
----------------------------------------------------------------------
diff --git a/mgmt/Makefile.am b/mgmt/Makefile.am
index 0c333dc..afaad15 100644
--- a/mgmt/Makefile.am
+++ b/mgmt/Makefile.am
@@ -19,7 +19,11 @@
 
 SUBDIRS = cluster utils web2 api
 
-noinst_LTLIBRARIES = libmgmt_p.la libmgmt_lm.la
+# Decoder ring:
+# 	libmgmt_c.la 	Configuration records (defaults).
+# 	libmgmt_lm.la	libmgmt for Local Manager applications (traffic_manager)
+# 	libmgmt_p.la	libmgmt for Process Manager applications (everything else)
+noinst_LTLIBRARIES = libmgmt_c.la libmgmt_p.la libmgmt_lm.la
 
 AM_CPPFLAGS = \
   $(iocore_include_dirs) \
@@ -35,12 +39,15 @@ AM_CPPFLAGS = \
   -I$(top_srcdir)/lib \
   -I$(top_builddir)/lib
 
+libmgmt_c_la_SOURCES = \
+  RecordsConfig.cc \
+  RecordsConfig.h
+
 libmgmt_COMMON = \
   BaseManager.cc \
   BaseManager.h \
   MgmtDefs.h \
-  RecordsConfig.cc \
-  RecordsConfig.h
+  RecordsConfigUtils.cc
 
 libmgmt_p_la_SOURCES = \
   $(libmgmt_COMMON) \
@@ -65,8 +72,10 @@ libmgmt_lm_la_SOURCES = \
   WebMgmtUtils.h
 
 libmgmt_lm_la_LIBADD = \
+  libmgmt_c.la \
   cluster/libcluster.la \
   utils/libutils_lm.la
 
 libmgmt_p_la_LIBADD = \
+  libmgmt_c.la \
   utils/libutils_p.la

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/ProcessManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/ProcessManager.cc b/mgmt/ProcessManager.cc
index 4030e41..998ccef 100644
--- a/mgmt/ProcessManager.cc
+++ b/mgmt/ProcessManager.cc
@@ -68,9 +68,6 @@ startProcessManager(void *arg)
 ProcessManager::ProcessManager(bool rlm):
 BaseManager(), require_lm(rlm), mgmt_sync_key(0), local_manager_sockfd(0), cbtable(NULL)
 {
-  ats_scoped_str rundir(RecConfigReadRuntimeDir());
-
-  ink_strlcpy(pserver_path, rundir, sizeof(pserver_path));
   mgmt_signal_queue = create_queue();
 
   // Set temp. process/manager timeout. Will be reconfigure later.
@@ -167,7 +164,8 @@ ProcessManager::processSignalQueue()
 void
 ProcessManager::initLMConnection()
 {
-  char message[1024];
+  ats_scoped_str rundir(RecConfigReadRuntimeDir());
+  ats_scoped_str sockpath(Layout::relative_to(rundir, LM_CONNECTION_SERVER));
 
   MgmtMessageHdr mh_hdr;
   MgmtMessageHdr *mh_full;
@@ -181,8 +179,7 @@ ProcessManager::initLMConnection()
   memset((char *) &serv_addr, 0, sizeof(serv_addr));
   serv_addr.sun_family = AF_UNIX;
 
-  snprintf(message, sizeof(message), "%s/%s", pserver_path, LM_CONNECTION_SERVER);
-  ink_strlcpy(serv_addr.sun_path, message, sizeof(serv_addr.sun_path));
+  ink_strlcpy(serv_addr.sun_path, sockpath, sizeof(serv_addr.sun_path));
 #if defined(darwin) || defined(freebsd)
   servlen = sizeof(sockaddr_un);
 #else
@@ -197,7 +194,7 @@ ProcessManager::initLMConnection()
   }
 
   if ((connect(local_manager_sockfd, (struct sockaddr *) &serv_addr, servlen)) < 0) {
-    mgmt_fatal(stderr, errno, "[ProcessManager::initLMConnection] Connect failed\n");
+    mgmt_fatal(stderr, errno, "[ProcessManager::initLMConnection] failed to connect management socket '%s'\n", (const char *)sockpath);
   }
 
   data_len = sizeof(pid_t);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/ProcessManager.h
----------------------------------------------------------------------
diff --git a/mgmt/ProcessManager.h b/mgmt/ProcessManager.h
index 8afbbdd..d984204 100644
--- a/mgmt/ProcessManager.h
+++ b/mgmt/ProcessManager.h
@@ -88,7 +88,6 @@ public:
 
   bool require_lm;
   time_t timeout;
-  char pserver_path[1024];
   int mgmt_sync_key;
 
   LLQ *mgmt_signal_queue;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/RecordsConfig.cc
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
index ad6e5cb..364bbd2 100644
--- a/mgmt/RecordsConfig.cc
+++ b/mgmt/RecordsConfig.cc
@@ -24,7 +24,6 @@
 
 #include "ink_config.h"
 #include "RecordsConfig.h"
-#include "ParseRules.h"
 
 #if TS_USE_REMOTE_UNWINDING
 #define MGMT_CRASHLOG_HELPER "traffic_crashlog"
@@ -36,7 +35,8 @@
 // RecordsConfig
 //-------------------------------------------------------------------------
 
-RecordElement RecordsConfig[] = {
+static const RecordElement RecordsConfig[] =
+{
   //##############################################################################
   //#
   //# records.config items
@@ -50,7 +50,7 @@ RecordElement RecordsConfig[] = {
   ,
   {RECT_CONFIG, "proxy.config.proxy_name", RECD_STRING, "<proxy_name>", RECU_DYNAMIC, RR_REQUIRED, RECC_STR, ".+", RECA_NULL}
   ,
-  {RECT_CONFIG, "proxy.config.bin_path", RECD_STRING, "bin", RECU_NULL, RR_REQUIRED, RECC_NULL, NULL, RECA_NULL}
+  {RECT_CONFIG, "proxy.config.bin_path", RECD_STRING, "bin", RECU_NULL, RR_REQUIRED, RECC_NULL, NULL, RECA_READ_ONLY}
   ,
   {RECT_CONFIG, "proxy.config.proxy_binary", RECD_STRING, "traffic_server", RECU_NULL, RR_REQUIRED, RECC_NULL, NULL, RECA_NULL}
   ,
@@ -60,10 +60,10 @@ RecordElement RecordsConfig[] = {
   ,
   {RECT_CONFIG, "proxy.config.env_prep", RECD_STRING, NULL, RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}
   ,
-  {RECT_CONFIG, "proxy.config.config_dir", RECD_STRING, TS_BUILD_SYSCONFDIR, RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}
+  {RECT_CONFIG, "proxy.config.config_dir", RECD_STRING, TS_BUILD_SYSCONFDIR, RECU_RESTART_TC, RR_NULL, RECC_NULL, NULL, RECA_READ_ONLY}
   ,
   // Jira TS-21
-  {RECT_CONFIG, "proxy.config.local_state_dir", RECD_STRING, TS_BUILD_RUNTIMEDIR, RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_NULL}
+  {RECT_CONFIG, "proxy.config.local_state_dir", RECD_STRING, TS_BUILD_RUNTIMEDIR, RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_READ_ONLY}
   ,
   {RECT_CONFIG, "proxy.config.alarm_email", RECD_STRING, TS_PKGSYSUSER, RECU_DYNAMIC, RR_NULL, RECC_STR, ".*", RECA_NULL}
   ,
@@ -258,7 +258,7 @@ RecordElement RecordsConfig[] = {
   ,
   {RECT_CONFIG, "proxy.config.admin.number_config_bak", RECD_INT, "3", RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}
   ,
-  {RECT_CONFIG, "proxy.config.admin.user_id", RECD_STRING, TS_PKGSYSUSER, RECU_NULL, RR_REQUIRED, RECC_NULL, NULL, RECA_NULL}
+  {RECT_CONFIG, "proxy.config.admin.user_id", RECD_STRING, TS_PKGSYSUSER, RECU_NULL, RR_REQUIRED, RECC_NULL, NULL, RECA_READ_ONLY}
   ,
   {RECT_CONFIG, "proxy.config.admin.cli_path", RECD_STRING, "cli", RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}
   ,
@@ -1395,7 +1395,7 @@ RecordElement RecordsConfig[] = {
   //# Plug-in Configuration
   //##############################################################################
   //# Directory in which to find plugins
-  {RECT_CONFIG, "proxy.config.plugin.plugin_dir", RECD_STRING, TS_BUILD_LIBEXECDIR, RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}
+  {RECT_CONFIG, "proxy.config.plugin.plugin_dir", RECD_STRING, TS_BUILD_LIBEXECDIR, RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_READ_ONLY}
   ,
   {RECT_CONFIG, "proxy.config.plugin.load_elevated", RECD_INT, "0", RECU_RESTART_TS, RR_NULL, RECC_INT, "[0-1]", RECA_READ_ONLY}
   ,
@@ -2057,137 +2057,12 @@ RecordElement RecordsConfig[] = {
   //#
   //###########
   {RECT_CONFIG, "proxy.config.cache.http.compatibility.4-2-0-fixup", RECD_INT, "1", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
-  ,
 
-  //##############################################################################
-  //#
-  //# The End
-  //#
-  //##############################################################################
-  {RECT_CONFIG, NULL, RECD_NULL, NULL, RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}
 };
 
-//-------------------------------------------------------------------------
-// RecordsConfigOverrideFromEnvironment
-//-------------------------------------------------------------------------
-
-// We process environment variable overrides when we parse the records.config configuration file, but the
-// operator might choose to override a variable that is not present in records.config so we have to post-
-// process the full set of configuration valriables as well.
-void
-RecordsConfigOverrideFromEnvironment()
-{
-  ink_mutex_acquire(&g_rec_config_lock);
-
-  for (const RecordElement * record = RecordsConfig; record->value_type != RECD_NULL; ++record) {
-    const char * value;
-    RecData data = {0};
-
-    if (record->type != RECT_CONFIG && record->type != RECT_LOCAL) {
-      continue;
-    }
-
-    if ((value = RecConfigOverrideFromEnvironment(record->name, NULL))) {
-      if (RecDataSetFromString(record->value_type, &data, value)) {
-          RecSetRecord(record->type, record->name, record->value_type, &data, NULL, false);
-          RecDataClear(record->value_type, &data);
-      }
-    }
-  }
-
-  ink_mutex_release(&g_rec_config_lock);
-}
-
-
-//-------------------------------------------------------------------------
-// LibRecordsConfigInit
-//-------------------------------------------------------------------------
-void
-LibRecordsConfigInit()
+void RecordsConfigIterate(RecordElementCallback callback, void * data)
 {
-  int r = 0;
-  RecInt tempInt = 0;
-  RecFloat tempFloat = 0.0;
-  RecCounter tempCounter = 0;
-
-  RecUpdateT update;
-  RecCheckT check;
-  RecAccessT access;
-  RecT type;
-
-  for (r = 0; RecordsConfig[r].value_type != RECD_NULL; r++) {
-    // Less typing ...
-    type = RecordsConfig[r].type;
-    update = RecordsConfig[r].update;
-    check = RecordsConfig[r].check;
-    access = RecordsConfig[r].access;
-
-    if (type == RECT_CONFIG || type == RECT_LOCAL) {
-      switch (RecordsConfig[r].value_type) {
-      case RECD_INT:
-        tempInt = (RecInt) ink_atoi64(RecordsConfig[r].value);
-        RecRegisterConfigInt(type, RecordsConfig[r].name, tempInt, update, check, RecordsConfig[r].regex, access);
-        break;
-
-      case RECD_FLOAT:
-        tempFloat = (RecFloat) atof(RecordsConfig[r].value);
-        RecRegisterConfigFloat(type, RecordsConfig[r].name, tempFloat, update, check, RecordsConfig[r].regex, access);
-        break;
-
-      case RECD_STRING:
-        RecRegisterConfigString(type, RecordsConfig[r].name, RecordsConfig[r].value, update, check,
-                                RecordsConfig[r].regex, access);
-        break;
-
-      case RECD_COUNTER:
-        tempCounter = (RecCounter) ink_atoi64(RecordsConfig[r].value);
-        RecRegisterConfigCounter(type, RecordsConfig[r].name, tempCounter, update, check,
-                                 RecordsConfig[r].regex, access);
-        break;
-
-      default:
-        ink_assert(true);
-        break;
-
-      }                         // switch
-    } else { // Everything else, except PROCESS, are stats. TODO: Should modularize this too like PROCESS was done.
-      switch (RecordsConfig[r].value_type) {
-
-      case RECD_INT:
-        tempInt = (RecInt) ink_atoi64(RecordsConfig[r].value);
-        RecRegisterStatInt(type, RecordsConfig[r].name, tempInt, RECP_NON_PERSISTENT);
-        break;
-
-      case RECD_FLOAT:
-        tempFloat = (RecFloat) atof(RecordsConfig[r].value);
-        RecRegisterStatFloat(type, RecordsConfig[r].name, tempFloat, RECP_NON_PERSISTENT);
-        break;
-
-      case RECD_STRING:
-        RecRegisterStatString(type, RecordsConfig[r].name, (RecString)RecordsConfig[r].value, RECP_NON_PERSISTENT);
-        break;
-
-      case RECD_COUNTER:
-        tempCounter = (RecCounter) ink_atoi64(RecordsConfig[r].value);
-        RecRegisterStatCounter(type, RecordsConfig[r].name, tempCounter, RECP_NON_PERSISTENT);
-        break;
-
-      default:
-        ink_assert(true);
-        break;
-      }                         // switch
-    }
+  for (unsigned i = 0; i < countof(RecordsConfig); ++i) {
+    callback(&RecordsConfig[i], data);
   }
-
-  // test_librecords();
 }
-
-void
-test_librecords()
-{
-  RecRegisterStatInt(RECT_PROCESS, "proxy.process.librecords.testing.int", (RecInt) 100, RECP_NON_PERSISTENT);
-  RecRegisterStatFloat(RECT_NODE, "proxy.node.librecords.testing.float", (RecFloat) 100.1, RECP_NON_PERSISTENT);
-  RecRegisterStatString(RECT_CLUSTER, "proxy.cluster.librecords.testing.string", (RecString) "Hello World\n", RECP_NON_PERSISTENT);
-  RecRegisterStatCounter(RECT_LOCAL, "proxy.local.librecords.testing.counter", (RecCounter) 99, RECP_NON_PERSISTENT);
-}
-

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/RecordsConfig.h
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfig.h b/mgmt/RecordsConfig.h
index d10f56e..e97ed72 100644
--- a/mgmt/RecordsConfig.h
+++ b/mgmt/RecordsConfig.h
@@ -26,8 +26,7 @@
 #if !defined (_RECORDS_CONFIG_H_)
 #define _RECORDS_CONFIG_H_
 
-#include "MgmtDefs.h"
-#include "MgmtHashTable.h"
+//#include "MgmtDefs.h"
 #include "P_RecCore.h"
 
 enum RecordRequiredType {
@@ -49,6 +48,9 @@ struct RecordElement
   RecAccessT access;            // access level of the record
 };
 
+typedef void (*RecordElementCallback)(const RecordElement *, void *);
+void RecordsConfigIterate(RecordElementCallback, void *);
+
 void LibRecordsConfigInit();                  // initializes RecordsConfigIndex
 void RecordsConfigOverrideFromEnvironment();  // Override records from the environment
 void test_librecords();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/RecordsConfigUtils.cc
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfigUtils.cc b/mgmt/RecordsConfigUtils.cc
new file mode 100644
index 0000000..3b1e334
--- /dev/null
+++ b/mgmt/RecordsConfigUtils.cc
@@ -0,0 +1,160 @@
+/** @file
+
+  A brief file description
+
+  @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 "ink_config.h"
+#include "RecordsConfig.h"
+#include "ParseRules.h"
+
+//-------------------------------------------------------------------------
+// RecordsConfigOverrideFromEnvironment
+//-------------------------------------------------------------------------
+
+static void
+override_record(const RecordElement * record, void *)
+{
+  if (REC_TYPE_IS_CONFIG(record->type)) {
+    const char * value;
+    RecData data = {0};
+
+    if ((value = RecConfigOverrideFromEnvironment(record->name, NULL))) {
+
+      if (RecDataSetFromString(record->value_type, &data, value)) {
+        // WARNING: If we are not the record owner, RecSetRecord() doesn't set our copy
+        // of the record. It sends a set message to the local manager. This can cause
+        // "interesting" results if you are trying to override configuration values
+        // early in startup (before we have synced with the local manager).
+        RecSetRecord(record->type, record->name, record->value_type, &data, NULL, false);
+        RecDataClear(record->value_type, &data);
+      }
+    }
+  }
+
+}
+
+// We process environment variable overrides when we parse the records.config configuration file, but the
+// operator might choose to override a variable that is not present in records.config so we have to post-
+// process the full set of configuration valriables as well.
+void
+RecordsConfigOverrideFromEnvironment()
+{
+  RecordsConfigIterate(override_record, NULL);
+}
+
+//-------------------------------------------------------------------------
+// LibRecordsConfigInit
+//-------------------------------------------------------------------------
+
+static void
+initialize_record(const RecordElement * record, void *)
+{
+  RecInt tempInt = 0;
+  RecFloat tempFloat = 0.0;
+  RecCounter tempCounter = 0;
+
+  RecUpdateT update;
+  RecCheckT check;
+  RecAccessT access;
+  RecT type;
+
+  // Less typing ...
+  type = record->type;
+  update = record->update;
+  check = record->check;
+  access = record->access;
+
+  if (REC_TYPE_IS_CONFIG(type)) {
+    const char * value = RecConfigOverrideFromEnvironment(record->name, record->value);
+    RecData data = {0};
+
+    RecDataSetFromString(record->value_type, &data, value);
+
+    switch (record->value_type) {
+    case RECD_INT:
+      RecRegisterConfigInt(type, record->name, data.rec_int, update, check, record->regex, access);
+      break;
+
+    case RECD_FLOAT:
+      RecRegisterConfigFloat(type, record->name, data.rec_float, update, check, record->regex, access);
+      break;
+
+    case RECD_STRING:
+      RecRegisterConfigString(type, record->name, data.rec_string, update, check, record->regex, access);
+      break;
+
+    case RECD_COUNTER:
+      tempCounter = (RecCounter) ink_atoi64(record->value);
+      RecRegisterConfigCounter(type, record->name, data.rec_counter, update, check, record->regex, access);
+      break;
+
+    default:
+      ink_assert(true);
+      break;
+    }                         // switch
+
+    RecDataClear(record->value_type, &data);
+  } else { // Everything else, except PROCESS, are stats. TODO: Should modularize this too like PROCESS was done.
+    ink_assert(REC_TYPE_IS_STAT(type));
+
+    switch (record->value_type) {
+
+    case RECD_INT:
+      tempInt = (RecInt) ink_atoi64(record->value);
+      RecRegisterStatInt(type, record->name, tempInt, RECP_NON_PERSISTENT);
+      break;
+
+    case RECD_FLOAT:
+      tempFloat = (RecFloat) atof(record->value);
+      RecRegisterStatFloat(type, record->name, tempFloat, RECP_NON_PERSISTENT);
+      break;
+
+    case RECD_STRING:
+      RecRegisterStatString(type, record->name, (RecString)record->value, RECP_NON_PERSISTENT);
+      break;
+
+    case RECD_COUNTER:
+      tempCounter = (RecCounter) ink_atoi64(record->value);
+      RecRegisterStatCounter(type, record->name, tempCounter, RECP_NON_PERSISTENT);
+      break;
+
+    default:
+      ink_assert(true);
+      break;
+    }                         // switch
+  }
+
+}
+
+void
+LibRecordsConfigInit()
+{
+  RecordsConfigIterate(initialize_record, NULL);
+}
+
+void
+test_librecords()
+{
+  RecRegisterStatInt(RECT_PROCESS, "proxy.process.librecords.testing.int", (RecInt) 100, RECP_NON_PERSISTENT);
+  RecRegisterStatFloat(RECT_NODE, "proxy.node.librecords.testing.float", (RecFloat) 100.1, RECP_NON_PERSISTENT);
+  RecRegisterStatString(RECT_CLUSTER, "proxy.cluster.librecords.testing.string", (RecString) "Hello World\n", RECP_NON_PERSISTENT);
+  RecRegisterStatCounter(RECT_LOCAL, "proxy.local.librecords.testing.counter", (RecCounter) 99, RECP_NON_PERSISTENT);
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/Rollback.cc
----------------------------------------------------------------------
diff --git a/mgmt/Rollback.cc b/mgmt/Rollback.cc
index 56a4608..cef1fa0 100644
--- a/mgmt/Rollback.cc
+++ b/mgmt/Rollback.cc
@@ -72,11 +72,9 @@ Rollback::Rollback(const char *baseFileName, bool root_access_needed_)
   // TODO: Use the runtime directory for storing mutable data
   // XXX: Sysconfdir should be imutable!!!
 
-  if (access(Layout::get()->sysconfdir, F_OK) < 0) {
-    mgmt_elog(0, "[Rollback::Rollback] unable to access() directory '%s': %d, %s\n",
-              Layout::get()->sysconfdir, errno, strerror(errno));
-    mgmt_elog(0, "[Rollback::Rollback] please set the 'TS_ROOT' environment variable\n");
-    _exit(1);
+  ats_scoped_str sysconfdir(RecConfigReadConfigDir());
+  if (access(sysconfdir, F_OK) < 0) {
+    mgmt_fatal(0, "[Rollback::Rollback] unable to access() directory '%s': %d, %s\n", (const char *)sysconfdir, errno, strerror(errno));
   }
 
   if (varIntFromName("proxy.config.admin.number_config_bak", &numBak) == true) {
@@ -223,11 +221,11 @@ Rollback::~Rollback()
 char *
 Rollback::createPathStr(version_t version)
 {
-
-  int bufSize = strlen(Layout::get()->sysconfdir) + fileNameLen + MAX_VERSION_DIGITS + 1;
+  ats_scoped_str sysconfdir(RecConfigReadConfigDir());
+  int bufSize = strlen(sysconfdir) + fileNameLen + MAX_VERSION_DIGITS + 1;
   char * buffer = (char *)ats_malloc(bufSize);
 
-  Layout::get()->relative_to(buffer, bufSize, Layout::get()->sysconfdir, fileName);
+  Layout::get()->relative_to(buffer, bufSize, sysconfdir, fileName);
 
   if (version != ACTIVE_VERSION) {
     size_t pos = strlen(buffer);
@@ -654,16 +652,16 @@ Rollback::findVersions_ml(ExpandingArray * listNames)
 
   int count = 0;
   version_t highestSeen = 0, version = 0;
+  ats_scoped_str sysconfdir(RecConfigReadConfigDir());
 
   DIR *dir;
   struct dirent *dirEntrySpace;
   struct dirent *entryPtr;
 
-  dir = opendir(Layout::get()->sysconfdir);
+  dir = opendir(sysconfdir);
 
   if (dir == NULL) {
-    mgmt_log(stderr, "[Rollback::findVersions] Unable to open configuration directory: %s: %s\n",
-             Layout::get()->sysconfdir, strerror(errno));
+    mgmt_log(stderr, "[Rollback::findVersions] Unable to open configuration directory: %s: %s\n", (const char *)sysconfdir, strerror(errno));
     return INVALID_VERSION;
   }
   // The fun of Solaris - readdir_r requires a buffer passed into it

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/api/CoreAPI.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/CoreAPI.cc b/mgmt/api/CoreAPI.cc
index e0fa0c1..059bd1e 100644
--- a/mgmt/api/CoreAPI.cc
+++ b/mgmt/api/CoreAPI.cc
@@ -949,23 +949,14 @@ EventSignalCbUnregister(const char *event_name, TSEventSignalFunc func)
 TSMgmtError
 SnapshotTake(const char * snapshot_name)
 {
-  char *snapDirFromRecordsConf;
-  bool found;
-  char snapDir[PATH_NAME_MAX + 1];
+  ats_scoped_str snapdir;
 
   if (!snapshot_name)
     return TS_ERR_PARAMS;
 
-  int rec_err = RecGetRecordString_Xmalloc("proxy.config.snapshot_dir", &snapDirFromRecordsConf);
-  found = (rec_err == REC_ERR_OKAY);
-  ink_release_assert(found);
-  // XXX: Why was that offset to config dir?
-  //      Any path should be prefix relative thought
-  //
-  Layout::relative_to(snapDir, sizeof(snapDir), Layout::get()->sysconfdir, snapDirFromRecordsConf);
-  ats_free(snapDirFromRecordsConf);
+  snapdir = RecConfigReadSnapshotDir();
 
-  SnapResult result = configFiles->takeSnap(snapshot_name, snapDir);
+  SnapResult result = configFiles->takeSnap(snapshot_name, snapdir);
   if (result != SNAP_OK)
     return TS_ERR_FAIL;
   else
@@ -975,23 +966,14 @@ SnapshotTake(const char * snapshot_name)
 TSMgmtError
 SnapshotRestore(const char * snapshot_name)
 {
-  char *snapDirFromRecordsConf;
-  bool found;
-  char snapDir[PATH_NAME_MAX + 1];
+  ats_scoped_str snapdir;
 
   if (!snapshot_name)
     return TS_ERR_PARAMS;
 
-  int rec_err = RecGetRecordString_Xmalloc("proxy.config.snapshot_dir", &snapDirFromRecordsConf);
-  found = (rec_err == REC_ERR_OKAY);
-  ink_release_assert(found);
-  // XXX: Why was that offset to config dir?
-  //      Any path should be prefix relative thought
-  //
-  Layout::relative_to(snapDir, sizeof(snapDir), Layout::get()->sysconfdir, snapDirFromRecordsConf);
-  ats_free(snapDirFromRecordsConf);
+  snapdir = RecConfigReadSnapshotDir();
 
-  SnapResult result = configFiles->restoreSnap(snapshot_name, snapDir);
+  SnapResult result = configFiles->restoreSnap(snapshot_name, snapdir);
   if (result != SNAP_OK)
     return TS_ERR_FAIL;
   else
@@ -1001,23 +983,14 @@ SnapshotRestore(const char * snapshot_name)
 TSMgmtError
 SnapshotRemove(const char * snapshot_name)
 {
-  char *snapDirFromRecordsConf;
-  bool found;
-  char snapDir[PATH_NAME_MAX + 1];
+  ats_scoped_str snapdir;
 
   if (!snapshot_name)
     return TS_ERR_PARAMS;
 
-  int rec_err = RecGetRecordString_Xmalloc("proxy.config.snapshot_dir", &snapDirFromRecordsConf);
-  found = (rec_err == REC_ERR_OKAY);
-  ink_release_assert(found);
-  // XXX: Why was that offset to config dir?
-  //      Any path should be prefix relative thought
-  //
-  Layout::relative_to(snapDir, sizeof(snapDir), Layout::get()->sysconfdir, snapDirFromRecordsConf);
-  ats_free(snapDirFromRecordsConf);
+  snapdir = RecConfigReadSnapshotDir();
 
-  SnapResult result = configFiles->removeSnap(snapshot_name, snapDir);
+  SnapResult result = configFiles->removeSnap(snapshot_name, snapdir);
   if (result != SNAP_OK)
     return TS_ERR_FAIL;
   else

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/cluster/VMap.cc
----------------------------------------------------------------------
diff --git a/mgmt/cluster/VMap.cc b/mgmt/cluster/VMap.cc
index 993f44b..b7c8ee6 100644
--- a/mgmt/cluster/VMap.cc
+++ b/mgmt/cluster/VMap.cc
@@ -303,14 +303,14 @@ VMap::lt_runGambit()
  * vaddr file.
  */
 void
-VMap::lt_readAListFile(char * data)
+VMap::lt_readAListFile(const char * fname)
 {
   int tmp_num_addrs = 0;
   char buf[1024];
   char tmp_addr[1024], tmp_interface[1024];
   FILE *fin;
   char tmp_id[1024];
-  ats_scoped_str vaddr_path(Layout::get()->relative_to(Layout::get()->sysconfdir, data));
+  ats_scoped_str vaddr_path(RecConfigReadConfigPath(NULL, fname));
 
   if (!(fin = fopen(vaddr_path, "r"))) {
     mgmt_log(stderr, "[VMap::lt_readAListFile] Unable to open file: %s, addr list unchanged\n", (const char *)vaddr_path);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/cluster/VMap.h
----------------------------------------------------------------------
diff --git a/mgmt/cluster/VMap.h b/mgmt/cluster/VMap.h
index 099e2ca..90d4f36 100644
--- a/mgmt/cluster/VMap.h
+++ b/mgmt/cluster/VMap.h
@@ -70,7 +70,7 @@ public:
   void rl_downAddrs();
   void removeAddressMapping(int i);
   void lt_runGambit();
-  void lt_readAListFile(char *data);
+  void lt_readAListFile(const char *data);
   void lt_constructVMapMessage(char *ip, char *message, int max);
 
   bool rl_remote_map(char *virt_ip, char *real_ip);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/mgmt/web2/WebIntrMain.cc
----------------------------------------------------------------------
diff --git a/mgmt/web2/WebIntrMain.cc b/mgmt/web2/WebIntrMain.cc
index 81f8a39..e757372 100644
--- a/mgmt/web2/WebIntrMain.cc
+++ b/mgmt/web2/WebIntrMain.cc
@@ -393,7 +393,7 @@ webIntr_main(void *)
 
     if ((err = stat(autoconfContext.docRoot, &s)) < 0) {
       ats_free(autoconfContext.docRoot);
-      autoconfContext.docRoot = ats_strdup(Layout::get()->sysconfdir);
+      autoconfContext.docRoot = RecConfigReadConfigDir();
       if ((err = stat(autoconfContext.docRoot, &s)) < 0) {
         mgmt_elog(0, "[WebIntrMain] unable to stat() directory '%s': %d %d, %s\n",
                 autoconfContext.docRoot, err, errno, strerror(errno));

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/InkAPI.cc
----------------------------------------------------------------------
diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
index 6ff206e..3ce524d 100644
--- a/proxy/InkAPI.cc
+++ b/proxy/InkAPI.cc
@@ -1730,7 +1730,14 @@ TSInstallDirGet(void)
 const char *
 TSConfigDirGet(void)
 {
-  return Layout::get()->sysconfdir;
+  static char * sysconfdir = NULL;
+
+  // This is not great, bit it's no worse than TSPluginDirGet :-/
+  if (sysconfdir == NULL) {
+    sysconfdir = RecConfigReadConfigDir();
+  }
+
+  return sysconfdir;
 }
 
 const char *

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/Main.cc
----------------------------------------------------------------------
diff --git a/proxy/Main.cc b/proxy/Main.cc
index 650ab0d..8af548e 100644
--- a/proxy/Main.cc
+++ b/proxy/Main.cc
@@ -403,10 +403,10 @@ static void
 check_config_directories(void)
 {
   ats_scoped_str rundir(RecConfigReadRuntimeDir());
+  ats_scoped_str sysconfdir(RecConfigReadConfigDir());
 
-  if (access(Layout::get()->sysconfdir, R_OK) == -1) {
-    fprintf(stderr,"unable to access() config dir '%s': %d, %s\n",
-            Layout::get()->sysconfdir, errno, strerror(errno));
+  if (access(sysconfdir, R_OK) == -1) {
+    fprintf(stderr,"unable to access() config dir '%s': %d, %s\n", (const char *)sysconfdir, errno, strerror(errno));
     fprintf(stderr, "please set the 'TS_ROOT' environment variable\n");
     _exit(1);
   }
@@ -439,11 +439,7 @@ initialize_process_manager()
   }
 
   RecProcessInit(remote_management_flag ? RECM_CLIENT : RECM_STAND_ALONE, diags);
-
-  if (!remote_management_flag) {
-    LibRecordsConfigInit();
-    RecordsConfigOverrideFromEnvironment();
-  }
+  LibRecordsConfigInit();
 
   // Start up manager
   pmgmt = new ProcessManager(remote_management_flag);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/Plugin.cc
----------------------------------------------------------------------
diff --git a/proxy/Plugin.cc b/proxy/Plugin.cc
index 5a2412a..4b0780f 100644
--- a/proxy/Plugin.cc
+++ b/proxy/Plugin.cc
@@ -186,7 +186,7 @@ not_found:
 void
 plugin_init(void)
 {
-  char path[PATH_NAME_MAX + 1];
+  ats_scoped_str path;
   char line[1024], *p;
   char *argv[64];
   char *vars[64];
@@ -197,14 +197,15 @@ plugin_init(void)
 
   if (INIT_ONCE) {
     api_init();
+    TSConfigDirGet();
     plugin_dir = TSPluginDirGet();
     INIT_ONCE = false;
   }
 
-  Layout::get()->relative_to(path, sizeof(path), Layout::get()->sysconfdir, "plugin.config");
+  path = RecConfigReadConfigPath(NULL, "plugin.config");
   fd = open(path, O_RDONLY);
   if (fd < 0) {
-    Warning("unable to open plugin config file '%s': %d, %s", path, errno, strerror(errno));
+    Warning("unable to open plugin config file '%s': %d, %s", (const char *)path, errno, strerror(errno));
     return;
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/Update.cc
----------------------------------------------------------------------
diff --git a/proxy/Update.cc b/proxy/Update.cc
index f72447c..822b9e0 100644
--- a/proxy/Update.cc
+++ b/proxy/Update.cc
@@ -878,7 +878,7 @@ UpdateConfigManager::BuildUpdateList()
   ats_scoped_str config_path;
 
   if (_filename) {
-    config_path = Layout::get()->relative_to(Layout::get()->sysconfdir, _filename);
+    config_path = RecConfigReadConfigPath(NULL, _filename);
   } else {
     return (UpdateConfigList *) NULL;
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/config/records.config.default.in
----------------------------------------------------------------------
diff --git a/proxy/config/records.config.default.in b/proxy/config/records.config.default.in
index 1595023..8a11063 100644
--- a/proxy/config/records.config.default.in
+++ b/proxy/config/records.config.default.in
@@ -187,14 +187,6 @@ CONFIG proxy.config.dump_mem_info_frequency INT 0
 CONFIG proxy.config.http.slow.log.threshold INT 0
 
 ##############################################################################
-# These are necessary (for now) to keep traffic_manager/_cop happy.
-##############################################################################
-CONFIG proxy.config.proxy_name STRING @build_machine@
-CONFIG proxy.config.admin.user_id STRING @pkgsysuser@
-CONFIG proxy.config.log.logfile_dir STRING @rel_logdir@
-CONFIG proxy.config.output.logfile STRING traffic.out
-
-##############################################################################
 # Cluster Subsystem. Docs:
 #    https://docs.trafficserver.apache.org/records.config#cluster
 ##############################################################################

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/http/remap/RemapConfig.cc
----------------------------------------------------------------------
diff --git a/proxy/http/remap/RemapConfig.cc b/proxy/http/remap/RemapConfig.cc
index 70987e1..fcbafaf 100644
--- a/proxy/http/remap/RemapConfig.cc
+++ b/proxy/http/remap/RemapConfig.cc
@@ -248,11 +248,11 @@ parse_include_directive(const char * directive, BUILD_TABLE_INFO * bti, char * e
     // to keep the ACL rules from the parent because ACLs must be global across the full set of config
     // files.
     BUILD_TABLE_INFO  nbti;
-    ats_scoped_str        path;
+    ats_scoped_str    path;
     bool              success;
 
     // The included path is relative to SYSCONFDIR, just like remap.config is.
-    path = Layout::relative_to(Layout::get()->sysconfdir, bti->paramv[i]);
+    path = RecConfigReadConfigPath(NULL, bti->paramv[i]);
 
     // XXX including directories is not supported (yet!).
     if (ink_file_is_directory(path)) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/http/remap/UrlRewrite.cc
----------------------------------------------------------------------
diff --git a/proxy/http/remap/UrlRewrite.cc b/proxy/http/remap/UrlRewrite.cc
index 02e9758..91d6022 100644
--- a/proxy/http/remap/UrlRewrite.cc
+++ b/proxy/http/remap/UrlRewrite.cc
@@ -58,18 +58,16 @@ UrlRewrite::UrlRewrite()
    http_default_redirect_url(NULL), num_rules_forward(0), num_rules_reverse(0), num_rules_redirect_permanent(0),
    num_rules_redirect_temporary(0), num_rules_forward_with_recv_port(0), _valid(false)
 {
+  ats_scoped_str config_file_path;
 
   forward_mappings.hash_lookup = reverse_mappings.hash_lookup =
     permanent_redirects.hash_lookup = temporary_redirects.hash_lookup =
     forward_mappings_with_recv_port.hash_lookup = NULL;
 
-  char * config_file = NULL;
-  char * config_file_path = NULL;
-
-  REC_ReadConfigStringAlloc(config_file, "proxy.config.url_remap.filename");
-  if (config_file == NULL) {
+  config_file_path = RecConfigReadConfigPath("proxy.config.url_remap.filename", "remap.config");
+  if (!config_file_path) {
     pmgmt->signalManager(MGMT_SIGNAL_CONFIG_ERROR, "Unable to find proxy.config.url_remap.filename");
-    Warning("%s Unable to locate remap.config.  No remappings in effect", modulePrefix);
+    Warning("%s Unable to locate remap.config. No remappings in effect", modulePrefix);
     return;
   }
 
@@ -96,8 +94,6 @@ UrlRewrite::UrlRewrite()
   REC_ReadConfigInteger(url_remap_mode, "proxy.config.url_remap.url_remap_mode");
   REC_ReadConfigInteger(backdoor_enabled, "proxy.config.url_remap.handle_backdoor_urls");
 
-  config_file_path = Layout::relative_to(Layout::get()->sysconfdir, config_file);
-
   if (0 == this->BuildTable(config_file_path)) {
     _valid = true;
     if (is_debug_tag_set("url_rewrite")) {
@@ -106,9 +102,6 @@ UrlRewrite::UrlRewrite()
   } else {
     Warning("something failed during BuildTable() -- check your remap plugins!");
   }
-
-  ats_free(config_file_path);
-  ats_free(config_file);
 }
 
 UrlRewrite::~UrlRewrite()

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/logging/LogConfig.cc
----------------------------------------------------------------------
diff --git a/proxy/logging/LogConfig.cc b/proxy/logging/LogConfig.cc
index 385a944..5c07a81 100644
--- a/proxy/logging/LogConfig.cc
+++ b/proxy/logging/LogConfig.cc
@@ -124,8 +124,6 @@ LogConfig::setup_default_values()
   roll_log_files_now = false;
 
   custom_logs_enabled = false;
-  xml_config_file = ats_strdup("logs_xml.config");
-  hosts_config_file = ats_strdup("log_hosts.config");
 
 /* The default values for the search log                         */
 
@@ -420,18 +418,6 @@ LogConfig::read_configuration_variables()
   val = (int) REC_ConfigReadInteger("proxy.config.log.custom_logs_enabled");
   custom_logs_enabled = (val > 0);
 
-  ptr = REC_ConfigReadString("proxy.config.log.xml_config_file");
-  if (ptr != NULL) {
-    ats_free(xml_config_file);
-    xml_config_file = ptr;
-  }
-
-  ptr = REC_ConfigReadString("proxy.config.log.hosts_config_file");
-  if (ptr != NULL) {
-    ats_free(hosts_config_file);
-    hosts_config_file = ptr;
-  }
-
   // PERFORMANCE
   val = (int) REC_ConfigReadInteger("proxy.config.log.sampling_frequency");
   if (val > 0) {
@@ -567,8 +553,6 @@ LogConfig::~LogConfig()
   ats_free(extended2_log_header);
   ats_free(collation_host);
   ats_free(collation_secret);
-  ats_free(xml_config_file);
-  ats_free(hosts_config_file);
   ats_free(search_log_file_one);
   ats_free(search_log_file_two);
   ats_free(m_dir_entry);
@@ -717,8 +701,6 @@ LogConfig::display(FILE * fd)
   fprintf(fd, "   hostname = %s\n", hostname);
   fprintf(fd, "   logfile_dir = %s\n", logfile_dir);
   fprintf(fd, "   logfile_perm = 0%o\n", logfile_perm);
-  fprintf(fd, "   xml_config_file = %s\n", xml_config_file);
-  fprintf(fd, "   hosts_config_file = %s\n", hosts_config_file);
   fprintf(fd, "   squid_log_enabled = %d\n", squid_log_enabled);
   fprintf(fd, "   squid_log_is_ascii = %d\n", squid_log_is_ascii);
   fprintf(fd, "   squid_log_name = %s\n", squid_log_name);
@@ -1619,12 +1601,7 @@ LogConfig::read_xml_log_config(int from_memory)
   ats_scoped_str config_path;
 
   if (!from_memory) {
-    if (xml_config_file == NULL) {
-      Note("No log config file to read");
-      return;
-    }
-
-    config_path = Layout::get()->relative_to(Layout::get()->sysconfdir, xml_config_file);
+    config_path = RecConfigReadConfigPath("proxy.config.log.xml_config_file", "logs_xml.config");
   }
 
   InkXmlConfigFile log_config(config_path ? (const char *)config_path : "memory://builtin");
@@ -2225,7 +2202,7 @@ LogConfig::read_xml_log_config(int from_memory)
 char **
 LogConfig::read_log_hosts_file(size_t * num_hosts)
 {
-  ats_scoped_str config_path(Layout::get()->relative_to(Layout::get()->sysconfdir, hosts_config_file));
+  ats_scoped_str config_path(RecConfigReadConfigPath("proxy.config.log.hosts_config_file", "log_hosts.config"));
   char line[LOG_MAX_FORMAT_LINE];
   char **hosts = NULL;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/logging/LogConfig.h
----------------------------------------------------------------------
diff --git a/proxy/logging/LogConfig.h b/proxy/logging/LogConfig.h
index f63ab78..51867b6 100644
--- a/proxy/logging/LogConfig.h
+++ b/proxy/logging/LogConfig.h
@@ -220,8 +220,6 @@ public:
   char *extended2_log_header;
   char *collation_host;
   char *collation_secret;
-  char *xml_config_file;
-  char *hosts_config_file;
 
 private:
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d513cf74/proxy/logging/LogStandalone.cc
----------------------------------------------------------------------
diff --git a/proxy/logging/LogStandalone.cc b/proxy/logging/LogStandalone.cc
index fd7ff24..9d11f07 100644
--- a/proxy/logging/LogStandalone.cc
+++ b/proxy/logging/LogStandalone.cc
@@ -109,13 +109,8 @@ initialize_process_manager()
   ink_assert(diags);
 
   RecProcessInit(remote_management_flag ? RECM_CLIENT : RECM_STAND_ALONE, diags);
+  LibRecordsConfigInit();
 
-  if (!remote_management_flag) {
-    LibRecordsConfigInit();
-    RecordsConfigOverrideFromEnvironment();
-  }
-
-  //
   // Start up manager
   pmgmt = new ProcessManager(remote_management_flag);
 


[2/2] trafficserver git commit: Fix typo in comment

Posted by jp...@apache.org.
Fix typo in comment


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

Branch: refs/heads/master
Commit: 435e756619951169792f25cc03f838ba4b9decc9
Parents: d513cf7
Author: James Peach <jp...@apache.org>
Authored: Thu Nov 13 16:14:21 2014 -0800
Committer: James Peach <jp...@apache.org>
Committed: Thu Nov 13 16:33:21 2014 -0800

----------------------------------------------------------------------
 proxy/api/ts/ts.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/435e7566/proxy/api/ts/ts.h
----------------------------------------------------------------------
diff --git a/proxy/api/ts/ts.h b/proxy/api/ts/ts.h
index 07a8760..ef6fe84 100644
--- a/proxy/api/ts/ts.h
+++ b/proxy/api/ts/ts.h
@@ -2252,7 +2252,7 @@ extern "C"
      make sure it is heap allocated, and that you do not free it.
 
      Calling this API implicitly also enables the "Follow Redirect" feature, so
-     there is no rason to call TSHttpTxnFollowRedirect() as well.
+     there is no reason to call TSHttpTxnFollowRedirect() as well.
 
      @param txnp the transaction pointer
      @param url  a heap allocated string with the URL