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 2015/12/15 07:00:43 UTC

trafficserver git commit: TS-3960: reload configuration when SSL certificates change

Repository: trafficserver
Updated Branches:
  refs/heads/master f17e7c6dd -> 4d2c262c3


TS-3960: reload configuration when SSL certificates change

traffic_ctl doesn't notice when SSL certs change file contents
without touching ssl_multicert.config. Add a callback so that SSL
loader can register files to be monitored by traffic_manager. This
then takes the normal configuration monitoring path except that
monitored children of a patch trigger a reload in the parent
configuration file.

This closes #301.


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

Branch: refs/heads/master
Commit: 4d2c262c3be3bbcbfce3214826ecb16dee2cd027
Parents: f17e7c6
Author: Zizhong Zhang <zi...@linkedin.com>
Authored: Mon Dec 14 21:56:51 2015 -0800
Committer: James Peach <jp...@apache.org>
Committed: Mon Dec 14 22:00:12 2015 -0800

----------------------------------------------------------------------
 ci/tsqa/tests/test_https.py | 68 ++++++++++++++++++++++++++++++-
 iocore/net/P_SSLConfig.h    |  2 +
 iocore/net/SSLConfig.cc     |  1 +
 iocore/net/SSLUtils.cc      |  4 ++
 mgmt/BaseManager.h          |  4 +-
 mgmt/FileManager.cc         | 88 +++++++++++++++++++++++++++++++++-------
 mgmt/FileManager.h          | 17 ++++----
 mgmt/LocalManager.cc        | 17 ++++++++
 mgmt/ProcessManager.cc      | 16 ++++++++
 mgmt/ProcessManager.h       |  1 +
 mgmt/ProxyConfig.h          |  4 ++
 mgmt/Rollback.cc            | 48 ++++++++++++++--------
 mgmt/Rollback.h             | 35 +++++++++++++---
 proxy/Main.cc               |  8 ++++
 14 files changed, 265 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/ci/tsqa/tests/test_https.py
----------------------------------------------------------------------
diff --git a/ci/tsqa/tests/test_https.py b/ci/tsqa/tests/test_https.py
index 10410cd..4d0f57a 100644
--- a/ci/tsqa/tests/test_https.py
+++ b/ci/tsqa/tests/test_https.py
@@ -16,11 +16,14 @@
 
 from OpenSSL import SSL
 import socket
-
+import time
 import helpers
 import tsqa.utils
+import os
+import logging
 unittest = tsqa.utils.import_unittest()
 
