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/08/28 05:59:59 UTC

[1/3] trafficserver git commit: TS-3818: add traffic_ctl config defaults subcommand

Repository: trafficserver
Updated Branches:
  refs/heads/master 080bcaa07 -> b1f81f881


TS-3818: add traffic_ctl config defaults subcommand

Add a traffic_ctl subcommand to show the default values of all
configuration records.


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

Branch: refs/heads/master
Commit: b1f81f8819b0624eff8e96a54ecdccbce9bcf7a9
Parents: 2c19cd2
Author: James Peach <jp...@apache.org>
Authored: Tue Aug 4 21:29:46 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Thu Aug 27 20:53:54 2015 -0700

----------------------------------------------------------------------
 cmd/traffic_ctl/config.cc                 | 82 +++++++++++++++++---------
 cmd/traffic_ctl/traffic_ctl.cc            |  6 +-
 doc/reference/commands/traffic_ctl.en.rst |  7 +++
 3 files changed, 66 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/b1f81f88/cmd/traffic_ctl/config.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/config.cc b/cmd/traffic_ctl/config.cc
index 61e07fe..4295407 100644
--- a/cmd/traffic_ctl/config.cc
+++ b/cmd/traffic_ctl/config.cc
@@ -157,6 +157,19 @@ rec_sourceof(int rec_source)
   }
 }
 
+static const char *
+rec_labelof(int rec_class)
+{
+  switch (rec_class) {
+  case RECT_CONFIG:
+    return "CONFIG";
+  case RECT_LOCAL:
+    return "LOCAL";
+  default:
+    return NULL;
+  }
+}
+
 static std::string
 timestr(time_t tm)
 {
@@ -170,20 +183,7 @@ format_record(const CtrlMgmtRecord &record, bool recfmt)
   CtrlMgmtRecordValue value(record);
 
   if (recfmt) {
-    const char *label;
-
-    switch (record.rclass()) {
-    case RECT_CONFIG:
-      label = "CONFIG ";
-      break;
-    case RECT_LOCAL:
-      label = "LOCAL ";
-      break;
-    default:
-      label = "";
-    }
-
-    printf("%s%s %s %s\n", label, record.name(), rec_typeof(record.type()), value.c_str());
+    printf("%s %s %s %s\n", rec_labelof(record.rclass()), record.name(), rec_typeof(record.type()), value.c_str());
   } else {
     printf("%s: %s\n", record.name(), value.c_str());
   }
@@ -391,6 +391,43 @@ config_status(unsigned argc, const char **argv)
 }
 
 static int