+log = logging.getLogger(__name__)
 # some ciphers to test with
 CIPHER_MAP = {
     'rsa': 'ECDHE-RSA-AES256-GCM-SHA384',
@@ -273,3 +276,66 @@ class TestMix(helpers.EnvironmentCase, CertSelectionMixin):
 
     def test_intermediate_ca_ecdsa(self):
         self._intermediate_ca_t('ecdsa')
+
+
+class TestConfigFileGroup(helpers.EnvironmentCase, CertSelectionMixin):
+    '''
+    Tests for config file group with https
+    The config file group includes a parent file ssl_multicert.config and some children files.
+    when the content of a child file is updated but the file name hasn't been changed.
+    The behavior is the same as the parent file in the group has been changed.
+    In the test, a child file named www.unknown.com.pem, which is rsa_keys/www.test.com.pem at first,
+      is updated to ec_keys/www.test.com.pem.
+    The difference can be told by different results from calling get_cert() with different ciphers as paramters
+    '''
+    @classmethod
+    def setUpEnv(cls, env):
+        # add an SSL port to ATS
+        cls.ssl_port = tsqa.utils.bind_unused_port()[1]
+        cls.configs['records.config']['CONFIG']['proxy.config.http.server_ports'] += ' {0}:ssl'.format(cls.ssl_port)
+        cls.configs['records.config']['CONFIG'].update({
+            'proxy.config.diags.debug.enabled': 1,
+            'proxy.config.diags.debug.tags': 'ssl',
+            'proxy.config.ssl.server.cipher_suite': '{0}:{1}'.format(CIPHER_MAP['ecdsa'], CIPHER_MAP['rsa']),
+        })
+        cls.configs['ssl_multicert.config'].add_line('dest_ip=* ssl_cert_name={0},{1} ssl_ca_name={2},{3}'.format(
+            helpers.tests_file_path('rsa_keys/www.example.com.pem'),
+            helpers.tests_file_path('ec_keys/www.example.com.pem'),
+            helpers.tests_file_path('rsa_keys/intermediate.crt'),
+            helpers.tests_file_path('ec_keys/intermediate.crt'),
+        ))
+        cls.configs['ssl_multicert.config'].add_line('dest_ip=127.0.0.3 ssl_cert_name={0}'.format(
+            helpers.tests_file_path('www.unknown.com.pem'),
+        ))
+        os.system('cp %s %s' % (helpers.tests_file_path('rsa_keys/www.test.com.pem'), helpers.tests_file_path('www.unknown.com.pem')))
+        log.info('cp %s %s' % (helpers.tests_file_path('rsa_keys/www.test.com.pem'), helpers.tests_file_path('www.unknown.com.pem')))
+
+    def test_config_file_group(self):
+        signal_cmd = os.path.join(self.environment.layout.bindir, 'traffic_line') + ' -x'
+        addr = ('127.0.0.3', self.ssl_port)
+        cert = self._get_cert(addr, ciphers=CIPHER_MAP['rsa'])
+        self.assertEqual(cert.get_subject().commonName.decode(), 'www.test.com')
+        with self.assertRaises(Exception):
+          self._get_cert(addr, ciphers=CIPHER_MAP['ecdsa'])
+        time.sleep(5)
+        os.system('cp %s %s' % (helpers.tests_file_path('ec_keys/www.test.com.pem'), helpers.tests_file_path('www.unknown.com.pem')))
+        log.info('cp %s %s' % (helpers.tests_file_path('ec_keys/www.test.com.pem'), helpers.tests_file_path('www.unknown.com.pem')))
+        os.system(signal_cmd)
+        log.info(signal_cmd)
+        # waiting for the reconfiguration completed
+        sec = 0
+        while True:
+          time.sleep(5)
+          sec += 5
+          log.info("reloading: %d seconds" % (sec))
+          self.assertLess(sec, 30)
+          try:
+            self._get_cert(addr, ciphers=CIPHER_MAP['ecdsa'])
+            break
+          except:
+            continue
+        cert = self._get_cert(addr, ciphers=CIPHER_MAP['ecdsa'])
+        self.assertEqual(cert.get_subject().commonName.decode(), 'www.test.com')
+        with self.assertRaises(Exception):
+          self._get_cert(addr, ciphers=CIPHER_MAP['rsa'])
+        os.system('rm %s' %(helpers.tests_file_path('www.unknown.com.pem')))

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/iocore/net/P_SSLConfig.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLConfig.h b/iocore/net/P_SSLConfig.h
index a8d5eb7..cef3ea3 100644
--- a/iocore/net/P_SSLConfig.h
+++ b/iocore/net/P_SSLConfig.h
@@ -47,6 +47,7 @@ struct SSLCertLookup;
 
 
 typedef void (*init_ssl_ctx_func)(void *, bool);
+typedef void (*load_ssl_file_func)(const char *, unsigned int);
 
 struct SSLConfigParams : public ConfigInfo {
   enum SSL_SESSION_CACHE_MODE {
@@ -107,6 +108,7 @@ struct SSLConfigParams : public ConfigInfo {
   static char *ssl_wire_trace_server_name;
 
   static init_ssl_ctx_func init_ssl_ctx_cb;
+  static load_ssl_file_func load_ssl_file_cb;
 
   void initialize();
   void cleanup();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/iocore/net/SSLConfig.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLConfig.cc b/iocore/net/SSLConfig.cc
index 56a0e88..7bb60fe 100644
--- a/iocore/net/SSLConfig.cc
+++ b/iocore/net/SSLConfig.cc
@@ -53,6 +53,7 @@ size_t SSLConfigParams::session_cache_number_buckets = 1024;
 bool SSLConfigParams::session_cache_skip_on_lock_contention = false;
 size_t SSLConfigParams::session_cache_max_bucket_size = 100;
 init_ssl_ctx_func SSLConfigParams::init_ssl_ctx_cb = NULL;
+load_ssl_file_func SSLConfigParams::load_ssl_file_cb = NULL;
 
 // TS-3534 Wiretracing for SSL Connections
 int SSLConfigParams::ssl_wire_trace_enabled = 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/iocore/net/SSLUtils.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLUtils.cc b/iocore/net/SSLUtils.cc
index ec44b47..4cccff8 100644
--- a/iocore/net/SSLUtils.cc
+++ b/iocore/net/SSLUtils.cc
@@ -1194,6 +1194,7 @@ SSLPrivateKeyHandler(SSL_CTX *ctx, const SSLConfigParams *params, const ats_scop
       SSLError("failed to load server private key from %s", (const char *)completeServerKeyPath);
       return false;
     }
+    SSLConfigParams::load_ssl_file_cb(completeServerKeyPath, CONFIG_FLAG_UNVERSIONED);
   } else {
     SSLError("empty SSL private key path in records.config");
     return false;
@@ -1376,6 +1377,7 @@ SSLInitServerContext(const SSLConfigParams *params, const ssl_user_config &sslMu
         goto fail;
       }
       certList.push_back(cert);
+      SSLConfigParams::load_ssl_file_cb(completeServerCertPath, CONFIG_FLAG_UNVERSIONED);
       // Load up any additional chain certificates
       X509 *ca;
       while ((ca = PEM_read_bio_X509(bio.get(), NULL, 0, NULL))) {
@@ -1398,6 +1400,7 @@ SSLInitServerContext(const SSLConfigParams *params, const ssl_user_config &sslMu
         SSLError("failed to load global certificate chain from %s", (const char *)completeServerCertChainPath);
         goto fail;
       }
+      SSLConfigParams::load_ssl_file_cb(completeServerCertChainPath, CONFIG_FLAG_UNVERSIONED);
     }
 
     // Now, load any additional certificate chains specified in this entry.
@@ -1407,6 +1410,7 @@ SSLInitServerContext(const SSLConfigParams *params, const ssl_user_config &sslMu
         SSLError("failed to load certificate chain from %s", (const char *)completeServerCertChainPath);
         goto fail;
       }
+      SSLConfigParams::load_ssl_file_cb(completeServerCertChainPath, CONFIG_FLAG_UNVERSIONED);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/BaseManager.h
----------------------------------------------------------------------
diff --git a/mgmt/BaseManager.h b/mgmt/BaseManager.h
index c2ece05..676277b 100644
--- a/mgmt/BaseManager.h
+++ b/mgmt/BaseManager.h
@@ -40,7 +40,7 @@
 #include "ts/ink_hash_table.h"
 
 #include "MgmtDefs.h"
-
+#include "MgmtMarshall.h"
 
 /*******************************************
  * used by LocalManager and in Proxy Main. *
@@ -100,6 +100,8 @@
 #define MGMT_SIGNAL_HTTP_CONGESTED_SERVER 20  /* Congestion control -- congested server */
 #define MGMT_SIGNAL_HTTP_ALLEVIATED_SERVER 21 /* Congestion control -- alleviated server */
 
+#define MGMT_SIGNAL_CONFIG_FILE_CHILD 22
+
 #define MGMT_SIGNAL_SAC_SERVER_DOWN 400
 
 typedef struct _mgmt_message_hdr_type {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/FileManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/FileManager.cc b/mgmt/FileManager.cc
index e2d1c90..7abac49 100644
--- a/mgmt/FileManager.cc
+++ b/mgmt/FileManager.cc
@@ -24,6 +24,7 @@
 #include "ts/ink_platform.h"
 #include "ts/ink_file.h"
 #include "ts/I_Layout.h"
+#include "ts/Vec.h"
 #include "FileManager.h"
 #include "Main.h"
 #include "Rollback.h"
@@ -125,7 +126,8 @@ FileManager::registerCallback(FileCallbackFunc func)
   ink_mutex_release(&cbListLock);
 }
 
-// void FileManager::addFile(char* baseFileName, const configFileInfo* file_info)
+// void FileManager::addFile(char* fileName, const configFileInfo* file_info,
+//  Rollback* parentRollback)
 //
 //  for the baseFile, creates a Rollback object for it
 //
@@ -135,47 +137,54 @@ FileManager::registerCallback(FileCallbackFunc func)
 //  Pointers to the new objects are stored in the bindings hashtable
 //
 void
-FileManager::addFile(const char *baseFileName, bool root_access_needed)
+FileManager::addFile(const char *fileName, bool root_access_needed, Rollback *parentRollback, unsigned flags)
 {
-  ink_assert(baseFileName != NULL);
+  ink_mutex_acquire(&accessLock);
+  addFileHelper(fileName, root_access_needed, parentRollback, flags);
+  ink_mutex_release(&accessLock);
+}
 
-  Rollback *rb = new Rollback(baseFileName, root_access_needed);
+// caller must hold the lock
+void
+FileManager::addFileHelper(const char *fileName, bool root_access_needed, Rollback *parentRollback, unsigned flags)
+{
+  ink_assert(fileName != NULL);
+
+  Rollback *rb = new Rollback(fileName, root_access_needed, parentRollback, flags);
   rb->configFiles = this;
 
-  ink_mutex_acquire(&accessLock);
-  ink_hash_table_insert(bindings, baseFileName, rb);
-  ink_mutex_release(&accessLock);
+  ink_hash_table_insert(bindings, fileName, rb);
 }
 
-// bool FileManager::getRollbackObj(char* baseFileName, Rollback** rbPtr)
+// bool FileManager::getRollbackObj(char* fileName, Rollback** rbPtr)
 //
 //  Sets rbPtr to the rollback object associated
-//    with the passed in baseFileName.
+//    with the passed in fileName.
 //
 //  If there is no binding, falseis returned
 //
 bool
-FileManager::getRollbackObj(const char *baseFileName, Rollback **rbPtr)
+FileManager::getRollbackObj(const char *fileName, Rollback **rbPtr)
 {
   InkHashTableValue lookup = NULL;
   int found;
 
   ink_mutex_acquire(&accessLock);
-  found = ink_hash_table_lookup(bindings, baseFileName, &lookup);
+  found = ink_hash_table_lookup(bindings, fileName, &lookup);
   ink_mutex_release(&accessLock);
 
   *rbPtr = (Rollback *)lookup;
   return (found == 0) ? false : true;
 }
 
-// bool FileManager::fileChanged(const char* baseFileName)
+// bool FileManager::fileChanged(const char* fileName)
 //
 //  Called by the Rollback class whenever a a config has changed
 //     Initiates callbacks
 //
 //
 void
-FileManager::fileChanged(const char *baseFileName, bool incVersion)
+FileManager::fileChanged(const char *fileName, bool incVersion)
 {
   callbackListable *cb;
   char *filenameCopy;
@@ -185,7 +194,7 @@ FileManager::fileChanged(const char *baseFileName, bool incVersion)
   for (cb = cblist.head; cb != NULL; cb = cb->link.next) {
     // Dup the string for each callback to be
     //  defensive incase it modified when it is not supposed to be
-    filenameCopy = ats_strdup(baseFileName);
+    filenameCopy = ats_strdup(fileName);
     (*cb->func)(filenameCopy, incVersion);
     ats_free(filenameCopy);
   }
@@ -625,14 +634,44 @@ FileManager::rereadConfig()
   InkHashTableEntry *entry;
   InkHashTableIteratorState iterator_state;
 
+  Vec<Rollback *> changedFiles;
+  Vec<Rollback *> parentFileNeedChange;
   ink_mutex_acquire(&accessLock);
   for (entry = ink_hash_table_iterator_first(bindings, &iterator_state); entry != NULL;
        entry = ink_hash_table_iterator_next(bindings, &iterator_state)) {
     rb = (Rollback *)ink_hash_table_entry_value(bindings, entry);
-    rb->checkForUserUpdate(ROLLBACK_CHECK_AND_UPDATE);
+    if (rb->checkForUserUpdate(rb->isVersioned() ? ROLLBACK_CHECK_ONLY : ROLLBACK_CHECK_AND_UPDATE)) {
+      changedFiles.push_back(rb);
+      if (rb->isChildRollback()) {
+        parentFileNeedChange.add_exclusive(rb->getParentRollback());
+      }
+    }
+  }
+
+  Vec<Rollback *> childFileNeedDelete;
+  for (size_t i = 0; i < changedFiles.n; i++) {
+    if (changedFiles[i]->isChildRollback())
+      continue;
+    // for each parent file, if it is changed, then delete all its children
+    for (entry = ink_hash_table_iterator_first(bindings, &iterator_state); entry != NULL;
+         entry = ink_hash_table_iterator_next(bindings, &iterator_state)) {
+      rb = (Rollback *)ink_hash_table_entry_value(bindings, entry);
+      if (rb->getParentRollback() == changedFiles[i]) {
+        childFileNeedDelete.add_exclusive(rb);
+      }
+    }
+  }
+  for (size_t i = 0; i < childFileNeedDelete.n; i++) {
+    ink_hash_table_delete(bindings, childFileNeedDelete[i]->getFileName());
+    delete childFileNeedDelete[i];
   }
   ink_mutex_release(&accessLock);
 
+  for (size_t i = 0; i < parentFileNeedChange.n; i++) {
+    if (!changedFiles.in(parentFileNeedChange[i])) {
+      fileChanged(parentFileNeedChange[i]->getFileName(), true);
+    }
+  }
   // INKqa11910
   // need to first check that enable_customizations is enabled
   bool found;
@@ -716,6 +755,25 @@ FileManager::createSelect(char *action, textBuffer *output, ExpandingArray *opti
   }
 }
 
+// void configFileChild(const char *parent, const char *child)
+//
+// Add child to the bindings with parentRollback
+void
+FileManager::configFileChild(const char *parent, const char *child, unsigned flags)
+{
+  InkHashTableValue lookup;
+  Rollback *parentRollback = NULL;
+  ink_mutex_acquire(&accessLock);
+  int htfound = ink_hash_table_lookup(bindings, parent, &lookup);
+  if (htfound) {
+    parentRollback = (Rollback *)lookup;
+  }
+  if (htfound) {
+    addFileHelper(child, true, parentRollback, flags);
+  }
+  ink_mutex_release(&accessLock);
+}
+
 // bool checkValidName(const char* name)
 //
 // if the string is invalid, ie. all white spaces or contains "irregular" chars,

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/FileManager.h
----------------------------------------------------------------------
diff --git a/mgmt/FileManager.h b/mgmt/FileManager.h
index 6f9a262..cf2b376 100644
--- a/mgmt/FileManager.h
+++ b/mgmt/FileManager.h
@@ -80,11 +80,11 @@ class ExpandingArray;
 //       is also created
 //
 //  getRollbckObj(char* , RollbackPtr**) - sets *rbPtr to Rollback
-//       object bound to baseFileName.  Returns true if there is
+//       object bound to fileName.  Returns true if there is
 //       a binding and false otherwise
 //
 //  getWFEObj(char*, WebFileEdit**)  - sets *wfePtr to WebFileEdit
-//       object bound to baseFileName.  Returns true if there is
+//       object bound to fileName.  Returns true if there is
 //       a binding and false otherwise
 //
 //  registerCallback(FileCallbackFunc) - registers a callback function
@@ -92,7 +92,7 @@ class ExpandingArray;
 //       callback function should NOT use the calling thread to
 //       access any Rollback objects or block for a long time
 //
-//  fileChanged(const char* baseFileName) - called by Rollback objects
+//  fileChanged(const char* fileName) - called by Rollback objects
 //       when their contents change.  Triggers callbacks to FileCallbackFuncs
 //
 //  filesManaged() - returns a textBuffer that contains a new line separated
@@ -110,16 +110,16 @@ class ExpandingArray;
 //
 //  rereadConfig() - Checks all managed files to see if they have been
 //       updated
-//
+//  addConfigFileGroup(char* data_str, int data_size) - update config file group infos
 class FileManager : public MultiFile
 {
 public:
   FileManager();
   ~FileManager();
-  void addFile(const char *baseFileName, bool root_access_needed);
-  bool getRollbackObj(const char *baseFileName, Rollback **rbPtr);
+  void addFile(const char *fileName, bool root_access_needed, Rollback *parentRollback = NULL, unsigned flags = 0);
+  bool getRollbackObj(const char *fileName, Rollback **rbPtr);
   void registerCallback(FileCallbackFunc func);
-  void fileChanged(const char *baseFileName, bool incVersion);
+  void fileChanged(const char *fileName, bool incVersion);
   textBuffer *filesManaged();
   void rereadConfig();
   bool isConfigStale();
@@ -131,6 +131,7 @@ public:
   SnapResult removeSnap(const char *snapName, const char *snapDir);
   void displaySnapOption(textBuffer *output);
   SnapResult WalkSnaps(ExpandingArray *snapList);
+  void configFileChild(const char *parent, const char *child, unsigned int options);
 
 private:
   void doRollbackLocks(lockAction_t action);
@@ -147,6 +148,8 @@ private:
   void snapSuccessResponse(char *action, textBuffer *output);
   void generateRestoreConfirm(char *snapName, textBuffer *output);
   bool checkValidName(const char *name);
+  const char *getParentFileName(const char *fileName);
+  void addFileHelper(const char *fileName, bool root_access_needed, Rollback *parentRollback, unsigned flags = 0);
 };
 
 int snapEntryCmpFunc(const void *e1, const void *e2);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/LocalManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/LocalManager.cc b/mgmt/LocalManager.cc
index 6fc1391..8481c6f 100644
--- a/mgmt/LocalManager.cc
+++ b/mgmt/LocalManager.cc
@@ -604,6 +604,23 @@ LocalManager::handleMgmtMsgFromProcesses(MgmtMessageHdr *mh)
     alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_HTTP_ALLEVIATED_SERVER, data_raw);
     break;
   // Congestion Control - end
+  case MGMT_SIGNAL_CONFIG_FILE_CHILD: {
+    static const MgmtMarshallType fields[] = {
+      MGMT_MARSHALL_STRING, MGMT_MARSHALL_STRING, MGMT_MARSHALL_INT
+    };
+    char *parent = NULL;
+    char *child = NULL;
+    MgmtMarshallInt options = 0;
+    if (mgmt_message_parse(data_raw, mh->data_len, fields, countof(fields), &parent, &child, &options) != -1) {
+      configFiles->configFileChild(parent, child, (unsigned int)options);
+    } else {
+      mgmt_elog(stderr, 0, "[LocalManager::handleMgmtMsgFromProcesses] "
+                           "MGMT_SIGNAL_CONFIG_FILE_CHILD mgmt_message_parse error\n");
+    }
+    // Output pointers are guaranteed to be NULL or valid.
+    ats_free_null(parent);
+    ats_free_null(child);
+  } break;
   case MGMT_SIGNAL_SAC_SERVER_DOWN:
     alarm_keeper->signalAlarm(MGMT_ALARM_SAC_SERVER_DOWN, data_raw);
     break;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/ProcessManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/ProcessManager.cc b/mgmt/ProcessManager.cc
index cb8cba2..e77b200 100644
--- a/mgmt/ProcessManager.cc
+++ b/mgmt/ProcessManager.cc
@@ -88,6 +88,22 @@ ProcessManager::reconfigure()
   return;
 } /* End ProcessManager::reconfigure */
 
+void
+ProcessManager::signalConfigFileChild(const char *parent, const char *child, unsigned int options)
+{
+  static const MgmtMarshallType fields[] = {
+    MGMT_MARSHALL_STRING, MGMT_MARSHALL_STRING, MGMT_MARSHALL_INT
+  };
+
+  MgmtMarshallInt mgmtopt = options;
+  size_t len = mgmt_message_length(fields, countof(fields), &parent, &child, &mgmtopt);
+  void *buffer = ats_malloc(len);
+
+  mgmt_message_marshall(buffer, len, fields, countof(fields), &parent, &child, &mgmtopt);
+  signalManager(MGMT_SIGNAL_CONFIG_FILE_CHILD, (const char *)buffer, len);
+
+  ats_free(buffer);
+}
 
 void
 ProcessManager::signalManager(int msg_id, const char *data_str)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/ProcessManager.h
----------------------------------------------------------------------
diff --git a/mgmt/ProcessManager.h b/mgmt/ProcessManager.h
index ad3e9ab..90aa03b 100644
--- a/mgmt/ProcessManager.h
+++ b/mgmt/ProcessManager.h
@@ -72,6 +72,7 @@ public:
     close_socket(local_manager_sockfd);
   }
 
+  inkcoreapi void signalConfigFileChild(const char *parent, const char *child, unsigned int options);
   inkcoreapi void signalManager(int msg_id, const char *data_str);
   inkcoreapi void signalManager(int msg_id, const char *data_raw, int data_len);
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/ProxyConfig.h
----------------------------------------------------------------------
diff --git a/mgmt/ProxyConfig.h b/mgmt/ProxyConfig.h
index ea481e7..eaa63aa 100644
--- a/mgmt/ProxyConfig.h
+++ b/mgmt/ProxyConfig.h
@@ -44,6 +44,10 @@ void *config_float_cb(void *data, void *value);
 void *config_string511_cb(void *data, void *value);
 void *config_string_alloc_cb(void *data, void *value);
 
+// Configuration file flags shared by proxy configuration and mgmt.
+#define CONFIG_FLAG_NONE        0u
+#define CONFIG_FLAG_UNVERSIONED 1u  // Don't version this config file
+
 //
 // Macros that spin waiting for the data to be bound
 //

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/Rollback.cc
----------------------------------------------------------------------
diff --git a/mgmt/Rollback.cc b/mgmt/Rollback.cc
index e7d581b..13ba97d 100644
--- a/mgmt/Rollback.cc
+++ b/mgmt/Rollback.cc
@@ -35,6 +35,7 @@
 #include "ts/ink_cap.h"
 #include "ts/I_Layout.h"
 #include "FileManager.h"
+#include "ProxyConfig.h"
 
 #define MAX_VERSION_DIGITS 11
 #define DEFAULT_BACKUPS 2
@@ -43,7 +44,8 @@
 const char *RollbackStrings[] = {"Rollback Ok", "File was not found", "Version was out of date", "System Call Error",
                                  "Invalid Version - Version Numbers Must Increase"};
 
-Rollback::Rollback(const char *baseFileName, bool root_access_needed_) : configFiles(NULL), root_access_needed(root_access_needed_)
+Rollback::Rollback(const char *fileName_, bool root_access_needed_, Rollback *parentRollback_, unsigned flags)
+  : configFiles(NULL), root_access_needed(root_access_needed_), parentRollback(parentRollback_)
 {
   version_t highestSeen;             // the highest backup version
   ExpandingArray existVer(25, true); // Exsisting versions
@@ -60,22 +62,27 @@ Rollback::Rollback(const char *baseFileName, bool root_access_needed_) : configF
   char *activeVerStr;
   bool needZeroLength;
 
-  ink_assert(baseFileName != NULL);
+  ink_assert(fileName_ != NULL);
 
-  // Copy the file name
-  fileNameLen = strlen(baseFileName);
-  fileName = (char *)ats_malloc(fileNameLen + 1);
-  ink_strlcpy(fileName, baseFileName, fileNameLen + 1);
+  // parent must not also have a parent
+  if (parentRollback) {
+    ink_assert(parentRollback->parentRollback == NULL);
+  }
 
-  // TODO: Use the runtime directory for storing mutable data
-  // XXX: Sysconfdir should be imutable!!!
+  // Copy the file name.
+  fileNameLen = strlen(fileName_);
+  fileName = ats_strdup(fileName_);
 
-  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));
+  // Extract the file base name.
+  fileBaseName = strrchr(fileName, '/');
+  if (fileBaseName) {
+    fileBaseName++;
+  } else {
+    fileBaseName = fileName;
   }
 
+  ink_mutex_init(&fileAccessLock, "RollBack Mutex");
+
   if (varIntFromName("proxy.config.admin.number_config_bak", &numBak) == true) {
     if (numBak > 1) {
       numberBackups = (int)numBak;
@@ -86,7 +93,13 @@ Rollback::Rollback(const char *baseFileName, bool root_access_needed_) : configF
     numberBackups = DEFAULT_BACKUPS;
   }
 
-  ink_mutex_init(&fileAccessLock, "RollBack Mutex");
+  // If we are not doing backups, bail early.
+  if ((numberBackups <= 0) || (flags & CONFIG_FLAG_UNVERSIONED)) {
+    currentVersion = 0;
+    setLastModifiedTime();
+    numberBackups = 0;
+    return;
+  }
 
   currentVersion = 0; // Prevent UMR with stat file
   highestSeen = findVersions_ml(versionQ);
@@ -216,12 +229,12 @@ Rollback::~Rollback()
 char *
 Rollback::createPathStr(version_t version)
 {
+  int bufSize = 0;
+  char *buffer = NULL;
   ats_scoped_str sysconfdir(RecConfigReadConfigDir());
-  int bufSize = strlen(sysconfdir) + fileNameLen + MAX_VERSION_DIGITS + 1;
-  char *buffer = (char *)ats_malloc(bufSize);
-
+  bufSize = strlen(sysconfdir) + fileNameLen + MAX_VERSION_DIGITS + 1;
+  buffer = (char *)ats_malloc(bufSize);
   Layout::get()->relative_to(buffer, bufSize, sysconfdir, fileName);
-
   if (version != ACTIVE_VERSION) {
     size_t pos = strlen(buffer);
     snprintf(buffer + pos, bufSize - pos, "_%d", version);
@@ -389,7 +402,6 @@ Rollback::internalUpdate(textBuffer *buf, version_t newVersion, bool notifyChang
   currentVersion_local = createPathStr(this->currentVersion);
   activeVersion = createPathStr(ACTIVE_VERSION);
   nextVersion = createPathStr(newVersion);
-
   // Create the new configuration file
   // TODO: Make sure they are not created in Sysconfigdir!
   diskFD = openFile(newVersion, O_WRONLY | O_CREAT | O_TRUNC);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/mgmt/Rollback.h
----------------------------------------------------------------------
diff --git a/mgmt/Rollback.h b/mgmt/Rollback.h
index 082addb..6aefdb2 100644
--- a/mgmt/Rollback.h
+++ b/mgmt/Rollback.h
@@ -156,7 +156,8 @@ struct versionInfo {
 class Rollback
 {
 public:
-  Rollback(const char *baseFileName, bool root_access_needed);
+  // fileName_ should be rooted or a base file name.
+  Rollback(const char *fileName_, bool root_access_needed, Rollback *parentRollback = NULL, unsigned flags = 0);
   ~Rollback();
 
   // Manual take out of lock required
@@ -197,22 +198,42 @@ public:
   // Lock not necessary since these are only valid for a
   //  snap shot in time
   version_t
-  getCurrentVersion()
+  getCurrentVersion() const
   {
     return currentVersion;
   };
   int
-  numberOfVersions()
+  numberOfVersions() const
   {
     return numVersions;
-  };
+  }
 
   // Not file based so no lock necessary
   const char *
-  getBaseName()
+  getBaseName() const
+  {
+    return fileBaseName;
+  }
+  const char *
+  getFileName() const
   {
     return fileName;
-  };
+  }
+  bool
+  isChildRollback() const
+  {
+    return parentRollback != NULL;
+  }
+  Rollback *
+  getParentRollback() const
+  {
+    return parentRollback;
+  }
+  bool
+  isVersioned() const
+  {
+    return numberBackups > 0;
+  }
 
   FileManager *configFiles; // Manager to notify on an update.
 
@@ -225,8 +246,10 @@ private:
   RollBackCodes internalUpdate(textBuffer *buf, version_t newVersion, bool notifyChange = true, bool incVersion = true);
   ink_mutex fileAccessLock;
   char *fileName;
+  char *fileBaseName;
   size_t fileNameLen;
   bool root_access_needed;
+  Rollback *parentRollback;
   version_t currentVersion;
   time_t fileLastModified;
   int numVersions;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d2c262c/proxy/Main.cc
----------------------------------------------------------------------
diff --git a/proxy/Main.cc b/proxy/Main.cc
index 8efbb18..7e91eaf 100644
--- a/proxy/Main.cc
+++ b/proxy/Main.cc
@@ -124,6 +124,7 @@ static const long MAX_LOGIN = ink_login_name_max();
 static void *mgmt_restart_shutdown_callback(void *, char *, int data_len);
 static void *mgmt_storage_device_cmd_callback(void *x, char *data, int len);
 static void init_ssl_ctx_callback(void *ctx, bool server);
+static void load_ssl_file_callback(const char *ssl_file, unsigned int options);
 
 static int num_of_net_threads = ink_number_of_processors();
 static int num_of_udp_threads = 0;
@@ -1801,6 +1802,7 @@ main(int /* argc ATS_UNUSED */, const char **argv)
     (void)plugin_init(); // plugin.config
 
     SSLConfigParams::init_ssl_ctx_cb = init_ssl_ctx_callback;
+    SSLConfigParams::load_ssl_file_cb = load_ssl_file_callback;
     sslNetProcessor.start(getNumSSLThreads(), stacksize);
 
     pmgmt->registerPluginCallbacks(global_config_cbs);
@@ -1976,3 +1978,9 @@ init_ssl_ctx_callback(void *ctx, bool server)
     hook = hook->next();
   }
 }
+
+static void
+load_ssl_file_callback(const char *ssl_file, unsigned int options)
+{
+  pmgmt->signalConfigFileChild("ssl_multicert.config", ssl_file, options);
+}