+config_defaults(unsigned argc, const char **argv)
+{
+  int recfmt = 0;
+  const ArgumentDescription opts[] = {
+    {"records", '-', "Emit output in records.config format", "F", &recfmt, NULL, NULL},
+  };
+
+  if (!CtrlProcessArguments(argc, argv, opts, countof(opts)) || n_file_arguments != 0) {
+    return CtrlCommandUsage("config diff [OPTIONS]");
+  }
+
+  TSMgmtError error;
+  CtrlMgmtRecordDescriptionList descriptions;
+
+  error = descriptions.match(".*");
+  if (error != TS_ERR_OKAY) {
+    CtrlMgmtError(error, "failed to fetch record metadata");
+    return CTRL_EX_ERROR;
+  }
+
+  while (!descriptions.empty()) {
+    TSConfigRecordDescription *desc = descriptions.next();
+    CtrlMgmtRecordValue deflt(desc->rec_type, desc->rec_default);
+
+    if (recfmt) {
+      printf("%s %s %s %s\n", rec_labelof(desc->rec_class), desc->rec_name, rec_typeof(desc->rec_type), deflt.c_str());
+    } else {
+      printf("%s: %s\n", desc->rec_name, deflt.c_str());
+    }
+
+    TSConfigRecordDescriptionDestroy(desc);
+  }
+
+  return CTRL_EX_OK;
+}
+
+static int
 config_diff(unsigned argc, const char **argv)
 {
   int recfmt = 0;
@@ -439,20 +476,8 @@ config_diff(unsigned argc, const char **argv)
       CtrlMgmtRecordValue deflt(desc->rec_type, desc->rec_default);
 
       if (recfmt) {
-        const char *label;
-
-        switch (desc->rec_class) {
-        case RECT_CONFIG:
-          label = "CONFIG ";
-          break;
-        case RECT_LOCAL:
-          label = "LOCAL ";
-          break;
-        default:
-          label = "";
-        }
-
-        printf("%s%s %s %s # default: %s\n", label, desc->rec_name, rec_typeof(desc->rec_type), current.c_str(), deflt.c_str());
+        printf("%s %s %s %s # default: %s\n", rec_labelof(desc->rec_class), desc->rec_name, rec_typeof(desc->rec_type),
+               current.c_str(), deflt.c_str());
       } else {
         printf("%s has changed\n", desc->rec_name);
         printf("\t%-16s: %s\n", "Current Value", current.c_str());
@@ -470,6 +495,7 @@ int
 subcommand_config(unsigned argc, const char **argv)
 {
   const subcommand commands[] = {
+    {config_defaults, "defaults", "Show default information configuration values"},
     {config_describe, "describe", "Show detailed information about configuration values"},
     {config_diff, "diff", "Show non-default configuration values"},
     {config_get, "get", "Get one or more configuration values"},

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/b1f81f88/cmd/traffic_ctl/traffic_ctl.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/traffic_ctl.cc b/cmd/traffic_ctl/traffic_ctl.cc
index c1e11f2..7601c6d 100644
--- a/cmd/traffic_ctl/traffic_ctl.cc
+++ b/cmd/traffic_ctl/traffic_ctl.cc
@@ -98,7 +98,11 @@ CtrlMgmtRecordValue::init(TSRecordT _t, TSRecordValueT _v)
     snprintf(this->fmt.nbuf, sizeof(this->fmt.nbuf), "%f", _v.float_val);
     break;
   case TS_REC_STRING:
-    this->fmt.str = _v.string_val;
+    if (strcmp(_v.string_val, "") == 0) {
+      this->fmt.str = "\"\"";
+    } else {
+      this->fmt.str = _v.string_val;
+    }
     break;
   default:
     rec_type = TS_REC_STRING;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/b1f81f88/doc/reference/commands/traffic_ctl.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/commands/traffic_ctl.en.rst b/doc/reference/commands/traffic_ctl.en.rst
index 7393e6a..f40e2bf 100644
--- a/doc/reference/commands/traffic_ctl.en.rst
+++ b/doc/reference/commands/traffic_ctl.en.rst
@@ -97,6 +97,13 @@ traffic_ctl cluster
 traffic_ctl config
 ------------------
 .. program:: traffic_ctl config
+.. option:: defaults [--records]
+
+    Display the default values for all configuration records. The
+    *--records* flag has the same behavior as :option:`traffic_ctl
+    config get --records`.
+
+.. program:: traffic_ctl config
 .. option:: describe RECORD [RECORD...]
 
     Display all the known information about a configuration record.


[2/3] trafficserver git commit: TS-2861: add traffic_ctl config diff

Posted by jp...@apache.org.
TS-2861: add traffic_ctl config diff

Add a traffic_ctl option to show which options have been changed
from the defaults.


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

Branch: refs/heads/master
Commit: 2c19cd2b70b42b283811a042ab0126436c4670d2
Parents: 244b78e
Author: James Peach <jp...@apache.org>
Authored: Tue Aug 4 16:57:53 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Thu Aug 27 20:53:54 2015 -0700

----------------------------------------------------------------------
 cmd/traffic_ctl/config.cc                 | 101 +++++++++++++++
 cmd/traffic_ctl/traffic_ctl.h             |  43 ++-----
 doc/reference/commands/traffic_ctl.en.rst |   7 ++
 lib/records/I_RecCore.h                   |   2 +-
 lib/records/RecCore.cc                    |  17 ++-
 mgmt/api/CoreAPI.cc                       |   6 +
 mgmt/api/CoreAPI.h                        |   1 +
 mgmt/api/CoreAPIRemote.cc                 | 165 ++++++++++++++++---------
 mgmt/api/INKMgmtAPI.cc                    |  28 +++++
 mgmt/api/NetworkMessage.h                 |   4 +
 mgmt/api/TSControlMain.cc                 | 139 +++++++++++++--------
 mgmt/api/include/mgmtapi.h                |   6 +
 12 files changed, 372 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/cmd/traffic_ctl/config.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/config.cc b/cmd/traffic_ctl/config.cc
index 44b6e82..61e07fe 100644
--- a/cmd/traffic_ctl/config.cc
+++ b/cmd/traffic_ctl/config.cc
@@ -26,6 +26,30 @@
 #include <I_RecDefs.h>
 #include <P_RecUtils.h>
 
+struct RecordDescriptionPolicy {
+  typedef TSConfigRecordDescription *entry_type;
+
+  static void
+  free(entry_type e)
+  {
+    TSConfigRecordDescriptionDestroy(e);
+  }
+
+  static entry_type
+  cast(void *ptr)
+  {
+    return (entry_type)ptr;
+  }
+};
+
+struct CtrlMgmtRecordDescriptionList : CtrlMgmtList<RecordDescriptionPolicy> {
+  TSMgmtError
+  match(const char *regex)
+  {
+    return TSConfigRecordDescribeMatchMlt(regex, 0u /* flags */, this->list);
+  }
+};
+
 // Record data type names, indexed by TSRecordT.
 static const char *
 rec_typeof(int rec_type)
@@ -366,11 +390,88 @@ config_status(unsigned argc, const char **argv)
   return CTRL_EX_OK;
 }
 
+static int
+config_diff(unsigned argc, const char **argv)
+{
+  int recfmt = 0;
+  const ArgumentDescription opts[] = {
+    {"records", '-', "Emit output in records.config format", "F", &recfmt, NULL, NULL},
+  };
+
+  if (!CtrlProcessArguments(argc, argv, opts, countof(opts)) || n_file_arguments != 0) {
+    return CtrlCommandUsage("config diff [OPTIONS]");
+  }
+
+  TSMgmtError error;
+  CtrlMgmtRecordDescriptionList descriptions;
+
+  error = descriptions.match(".*");
+  if (error != TS_ERR_OKAY) {
+    CtrlMgmtError(error, "failed to fetch record metadata");
+    return CTRL_EX_ERROR;
+  }
+
+  while (!descriptions.empty()) {
+    TSConfigRecordDescription *desc;
+    bool changed = false;
+
+    desc = descriptions.next();
+
+    switch (desc->rec_type) {
+    case TS_REC_INT:
+      changed = (desc->rec_value.int_val != desc->rec_default.int_val);
+      break;
+    case TS_REC_COUNTER:
+      changed = (desc->rec_value.counter_val != desc->rec_default.counter_val);
+      break;
+    case TS_REC_FLOAT:
+      changed = (desc->rec_value.float_val != desc->rec_default.float_val);
+      break;
+    case TS_REC_STRING:
+      changed = (strcmp(desc->rec_value.string_val, desc->rec_default.string_val) != 0);
+      break;
+    default:
+      break;
+    }
+
+    if (changed) {
+      CtrlMgmtRecordValue current(desc->rec_type, desc->rec_value);
+      CtrlMgmtRecordValue deflt(desc->rec_type, desc->rec_default);
+
+      if (recfmt) {
+        const char *label;
+
+        switch (desc->rec_class) {
+        case RECT_CONFIG:
+          label = "CONFIG ";
+          break;
+        case RECT_LOCAL:
+          label = "LOCAL ";
+          break;
+        default:
+          label = "";
+        }
+
+        printf("%s%s %s %s # default: %s\n", label, desc->rec_name, rec_typeof(desc->rec_type), current.c_str(), deflt.c_str());
+      } else {
+        printf("%s has changed\n", desc->rec_name);
+        printf("\t%-16s: %s\n", "Current Value", current.c_str());
+        printf("\t%-16s: %s\n", "Default Value", deflt.c_str());
+      }
+    }
+
+    TSConfigRecordDescriptionDestroy(desc);
+  }
+
+  return CTRL_EX_OK;
+}
+
 int
 subcommand_config(unsigned argc, const char **argv)
 {
   const subcommand commands[] = {
     {config_describe, "describe", "Show detailed information about configuration values"},
+    {config_diff, "diff", "Show non-default configuration values"},
     {config_get, "get", "Get one or more configuration values"},
     {config_match, "match", "Get configuration matching a regular expression"},
     {config_reload, "reload", "Request a configuration reload"},

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/cmd/traffic_ctl/traffic_ctl.h
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/traffic_ctl.h b/cmd/traffic_ctl/traffic_ctl.h
index bc0847a..751d266 100644
--- a/cmd/traffic_ctl/traffic_ctl.h
+++ b/cmd/traffic_ctl/traffic_ctl.h
@@ -110,43 +110,20 @@ private:
   } fmt;
 };
 
-struct CtrlMgmtRecordList {
-  CtrlMgmtRecordList() : list(TSListCreate()) {}
+struct RecordListPolicy {
+  typedef TSRecordEle *entry_type;
 
-  ~CtrlMgmtRecordList()
+  static void
+  free(entry_type e)
   {
-    this->clear();
-    TSListDestroy(this->list);
-  }
-
-  bool
-  empty() const
-  {
-    return TSListIsEmpty(this->list);
-  }
-
-  void
-  clear() const
-  {
-    while (!this->empty()) {
-      TSRecordEleDestroy((TSRecordEle *)TSListDequeue(this->list));
-    }
+    TSRecordEleDestroy(e);
   }
 
-  // Return (ownership of) the next list entry.
-  TSRecordEle *
-  next()
+  static entry_type
+  cast(void *ptr)
   {
-    return (TSRecordEle *)TSListDequeue(this->list);
+    return (entry_type)ptr;
   }
-
-  TSMgmtError match(const char *);
-
-private:
-  CtrlMgmtRecordList(const CtrlMgmtRecordList &);            // disabled
-  CtrlMgmtRecordList &operator=(const CtrlMgmtRecordList &); // disabled
-
-  TSList list;
 };
 
 template <typename T> struct CtrlMgmtList {
@@ -186,6 +163,10 @@ private:
   CtrlMgmtList &operator=(const CtrlMgmtList &); // disabled
 };
 
+struct CtrlMgmtRecordList : CtrlMgmtList<RecordListPolicy> {
+  TSMgmtError match(const char *);
+};
+
 struct CtrlCommandLine {
   CtrlCommandLine() { this->args.push_back(NULL); }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/doc/reference/commands/traffic_ctl.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/commands/traffic_ctl.en.rst b/doc/reference/commands/traffic_ctl.en.rst
index 86df74a..7393e6a 100644
--- a/doc/reference/commands/traffic_ctl.en.rst
+++ b/doc/reference/commands/traffic_ctl.en.rst
@@ -104,6 +104,13 @@ traffic_ctl config
     the record class and syntax checking expression.
 
 .. program:: traffic_ctl config
+.. option:: diff [--records]
+
+    Display configuration records that have non-default values. The
+    *--records* flag has the same behavior as :option:`traffic_ctl
+    config get --records`.
+
+.. program:: traffic_ctl config
 .. option:: get [--records] RECORD [RECORD...]
 
     Display the current value of a configuration record.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/lib/records/I_RecCore.h
----------------------------------------------------------------------
diff --git a/lib/records/I_RecCore.h b/lib/records/I_RecCore.h
index 1d8ab52..eb4eb4b 100644
--- a/lib/records/I_RecCore.h
+++ b/lib/records/I_RecCore.h
@@ -169,7 +169,7 @@ int RecGetRecordBool(const char *name, RecBool *rec_byte, bool lock = true);
 typedef void (*RecLookupCallback)(const RecRecord *, void *);
 
 int RecLookupRecord(const char *name, RecLookupCallback callback, void *data, bool lock = true);
-int RecLookupMatchingRecords(const char *match, RecLookupCallback callback, void *data, bool lock = true);
+int RecLookupMatchingRecords(unsigned rec_type, const char *match, RecLookupCallback callback, void *data, bool lock = true);
 
 int RecGetRecordType(const char *name, RecT *rec_type, bool lock = true);
 int RecGetRecordDataType(const char *name, RecDataT *data_type, bool lock = true);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/lib/records/RecCore.cc
----------------------------------------------------------------------
diff --git a/lib/records/RecCore.cc b/lib/records/RecCore.cc
index 041a218..d35230d 100644
--- a/lib/records/RecCore.cc
+++ b/lib/records/RecCore.cc
@@ -455,7 +455,7 @@ RecLookupRecord(const char *name, void (*callback)(const RecRecord *, void *), v
 }
 
 int
-RecLookupMatchingRecords(const char *match, void (*callback)(const RecRecord *, void *), void *data, bool lock)
+RecLookupMatchingRecords(unsigned rec_type, const char *match, void (*callback)(const RecRecord *, void *), void *data, bool lock)
 {
   int num_records;
   DFA regex;
@@ -467,11 +467,18 @@ RecLookupMatchingRecords(const char *match, void (*callback)(const RecRecord *,
   num_records = g_num_records;
   for (int i = 0; i < num_records; i++) {
     RecRecord *r = &(g_records[i]);
-    if (regex.match(r->name) >= 0) {
-      rec_mutex_acquire(&(r->lock));
-      callback(r, data);
-      rec_mutex_release(&(r->lock));
+
+    if ((r->rec_type & rec_type) == 0) {
+      continue;
+    }
+
+    if (regex.match(r->name) < 0) {
+      continue;
     }
+
+    rec_mutex_acquire(&(r->lock));
+    callback(r, data);
+    rec_mutex_release(&(r->lock));
   }
 
   return REC_ERR_OKAY;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/mgmt/api/CoreAPI.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/CoreAPI.cc b/mgmt/api/CoreAPI.cc
index b14320b..972de27 100644
--- a/mgmt/api/CoreAPI.cc
+++ b/mgmt/api/CoreAPI.cc
@@ -565,6 +565,12 @@ MgmtConfigRecordDescribe(const char * /* rec_name */, unsigned /* flags */, TSCo
   return TS_ERR_NOT_SUPPORTED;
 }
 
+TSMgmtError
+MgmtConfigRecordDescribeMatching(const char *, unsigned, TSList)
+{
+  return TS_ERR_NOT_SUPPORTED;
+}
+
 /*-------------------------------------------------------------------------
  * reads the RecordsConfig info to determine which type of action is needed
  * when the record rec_name is changed; if the rec_name is invalid,

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/mgmt/api/CoreAPI.h
----------------------------------------------------------------------
diff --git a/mgmt/api/CoreAPI.h b/mgmt/api/CoreAPI.h
index c803299..c6ed7e5 100644
--- a/mgmt/api/CoreAPI.h
+++ b/mgmt/api/CoreAPI.h
@@ -66,6 +66,7 @@ TSMgmtError MgmtRecordSetString(const char *rec_name, const char *string_val, TS
 TSMgmtError MgmtRecordGetMatching(const char *regex, TSList rec_vals);
 
 TSMgmtError MgmtConfigRecordDescribe(const char *rec_name, unsigned flags, TSConfigRecordDescription *val);
+TSMgmtError MgmtConfigRecordDescribeMatching(const char *regex, unsigned flags, TSList rec_vals);
 
 /***************************************************************************
  * File Operations

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/mgmt/api/CoreAPIRemote.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/CoreAPIRemote.cc b/mgmt/api/CoreAPIRemote.cc
index 4ce8cae..8622945 100644
--- a/mgmt/api/CoreAPIRemote.cc
+++ b/mgmt/api/CoreAPIRemote.cc
@@ -534,6 +534,72 @@ done:
   return ret;
 }
 
+static TSMgmtError
+mgmt_record_describe_reply(TSConfigRecordDescription *val)
+{
+  TSMgmtError ret;
+  MgmtMarshallData reply = {NULL, 0};
+
+  ret = recv_mgmt_message(main_socket_fd, reply);
+  if (ret != TS_ERR_OKAY) {
+    return ret;
+  }
+
+  MgmtMarshallInt err;
+  MgmtMarshallString name = NULL;
+  MgmtMarshallString expr = NULL;
+  MgmtMarshallData value = {NULL, 0};
+  MgmtMarshallData deflt = {NULL, 0};
+
+  MgmtMarshallInt rtype;
+  MgmtMarshallInt rclass;
+  MgmtMarshallInt version;
+  MgmtMarshallInt rsb;
+  MgmtMarshallInt order;
+  MgmtMarshallInt access;
+  MgmtMarshallInt update;
+  MgmtMarshallInt updatetype;
+  MgmtMarshallInt checktype;
+  MgmtMarshallInt source;
+
+  ret = recv_mgmt_response(reply.ptr, reply.len, RECORD_DESCRIBE_CONFIG, &err, &name, &value, &deflt, &rtype, &rclass, &version,
+                           &rsb, &order, &access, &update, &updatetype, &checktype, &source, &expr);
+
+  ats_free(reply.ptr);
+
+  if (ret != TS_ERR_OKAY) {
+    goto done;
+  }
+
+  if (err != TS_ERR_OKAY) {
+    ret = (TSMgmtError)err;
+    goto done;
+  }
+
+  // Everything is cool, populate the description ...
+  val->rec_name = ats_strdup(name);
+  val->rec_checkexpr = ats_strdup(expr);
+  val->rec_type = (TSRecordT)rtype;
+  val->rec_class = rclass;
+  val->rec_version = version;
+  val->rec_rsb = rsb;
+  val->rec_order = order;
+  val->rec_access = access;
+  val->rec_updatetype = updatetype;
+  val->rec_checktype = checktype;
+  val->rec_source = source;
+
+  mgmt_record_convert_value(val->rec_type, value, val->rec_value);
+  mgmt_record_convert_value(val->rec_type, deflt, val->rec_default);
+
+done:
+  ats_free(name);
+  ats_free(expr);
+  ats_free(value.ptr);
+  ats_free(deflt.ptr);
+  return ret;
+}
+
 // note that the record value is being sent as chunk of memory, regardless of
 // record type; it's not being converted to a string!!
 TSMgmtError
@@ -553,84 +619,69 @@ MgmtRecordGet(const char *rec_name, TSRecordEle *rec_ele)
 }
 
 TSMgmtError
-MgmtConfigRecordDescribe(const char *rec_name, unsigned options, TSConfigRecordDescription *val)
+MgmtConfigRecordDescribeMatching(const char *rec_name, unsigned options, TSList rec_vals)
 {
   TSMgmtError ret;
   MgmtMarshallInt optype = RECORD_DESCRIBE_CONFIG;
-  MgmtMarshallInt flags = options;
+  MgmtMarshallInt flags = options | RECORD_DESCRIBE_FLAGS_MATCH;
   MgmtMarshallString record = const_cast<MgmtMarshallString>(rec_name);
 
-  MgmtMarshallData reply = {NULL, 0};
-
   // create and send request
   ret = MGMTAPI_SEND_MESSAGE(main_socket_fd, RECORD_DESCRIBE_CONFIG, &optype, &record, &flags);
   if (ret != TS_ERR_OKAY) {
     return ret;
   }
 
-  ret = recv_mgmt_message(main_socket_fd, reply);
-  if (ret != TS_ERR_OKAY) {
-    return ret;
-  } else {
-    MgmtMarshallInt err;
-    MgmtMarshallString name = NULL;
-    MgmtMarshallString expr = NULL;
-    MgmtMarshallData value = {NULL, 0};
-    MgmtMarshallData deflt = {NULL, 0};
-
-    MgmtMarshallInt rtype;
-    MgmtMarshallInt rclass;
-    MgmtMarshallInt version;
-    MgmtMarshallInt rsb;
-    MgmtMarshallInt order;
-    MgmtMarshallInt access;
-    MgmtMarshallInt update;
-    MgmtMarshallInt updatetype;
-    MgmtMarshallInt checktype;
-    MgmtMarshallInt source;
-
-    ret = recv_mgmt_response(reply.ptr, reply.len, RECORD_DESCRIBE_CONFIG, &err, &name, &value, &deflt, &rtype, &rclass, &version,
-                             &rsb, &order, &access, &update, &updatetype, &checktype, &source, &expr);
-
-    ats_free(reply.ptr);
+  for (;;) {
+    TSConfigRecordDescription *val;
+
+    val = TSConfigRecordDescriptionCreate();
 
+    // parse the reply to get record value and type
+    ret = mgmt_record_describe_reply(val);
     if (ret != TS_ERR_OKAY) {
-      goto done;
+      TSConfigRecordDescriptionDestroy(val);
+      goto fail;
     }
 
-    if (err != TS_ERR_OKAY) {
-      ret = (TSMgmtError)err;
-      goto done;
+    // A NULL record ends the list.
+    if (val->rec_type == TS_REC_UNDEFINED) {
+      TSConfigRecordDescriptionDestroy(val);
+      break;
     }
 
-    // Everything is cool, populate the description ...
-    val->rec_name = ats_strdup(name);
-    val->rec_checkexpr = ats_strdup(expr);
-    val->rec_type = (TSRecordT)rtype;
-    val->rec_class = rclass;
-    val->rec_version = version;
-    val->rec_rsb = rsb;
-    val->rec_order = order;
-    val->rec_access = access;
-    val->rec_updatetype = updatetype;
-    val->rec_checktype = checktype;
-    val->rec_source = source;
-
-    mgmt_record_convert_value(val->rec_type, value, val->rec_value);
-    mgmt_record_convert_value(val->rec_type, deflt, val->rec_default);
-
-  done:
-    ats_free(name);
-    ats_free(expr);
-    ats_free(value.ptr);
-    ats_free(deflt.ptr);
-    return ret;
+    enqueue((LLQ *)rec_vals, val);
+  }
+
+  return TS_ERR_OKAY;
+
+fail:
+  while (!queue_is_empty((LLQ *)rec_vals)) {
+    TSConfigRecordDescription *val = (TSConfigRecordDescription *)dequeue((LLQ *)rec_vals);
+    TSConfigRecordDescriptionDestroy(val);
   }
 
   return ret;
 }
 
 TSMgmtError
+MgmtConfigRecordDescribe(const char *rec_name, unsigned options, TSConfigRecordDescription *val)
+{
+  TSMgmtError ret;
+  MgmtMarshallInt optype = RECORD_DESCRIBE_CONFIG;
+  MgmtMarshallInt flags = options & ~RECORD_DESCRIBE_FLAGS_MATCH;
+  MgmtMarshallString record = const_cast<MgmtMarshallString>(rec_name);
+
+  // create and send request
+  ret = MGMTAPI_SEND_MESSAGE(main_socket_fd, RECORD_DESCRIBE_CONFIG, &optype, &record, &flags);
+  if (ret != TS_ERR_OKAY) {
+    return ret;
+  }
+
+  return mgmt_record_describe_reply(val);
+}
+
+TSMgmtError
 MgmtRecordGetMatching(const char *regex, TSList rec_vals)
 {
   TSMgmtError ret;
@@ -670,8 +721,8 @@ MgmtRecordGetMatching(const char *regex, TSList rec_vals)
   return TS_ERR_OKAY;
 
 fail:
-
-  for (rec_ele = (TSRecordEle *)dequeue((LLQ *)rec_vals); rec_ele; rec_ele = (TSRecordEle *)dequeue((LLQ *)rec_vals)) {
+  while (!queue_is_empty((LLQ *)rec_vals)) {
+    rec_ele = (TSRecordEle *)dequeue((LLQ *)rec_vals);
     TSRecordEleDestroy(rec_ele);
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/mgmt/api/INKMgmtAPI.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/INKMgmtAPI.cc b/mgmt/api/INKMgmtAPI.cc
index ad78811..f989b71 100644
--- a/mgmt/api/INKMgmtAPI.cc
+++ b/mgmt/api/INKMgmtAPI.cc
@@ -2358,6 +2358,24 @@ TSIsValid(TSCfgEle *ele)
   return (ele_obj->isValid());
 }
 
+TSConfigRecordDescription *
+TSConfigRecordDescriptionCreate(void)
+{
+  TSConfigRecordDescription *val = (TSConfigRecordDescription *)ats_malloc(sizeof(TSConfigRecordDescription));
+
+  ink_zero(*val);
+  val->rec_type = TS_REC_UNDEFINED;
+
+  return val;
+}
+
+void
+TSConfigRecordDescriptionDestroy(TSConfigRecordDescription *val)
+{
+  TSConfigRecordDescriptionFree(val);
+  ats_free(val);
+}
+
 void
 TSConfigRecordDescriptionFree(TSConfigRecordDescription *val)
 {
@@ -2384,3 +2402,13 @@ TSConfigRecordDescribe(const char *rec_name, unsigned flags, TSConfigRecordDescr
   TSConfigRecordDescriptionFree(val);
   return MgmtConfigRecordDescribe(rec_name, flags, val);
 }
+
+TSMgmtError
+TSConfigRecordDescribeMatchMlt(const char *rec_regex, unsigned flags, TSList rec_vals)
+{
+  if (!rec_regex || !rec_vals) {
+    return TS_ERR_PARAMS;
+  }
+
+  return MgmtConfigRecordDescribeMatching(rec_regex, flags, rec_vals);
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/mgmt/api/NetworkMessage.h
----------------------------------------------------------------------
diff --git a/mgmt/api/NetworkMessage.h b/mgmt/api/NetworkMessage.h
index 1aea548..88ef72d 100644
--- a/mgmt/api/NetworkMessage.h
+++ b/mgmt/api/NetworkMessage.h
@@ -65,6 +65,10 @@ typedef enum {
 
 #define MGMT_OPERATION_TYPE_MAX (UNDEFINED_OP)
 
+enum {
+  RECORD_DESCRIBE_FLAGS_MATCH = 0x0001,
+};
+
 struct mgmt_message_sender {
   virtual TSMgmtError send(void *msg, size_t msglen) const = 0;
   virtual ~mgmt_message_sender(){};

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/mgmt/api/TSControlMain.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/TSControlMain.cc b/mgmt/api/TSControlMain.cc
index 1f22a52..5c711cf 100644
--- a/mgmt/api/TSControlMain.cc
+++ b/mgmt/api/TSControlMain.cc
@@ -441,7 +441,7 @@ handle_record_match(int fd, void *req, size_t reqlen)
   match.err = TS_ERR_OKAY;
   match.fd = fd;
 
-  if (RecLookupMatchingRecords(name, send_record_match, &match) != REC_ERR_OKAY) {
+  if (RecLookupMatchingRecords(RECT_ALL, name, send_record_match, &match) != REC_ERR_OKAY) {
     ats_free(name);
     return TS_ERR_FAIL;
   }
@@ -1017,78 +1017,96 @@ handle_server_backtrace(int fd, void *req, size_t reqlen)
 }
 
 static void
-send_record_describe(const RecRecord *rec, void *ptr)
+send_record_describe(const RecRecord *rec, void *edata)
 {
-  MgmtMarshallString rec_name = const_cast<char *>(rec->name);
+  MgmtMarshallString rec_name = NULL;
   MgmtMarshallData rec_value = {NULL, 0};
   MgmtMarshallData rec_default = {NULL, 0};
-  MgmtMarshallInt rec_type = rec->data_type;
-  MgmtMarshallInt rec_class = rec->rec_type;
-  MgmtMarshallInt rec_version = rec->version;
-  MgmtMarshallInt rec_rsb = rec->rsb_id;
-  MgmtMarshallInt rec_order = rec->order;
-  MgmtMarshallInt rec_access = rec->config_meta.access_type;
-  MgmtMarshallInt rec_update = rec->config_meta.update_required;
-  MgmtMarshallInt rec_updatetype = rec->config_meta.update_type;
-  MgmtMarshallInt rec_checktype = rec->config_meta.check_type;
-  MgmtMarshallInt rec_source = rec->config_meta.source;
-  MgmtMarshallString rec_checkexpr = rec->config_meta.check_expr;
+  MgmtMarshallInt rec_type = TS_REC_UNDEFINED;
+  MgmtMarshallInt rec_class = RECT_NULL;
+  MgmtMarshallInt rec_version = 0;
+  MgmtMarshallInt rec_rsb = 0;
+  MgmtMarshallInt rec_order = 0;
+  MgmtMarshallInt rec_access = RECA_NULL;
+  MgmtMarshallInt rec_update = RECU_NULL;
+  MgmtMarshallInt rec_updatetype = 0;
+  MgmtMarshallInt rec_checktype = RECC_NULL;
+  MgmtMarshallInt rec_source = REC_SOURCE_NULL;
+  MgmtMarshallString rec_checkexpr = NULL;
+
+  TSMgmtError err = TS_ERR_OKAY;
 
-  MgmtMarshallInt err = TS_ERR_OKAY;
-
-  int *fderr = (int *)ptr;
+  record_match_state *match = (record_match_state *)edata;
 
-  // We only describe config variables (for now).
-  if (!REC_TYPE_IS_CONFIG(rec->rec_type)) {
-    *fderr = TS_ERR_PARAMS;
+  if (match->err != TS_ERR_OKAY) {
     return;
   }
 
-  switch (rec_type) {
-  case RECD_INT:
-    rec_type = TS_REC_INT;
-    break;
-  case RECD_FLOAT:
-    rec_type = TS_REC_FLOAT;
-    break;
-  case RECD_STRING:
-    rec_type = TS_REC_STRING;
-    break;
-  case RECD_COUNTER:
-    rec_type = TS_REC_COUNTER;
-    break;
-  default:
-    rec_type = TS_REC_UNDEFINED;
-  }
+  if (rec) {
+    // We only describe config variables (for now).
+    if (!REC_TYPE_IS_CONFIG(rec->rec_type)) {
+      match->err = TS_ERR_PARAMS;
+      return;
+    }
 
-  err = marshall_rec_data(rec->data_type, rec->data, rec_value);
-  if (err != TS_ERR_OKAY) {
-    goto done;
-  }
+    rec_name = const_cast<char *>(rec->name);
+    rec_type = rec->data_type;
+    rec_class = rec->rec_type;
+    rec_version = rec->version;
+    rec_rsb = rec->rsb_id;
+    rec_order = rec->order;
+    rec_access = rec->config_meta.access_type;
+    rec_update = rec->config_meta.update_required;
+    rec_updatetype = rec->config_meta.update_type;
+    rec_checktype = rec->config_meta.check_type;
+    rec_source = rec->config_meta.source;
+    rec_checkexpr = rec->config_meta.check_expr;
+
+    switch (rec_type) {
+    case RECD_INT:
+      rec_type = TS_REC_INT;
+      break;
+    case RECD_FLOAT:
+      rec_type = TS_REC_FLOAT;
+      break;
+    case RECD_STRING:
+      rec_type = TS_REC_STRING;
+      break;
+    case RECD_COUNTER:
+      rec_type = TS_REC_COUNTER;
+      break;
+    default:
+      rec_type = TS_REC_UNDEFINED;
+    }
 
-  err = marshall_rec_data(rec->data_type, rec->data_default, rec_default);
-  if (err != TS_ERR_OKAY) {
-    goto done;
+    err = marshall_rec_data(rec->data_type, rec->data, rec_value);
+    if (err != TS_ERR_OKAY) {
+      goto done;
+    }
+
+    err = marshall_rec_data(rec->data_type, rec->data_default, rec_default);
+    if (err != TS_ERR_OKAY) {
+      goto done;
+    }
   }
 
-  err = send_mgmt_response(*fderr, RECORD_DESCRIBE_CONFIG, &err, &rec_name, &rec_value, &rec_default, &rec_type, &rec_class,
+  err = send_mgmt_response(match->fd, RECORD_DESCRIBE_CONFIG, &err, &rec_name, &rec_value, &rec_default, &rec_type, &rec_class,
                            &rec_version, &rec_rsb, &rec_order, &rec_access, &rec_update, &rec_updatetype, &rec_checktype,
                            &rec_source, &rec_checkexpr);
 
 done:
-  *fderr = err;
+  match->err = err;
 }
 
 static TSMgmtError
 handle_record_describe(int fd, void *req, size_t reqlen)
 {
   TSMgmtError ret;
+  record_match_state match;
   MgmtMarshallInt optype;
   MgmtMarshallInt options;
   MgmtMarshallString name;
 
-  int fderr = fd; // [in,out] variable for the fd and error
-
   ret = recv_mgmt_request(req, reqlen, RECORD_DESCRIBE_CONFIG, &optype, &name, &options);
   if (ret != TS_ERR_OKAY) {
     return ret;
@@ -1099,19 +1117,34 @@ handle_record_describe(int fd, void *req, size_t reqlen)
     goto done;
   }
 
-  if (RecLookupRecord(name, send_record_describe, &fderr) != REC_ERR_OKAY) {
-    ret = TS_ERR_PARAMS;
-    goto done;
+  match.err = TS_ERR_OKAY;
+  match.fd = fd;
+
+  if (options & RECORD_DESCRIBE_FLAGS_MATCH) {
+    if (RecLookupMatchingRecords(RECT_CONFIG | RECT_LOCAL, name, send_record_describe, &match) != REC_ERR_OKAY) {
+      ret = TS_ERR_PARAMS;
+      goto done;
+    }
+
+    // If successful, send a list terminator.
+    if (match.err == TS_ERR_OKAY) {
+      send_record_describe(NULL, &match);
+    }
+
+  } else {
+    if (RecLookupRecord(name, send_record_describe, &match) != REC_ERR_OKAY) {
+      ret = TS_ERR_PARAMS;
+      goto done;
+    }
   }
 
-  // If the lookup succeeded, the final error is in "fderr".
   if (ret == TS_ERR_OKAY) {
-    ret = (TSMgmtError)fderr;
+    ret = match.err;
   }
 
 done:
   ats_free(name);
-  return ret;
+  return match.err;
 }
 
 struct control_message_handler {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2c19cd2b/mgmt/api/include/mgmtapi.h
----------------------------------------------------------------------
diff --git a/mgmt/api/include/mgmtapi.h b/mgmt/api/include/mgmtapi.h
index 65ff981..be20600 100644
--- a/mgmt/api/include/mgmtapi.h
+++ b/mgmt/api/include/mgmtapi.h
@@ -425,6 +425,11 @@ typedef struct {
 /* Free (the contents of) a TSConfigRecordDescription */
 tsapi void TSConfigRecordDescriptionFree(TSConfigRecordDescription *val);
 
+/* Heap-allocate a TSConfigRecordDescription. */
+tsapi TSConfigRecordDescription *TSConfigRecordDescriptionCreate(void);
+/* Free and destroy a heap-allocated TSConfigRecordDescription. */
+tsapi void TSConfigRecordDescriptionDestroy(TSConfigRecordDescription *);
+
 /*--- events --------------------------------------------------------------*/
 
 /* Note: Each event has a format String associated with it from which the
@@ -1108,6 +1113,7 @@ tsapi TSMgmtError TSRecordSetString(const char *rec_name, const char *string_val
  * Output: TSMgmtError
  */
 tsapi TSMgmtError TSConfigRecordDescribe(const char *rec_name, unsigned flags, TSConfigRecordDescription *val);
+tsapi TSMgmtError TSConfigRecordDescribeMatchMlt(const char *rec_regex, unsigned flags, TSList list);
 
 /* TSRecordSetMlt: sets a set of records
  * Input:  rec_list     - list of record names the user wants to set;


[3/3] trafficserver git commit: TS-3815: traffic_ctl should separate config and metric records

Posted by jp...@apache.org.
TS-3815: traffic_ctl should separate config and metric records

Send the record class in response to management record queries.
This lets us filter the response depending on whether the caller
wanted configuration records or statistics records. We can also
format configuration records correctly depending on whether they
are local or not.


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

Branch: refs/heads/master
Commit: 244b78e56576c86e1d511314ed7db4c27c42249a
Parents: 080bcaa
Author: James Peach <jp...@apache.org>
Authored: Fri Jul 31 15:41:12 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Thu Aug 27 20:53:54 2015 -0700

----------------------------------------------------------------------
 cmd/traffic_ctl/config.cc             |  25 ++++-
 cmd/traffic_ctl/metric.cc             |  11 ++-
 cmd/traffic_ctl/traffic_ctl.cc        |   6 ++
 cmd/traffic_ctl/traffic_ctl.h         |   1 +
 lib/perl/lib/Apache/TS/AdminClient.pm |   3 +-
 lib/records/I_RecCore.h               |   6 +-
 lib/records/RecCore.cc                |  24 +++++
 mgmt/api/CoreAPIRemote.cc             |   4 +-
 mgmt/api/NetworkMessage.cc            |  12 ++-
 mgmt/api/TSControlMain.cc             | 143 +++++++++++++++--------------
 mgmt/api/include/mgmtapi.h            |   1 +
 11 files changed, 151 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/cmd/traffic_ctl/config.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/config.cc b/cmd/traffic_ctl/config.cc
index 9efda2a..44b6e82 100644
--- a/cmd/traffic_ctl/config.cc
+++ b/cmd/traffic_ctl/config.cc
@@ -24,6 +24,7 @@
 #include "traffic_ctl.h"
 #include <time.h>
 #include <I_RecDefs.h>
+#include <P_RecUtils.h>
 
 // Record data type names, indexed by TSRecordT.
 static const char *
@@ -145,8 +146,20 @@ format_record(const CtrlMgmtRecord &record, bool recfmt)
   CtrlMgmtRecordValue value(record);
 
   if (recfmt) {
-    // XXX Detect CONFIG or LOCAL ...
-    printf("CONFIG %s %s %s\n", record.name(), rec_typeof(record.type()), value.c_str());
+    const char *label;
+
+    switch (record.rclass()) {
+    case RECT_CONFIG:
+      label = "CONFIG ";
+      break;
+    case RECT_LOCAL:
+      label = "LOCAL ";
+      break;
+    default:
+      label = "";
+    }
+
+    printf("%s%s %s %s\n", label, record.name(), rec_typeof(record.type()), value.c_str());
   } else {
     printf("%s: %s\n", record.name(), value.c_str());
   }
@@ -174,7 +187,9 @@ config_get(unsigned argc, const char **argv)
       return CTRL_EX_ERROR;
     }
 
-    format_record(record, recfmt);
+    if (REC_TYPE_IS_CONFIG(record.rclass())) {
+      format_record(record, recfmt);
+    }
   }
 
   return CTRL_EX_OK;
@@ -285,7 +300,9 @@ config_match(unsigned argc, const char **argv)
 
     while (!reclist.empty()) {
       CtrlMgmtRecord record(reclist.next());
-      format_record(record, recfmt);
+      if (REC_TYPE_IS_CONFIG(record.rclass())) {
+        format_record(record, recfmt);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/cmd/traffic_ctl/metric.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/metric.cc b/cmd/traffic_ctl/metric.cc
index 36a8b6e..26d70d1 100644
--- a/cmd/traffic_ctl/metric.cc
+++ b/cmd/traffic_ctl/metric.cc
@@ -22,6 +22,7 @@
  */
 
 #include "traffic_ctl.h"
+#include <P_RecUtils.h>
 
 static int
 metric_get(unsigned argc, const char **argv)
@@ -40,7 +41,9 @@ metric_get(unsigned argc, const char **argv)
       return CTRL_EX_ERROR;
     }
 
-    printf("%s %s\n", record.name(), CtrlMgmtRecordValue(record).c_str());
+    if (REC_TYPE_IS_STAT(record.rclass())) {
+      printf("%s %s\n", record.name(), CtrlMgmtRecordValue(record).c_str());
+    }
   }
 
   return CTRL_EX_OK;
@@ -57,8 +60,6 @@ metric_match(unsigned argc, const char **argv)
     CtrlMgmtRecordList reclist;
     TSMgmtError error;
 
-    // XXX filter the results to only match metric records.
-
     error = reclist.match(file_arguments[i]);
     if (error != TS_ERR_OKAY) {
       CtrlMgmtError(error, "failed to fetch %s", file_arguments[i]);
@@ -67,7 +68,9 @@ metric_match(unsigned argc, const char **argv)
 
     while (!reclist.empty()) {
       CtrlMgmtRecord record(reclist.next());
-      printf("%s %s\n", record.name(), CtrlMgmtRecordValue(record).c_str());
+      if (REC_TYPE_IS_STAT(record.rclass())) {
+        printf("%s %s\n", record.name(), CtrlMgmtRecordValue(record).c_str());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/cmd/traffic_ctl/traffic_ctl.cc
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/traffic_ctl.cc b/cmd/traffic_ctl/traffic_ctl.cc
index aff4a42..c1e11f2 100644
--- a/cmd/traffic_ctl/traffic_ctl.cc
+++ b/cmd/traffic_ctl/traffic_ctl.cc
@@ -37,6 +37,12 @@ CtrlMgmtRecord::type() const
   return this->ele->rec_type;
 }
 
+int
+CtrlMgmtRecord::rclass() const
+{
+  return this->ele->rec_class;
+}
+
 int64_t
 CtrlMgmtRecord::as_int() const
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/cmd/traffic_ctl/traffic_ctl.h
----------------------------------------------------------------------
diff --git a/cmd/traffic_ctl/traffic_ctl.h b/cmd/traffic_ctl/traffic_ctl.h
index aa1a51e..bc0847a 100644
--- a/cmd/traffic_ctl/traffic_ctl.h
+++ b/cmd/traffic_ctl/traffic_ctl.h
@@ -79,6 +79,7 @@ struct CtrlMgmtRecord {
 
   const char *name() const;
   TSRecordT type() const;
+  int rclass() const;
   int64_t as_int() const;
 
 private:

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/lib/perl/lib/Apache/TS/AdminClient.pm
----------------------------------------------------------------------
diff --git a/lib/perl/lib/Apache/TS/AdminClient.pm b/lib/perl/lib/Apache/TS/AdminClient.pm
index 9159e3e..e378303 100644
--- a/lib/perl/lib/Apache/TS/AdminClient.pm
+++ b/lib/perl/lib/Apache/TS/AdminClient.pm
@@ -214,11 +214,12 @@ sub get_stat {
     # The response format is:
     #   MGMT_MARSHALL_INT: message length
     #   MGMT_MARSHALL_INT: error code
+    #   MGMT_MARSHALL_INT: record class
     #   MGMT_MARSHALL_INT: record type
     #   MGMT_MARSHALL_STRING: record name
     #   MGMT_MARSHALL_DATA: record data
     ($msg) = unpack("l/a", $res);
-    my ($ecode, $type, $name, $value) = unpack("l l l/Z l/a", $msg);
+    my ($ecode, $rclass, $type, $name, $value) = unpack("l l l l/Z l/a", $msg);
 
     if ($ecode == TS_ERR_OKAY) {
         if ($type == TS_REC_INT || $type == TS_REC_COUNTER) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/lib/records/I_RecCore.h
----------------------------------------------------------------------
diff --git a/lib/records/I_RecCore.h b/lib/records/I_RecCore.h
index 719021f..1d8ab52 100644
--- a/lib/records/I_RecCore.h
+++ b/lib/records/I_RecCore.h
@@ -166,7 +166,10 @@ int RecGetRecordBool(const char *name, RecBool *rec_byte, bool lock = true);
 // Record Attributes Reading
 //------------------------------------------------------------------------
 
-int RecLookupRecord(const char *name, void (*callback)(const RecRecord *, void *), void *data, bool lock = true);
+typedef void (*RecLookupCallback)(const RecRecord *, void *);
+
+int RecLookupRecord(const char *name, RecLookupCallback callback, void *data, bool lock = true);
+int RecLookupMatchingRecords(const char *match, RecLookupCallback callback, void *data, bool lock = true);
 
 int RecGetRecordType(const char *name, RecT *rec_type, bool lock = true);
 int RecGetRecordDataType(const char *name, RecDataT *data_type, bool lock = true);
@@ -184,7 +187,6 @@ int RecSetRecordAccessType(const char *name, RecAccessT secure, bool lock = true
 
 int RecGetRecordPrefix_Xmalloc(char *prefix, char **result, int *result_len);
 
-
 //------------------------------------------------------------------------
 // Signal and Alarms
 //------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/lib/records/RecCore.cc
----------------------------------------------------------------------
diff --git a/lib/records/RecCore.cc b/lib/records/RecCore.cc
index 6ad0db8..041a218 100644
--- a/lib/records/RecCore.cc
+++ b/lib/records/RecCore.cc
@@ -455,6 +455,29 @@ RecLookupRecord(const char *name, void (*callback)(const RecRecord *, void *), v
 }
 
 int
+RecLookupMatchingRecords(const char *match, void (*callback)(const RecRecord *, void *), void *data, bool lock)
+{
+  int num_records;
+  DFA regex;
+
+  if (regex.compile(match, RE_CASE_INSENSITIVE | RE_UNANCHORED) != 0) {
+    return REC_ERR_FAIL;
+  }
+
+  num_records = g_num_records;
+  for (int i = 0; i < num_records; i++) {
+    RecRecord *r = &(g_records[i]);
+    if (regex.match(r->name) >= 0) {
+      rec_mutex_acquire(&(r->lock));
+      callback(r, data);
+      rec_mutex_release(&(r->lock));
+    }
+  }
+
+  return REC_ERR_OKAY;
+}
+
+int
 RecGetRecordType(const char *name, RecT *rec_type, bool lock)
 {
   int err = REC_ERR_FAIL;
@@ -928,6 +951,7 @@ debug_record_callback(RecT /* rec_type */, void * /* edata */, int registered, c
     break;
   }
 }
+
 void
 RecDumpRecords(RecT rec_type, RecDumpEntryCb callback, void *edata)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/mgmt/api/CoreAPIRemote.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/CoreAPIRemote.cc b/mgmt/api/CoreAPIRemote.cc
index 2626880..4ce8cae 100644
--- a/mgmt/api/CoreAPIRemote.cc
+++ b/mgmt/api/CoreAPIRemote.cc
@@ -498,6 +498,7 @@ mgmt_record_get_reply(OpType op, TSRecordEle *rec_ele)
 
   MgmtMarshallData reply = {NULL, 0};
   MgmtMarshallInt err;
+  MgmtMarshallInt rclass;
   MgmtMarshallInt type;
   MgmtMarshallString name = NULL;
   MgmtMarshallData value = {NULL, 0};
@@ -511,7 +512,7 @@ mgmt_record_get_reply(OpType op, TSRecordEle *rec_ele)
     return ret;
   }
 
-  ret = recv_mgmt_response(reply.ptr, reply.len, op, &err, &type, &name, &value);
+  ret = recv_mgmt_response(reply.ptr, reply.len, op, &err, &rclass, &type, &name, &value);
   ats_free(reply.ptr);
   if (ret != TS_ERR_OKAY) {
     goto done;
@@ -522,6 +523,7 @@ mgmt_record_get_reply(OpType op, TSRecordEle *rec_ele)
     goto done;
   }
 
+  rec_ele->rec_class = (TSInt)rclass;
   rec_ele->rec_type = (TSRecordT)type;
   rec_ele->rec_name = ats_strdup(name);
   mgmt_record_convert_value(rec_ele->rec_type, value, rec_ele->valueT);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/mgmt/api/NetworkMessage.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/NetworkMessage.cc b/mgmt/api/NetworkMessage.cc
index 9fb5e21..c4b96ca 100644
--- a/mgmt/api/NetworkMessage.cc
+++ b/mgmt/api/NetworkMessage.cc
@@ -74,7 +74,9 @@ static const struct NetCmdOperation responses[] = {
   /* FILE_READ                  */ {3, {MGMT_MARSHALL_INT, MGMT_MARSHALL_INT, MGMT_MARSHALL_DATA}},
   /* FILE_WRITE                 */ {1, {MGMT_MARSHALL_INT}},
   /* RECORD_SET                 */ {2, {MGMT_MARSHALL_INT, MGMT_MARSHALL_INT}},
-  /* RECORD_GET                 */ {4, {MGMT_MARSHALL_INT, MGMT_MARSHALL_INT, MGMT_MARSHALL_STRING, MGMT_MARSHALL_DATA}},
+  /* RECORD_GET                 */ {5,
+                                    {MGMT_MARSHALL_INT, MGMT_MARSHALL_INT, MGMT_MARSHALL_INT, MGMT_MARSHALL_STRING,
+                                     MGMT_MARSHALL_DATA}},
   /* PROXY_STATE_GET            */ {2, {MGMT_MARSHALL_INT, MGMT_MARSHALL_INT}},
   /* PROXY_STATE_SET            */ {1, {MGMT_MARSHALL_INT}},
   /* RECONFIGURE                */ {1, {MGMT_MARSHALL_INT}},
@@ -94,7 +96,9 @@ static const struct NetCmdOperation responses[] = {
   /* STATS_RESET_NODE           */ {1, {MGMT_MARSHALL_INT}},
   /* STATS_RESET_CLUSTER        */ {1, {MGMT_MARSHALL_INT}},
   /* STORAGE_DEVICE_CMD_OFFLINE */ {1, {MGMT_MARSHALL_INT}},
-  /* RECORD_MATCH_GET           */ {4, {MGMT_MARSHALL_INT, MGMT_MARSHALL_INT, MGMT_MARSHALL_STRING, MGMT_MARSHALL_DATA}},
+  /* RECORD_MATCH_GET           */ {5,
+                                    {MGMT_MARSHALL_INT, MGMT_MARSHALL_INT, MGMT_MARSHALL_INT, MGMT_MARSHALL_STRING,
+                                     MGMT_MARSHALL_DATA}},
   /* API_PING                   */ {0, {}}, // no reply
   /* SERVER_BACKTRACE           */ {2, {MGMT_MARSHALL_INT, MGMT_MARSHALL_STRING}},
   /* RECORD_DESCRIBE_CONFIG     */ {15,
@@ -232,8 +236,8 @@ send_mgmt_error(int fd, OpType optype, TSMgmtError error)
 
   case RECORD_GET:
   case RECORD_MATCH_GET:
-    ink_release_assert(responses[optype].nfields == 4);
-    return send_mgmt_response(fd, optype, &ecode, &intval, &strval, &dataval);
+    ink_release_assert(responses[optype].nfields == 5);
+    return send_mgmt_response(fd, optype, &ecode, &intval, &intval, &strval, &dataval);
 
   case RECORD_DESCRIBE_CONFIG:
     ink_release_assert(responses[optype].nfields == 15);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/mgmt/api/TSControlMain.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/TSControlMain.cc b/mgmt/api/TSControlMain.cc
index 0673b29..1f22a52 100644
--- a/mgmt/api/TSControlMain.cc
+++ b/mgmt/api/TSControlMain.cc
@@ -297,14 +297,58 @@ marshall_rec_data(RecDataT rec_type, const RecData &rec_data, MgmtMarshallData &
 }
 
 static TSMgmtError
-send_record_get_response(int fd, TSRecordT rec_type, const char *rec_name, const void *rec_data, size_t data_len)
+send_record_get_response(int fd, const RecRecord *rec)
 {
   MgmtMarshallInt err = TS_ERR_OKAY;
-  MgmtMarshallInt type = rec_type;
-  MgmtMarshallString name = const_cast<MgmtMarshallString>(rec_name);
-  MgmtMarshallData value = {const_cast<void *>(rec_data), data_len};
+  MgmtMarshallInt type;
+  MgmtMarshallInt rclass;
+  MgmtMarshallString name;
+  MgmtMarshallData value = {NULL, 0};
+
+  if (rec) {
+    type = rec->data_type;
+    rclass = rec->rec_type;
+    name = const_cast<MgmtMarshallString>(rec->name);
+  } else {
+    type = RECD_NULL;
+    rclass = RECT_NULL;
+    name = NULL;
+  }
+
+  switch (type) {
+  case RECD_INT:
+    type = TS_REC_INT;
+    value.ptr = (void *)&rec->data.rec_int;
+    value.len = sizeof(RecInt);
+    break;
+  case RECD_COUNTER:
+    type = TS_REC_COUNTER;
+    value.ptr = (void *)&rec->data.rec_counter;
+    value.len = sizeof(RecCounter);
+    break;
+  case RECD_FLOAT:
+    type = TS_REC_FLOAT;
+    value.ptr = (void *)&rec->data.rec_float;
+    value.len = sizeof(RecFloat);
+    break;
+  case RECD_STRING:
+    // For NULL string parameters, send the literal "NULL" to match the behavior of MgmtRecordGet(). Make sure to send
+    // the trailing NULL.
+    type = TS_REC_STRING;
+    if (rec->data.rec_string) {
+      value.ptr = rec->data.rec_string;
+      value.len = strlen(rec->data.rec_string) + 1;
+    } else {
+      value.ptr = const_cast<char *>("NULL");
+      value.len = countof("NULL");
+    }
+    break;
+  default:
+    type = TS_REC_UNDEFINED;
+    break; // skip it
+  }
 
-  return send_mgmt_response(fd, RECORD_GET, &err, &type, &name, &value);
+  return send_mgmt_response(fd, RECORD_GET, &err, &rclass, &type, &name, &value);
 }
 
 /**************************************************************************
@@ -317,69 +361,55 @@ send_record_get_response(int fd, TSRecordT rec_type, const char *rec_name, const
  * output: SUCC or ERR
  * note:
  *************************************************************************/
+static void
+send_record_get(const RecRecord *rec, void *edata)
+{
+  int *fd = (int *)edata;
+  *fd = send_record_get_response(*fd, rec);
+}
+
 static TSMgmtError
 handle_record_get(int fd, void *req, size_t reqlen)
 {
   TSMgmtError ret;
-  TSRecordEle *ele;
   MgmtMarshallInt optype;
   MgmtMarshallString name;
 
+  int fderr = fd; // [in,out] variable for the fd and error
+
   ret = recv_mgmt_request(req, reqlen, RECORD_GET, &optype, &name);
   if (ret != TS_ERR_OKAY) {
     return ret;
   }
 
   if (strlen(name) == 0) {
-    ats_free(name);
-    return ret;
+    ret = TS_ERR_PARAMS;
+    goto done;
   }
 
-  // call CoreAPI call on Traffic Manager side
-  ele = TSRecordEleCreate();
-  ret = MgmtRecordGet(name, ele);
-  ats_free(name);
-
-  if (ret != TS_ERR_OKAY) {
+  fderr = fd;
+  if (RecLookupRecord(name, send_record_get, &fderr) != REC_ERR_OKAY) {
+    ret = TS_ERR_PARAMS;
     goto done;
   }
 
-  // create and send reply back to client
-  switch (ele->rec_type) {
-  case TS_REC_INT:
-    ret = send_record_get_response(fd, ele->rec_type, ele->rec_name, &(ele->valueT.int_val), sizeof(TSInt));
-    break;
-  case TS_REC_COUNTER:
-    ret = send_record_get_response(fd, ele->rec_type, ele->rec_name, &(ele->valueT.counter_val), sizeof(TSCounter));
-    break;
-  case TS_REC_FLOAT:
-    ret = send_record_get_response(fd, ele->rec_type, ele->rec_name, &(ele->valueT.float_val), sizeof(TSFloat));
-    break;
-  case TS_REC_STRING:
-    // Make sure to send the NULL in the string value response.
-    if (ele->valueT.string_val) {
-      ret = send_record_get_response(fd, ele->rec_type, ele->rec_name, ele->valueT.string_val, strlen(ele->valueT.string_val) + 1);
-    } else {
-      ret = send_record_get_response(fd, ele->rec_type, ele->rec_name, "NULL", countof("NULL"));
-    }
-    break;
-  default: // invalid record type
-    ret = TS_ERR_FAIL;
+  // If the lookup succeeded, the final error is in "fderr".
+  if (ret == TS_ERR_OKAY) {
+    ret = (TSMgmtError)fderr;
   }
 
 done:
-  TSRecordEleDestroy(ele);
+  ats_free(name);
   return ret;
 }
 
 struct record_match_state {
   TSMgmtError err;
   int fd;
-  DFA regex;
 };
 
 static void
-send_record_match(RecT /* rec_type */, void *edata, int /* registered */, const char *name, int data_type, RecData *rec_val)
+send_record_match(const RecRecord *rec, void *edata)
 {
   record_match_state *match = (record_match_state *)edata;
 
@@ -387,30 +417,7 @@ send_record_match(RecT /* rec_type */, void *edata, int /* registered */, const
     return;
   }
 
-  if (match->regex.match(name) >= 0) {
-    switch (data_type) {
-    case RECD_INT:
-      match->err = send_record_get_response(match->fd, TS_REC_INT, name, &(rec_val->rec_int), sizeof(TSInt));
-      break;
-    case RECD_COUNTER:
-      match->err = send_record_get_response(match->fd, TS_REC_COUNTER, name, &(rec_val->rec_counter), sizeof(TSCounter));
-      break;
-    case RECD_STRING:
-      // For NULL string parameters, end the literal "NULL" to match the behavior of MgmtRecordGet(). Make sure to send
-      // the trailing NULL.
-      if (rec_val->rec_string) {
-        match->err = send_record_get_response(match->fd, TS_REC_STRING, name, rec_val->rec_string, strlen(rec_val->rec_string) + 1);
-      } else {
-        match->err = send_record_get_response(match->fd, TS_REC_STRING, name, "NULL", countof("NULL"));
-      }
-      break;
-    case RECD_FLOAT:
-      match->err = send_record_get_response(match->fd, TS_REC_FLOAT, name, &(rec_val->rec_float), sizeof(TSFloat));
-      break;
-    default:
-      break; // skip it
-    }
-  }
+  match->err = send_record_get_response(match->fd, rec);
 }
 
 static TSMgmtError
@@ -431,21 +438,19 @@ handle_record_match(int fd, void *req, size_t reqlen)
     return TS_ERR_FAIL;
   }
 
-  if (match.regex.compile(name, RE_CASE_INSENSITIVE | RE_UNANCHORED) != 0) {
+  match.err = TS_ERR_OKAY;
+  match.fd = fd;
+
+  if (RecLookupMatchingRecords(name, send_record_match, &match) != REC_ERR_OKAY) {
     ats_free(name);
     return TS_ERR_FAIL;
   }
 
   ats_free(name);
 
-  match.err = TS_ERR_OKAY;
-  match.fd = fd;
-
-  RecDumpRecords(RECT_NULL, send_record_match, &match);
-
   // If successful, send a list terminator.
   if (match.err == TS_ERR_OKAY) {
-    return send_record_get_response(fd, TS_REC_UNDEFINED, NULL, NULL, 0);
+    return send_record_get_response(fd, NULL);
   }
 
   return match.err;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/244b78e5/mgmt/api/include/mgmtapi.h
----------------------------------------------------------------------
diff --git a/mgmt/api/include/mgmtapi.h b/mgmt/api/include/mgmtapi.h
index aba1293..65ff981 100644
--- a/mgmt/api/include/mgmtapi.h
+++ b/mgmt/api/include/mgmtapi.h
@@ -397,6 +397,7 @@ typedef union {/* record value */
 
 typedef struct {
   char *rec_name;        /* record name */
+  TSInt rec_class;       /* record class (RecT) */
   TSRecordT rec_type;    /* record type {TS_REC_INT...} */
   TSRecordValueT valueT; /* record value */
 } TSRecordEle;