You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by el...@apache.org on 2017/01/17 20:05:08 UTC

[1/3] incubator-trafficcontrol git commit: Add TM2 CacheStats systems stats

Repository: incubator-trafficcontrol
Updated Branches:
  refs/heads/master 24802411a -> 8833d3e2a


Add TM2 CacheStats systems stats


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

Branch: refs/heads/master
Commit: ce71d1d8cfff9f86b0bfc8917ce1b1b859fd629f
Parents: 578d33b
Author: Robert Butts <ro...@gmail.com>
Authored: Fri Jan 13 13:32:05 2017 -0700
Committer: Jeff Elsloo <je...@cable.comcast.com>
Committed: Tue Jan 17 13:04:16 2017 -0700

----------------------------------------------------------------------
 .../experimental/traffic_monitor/cache/cache.go | 155 ++++++++++++++++++-
 .../experimental/traffic_monitor/cache/data.go  |  37 +++--
 .../traffic_monitor/manager/datarequest.go      |   6 +-
 .../traffic_monitor/manager/stathistory.go      |  10 +-
 4 files changed, 182 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/ce71d1d8/traffic_monitor/experimental/traffic_monitor/cache/cache.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/cache/cache.go b/traffic_monitor/experimental/traffic_monitor/cache/cache.go
index c8528e8..b590631 100644
--- a/traffic_monitor/experimental/traffic_monitor/cache/cache.go
+++ b/traffic_monitor/experimental/traffic_monitor/cache/cache.go
@@ -22,18 +22,20 @@ package cache
 import (
 	"encoding/json"
 	"fmt"
-	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/common/log"
-	dsdata "github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/deliveryservicedata"
-	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/enum"
-	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/peer"
-	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/srvhttp"
-	todata "github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/trafficopsdata"
 	"io"
 	"net/url"
 	"regexp"
 	"strconv"
 	"strings"
 	"time"
+
+	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/common/log"
+	dsdata "github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/deliveryservicedata"
+	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/enum"
+	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/peer"
+	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/srvhttp"
+	todata "github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/trafficopsdata"
+	to "github.com/apache/incubator-trafficcontrol/traffic_ops/client"
 )
 
 // Handler is a cache handler, which fulfills the common/handler `Handler` interface.
@@ -113,8 +115,10 @@ type Filter interface {
 	WithinStatHistoryMax(int) bool
 }
 
+const nsPerMs = 1000000
+
 // StatsMarshall encodes the stats in JSON, encoding up to historyCount of each stat. If statsToUse is empty, all stats are encoded; otherwise, only the given stats are encoded. If wildcard is true, stats which contain the text in each statsToUse are returned, instead of exact stat names. If cacheType is not CacheTypeInvalid, only stats for the given type are returned. If hosts is not empty, only the given hosts are returned.
-func StatsMarshall(statResultHistory ResultStatHistory, filter Filter, params url.Values) ([]byte, error) {
+func StatsMarshall(statResultHistory ResultStatHistory, statInfo ResultInfoHistory, combinedStates peer.Crstates, monitorConfig to.TrafficMonitorConfigMap, statMaxKbpses Kbpses, filter Filter, params url.Values) ([]byte, error) {
 	stats := Stats{
 		CommonAPIData: srvhttp.GetCommonAPIData(params, time.Now()),
 		Caches:        map[enum.CacheName]map[string][]ResultStatVal{},
@@ -145,9 +149,146 @@ func StatsMarshall(statResultHistory ResultStatHistory, filter Filter, params ur
 		}
 	}
 
+	for id, infos := range statInfo {
+		if !filter.UseCache(id) {
+			continue
+		}
+		for i, info := range infos {
+			if !filter.WithinStatHistoryMax(i + 1) {
+				break
+			}
+			if _, ok := stats.Caches[id]; !ok {
+				stats.Caches[id] = map[string][]ResultStatVal{}
+			}
+
+			t := info.Time
+
+			if stat := "availableBandwidthInKbps"; filter.UseStat(stat) {
+				v := info.Vitals.MaxKbpsOut - info.Vitals.KbpsOut
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "availableBandwidthInMbps"; filter.UseStat(stat) {
+				v := (info.Vitals.MaxKbpsOut - info.Vitals.KbpsOut) / 1000
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "bandwidth"; filter.UseStat(stat) {
+				v := info.Vitals
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "error-string"; filter.UseStat(stat) {
+				v := ""
+				if info.Error != nil {
+					v = info.Error.Error()
+				} else {
+					v = "false"
+				}
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "isAvailable"; filter.UseStat(stat) {
+				v := combinedStates.Caches[id].IsAvailable // if the cache is missing, default to false
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "isHealthy"; filter.UseStat(stat) {
+				adminDown := false
+				if srv, ok := monitorConfig.TrafficServer[string(id)]; ok && enum.CacheStatusFromString(srv.Status) == enum.CacheStatusAdminDown {
+					adminDown = true
+				}
+				v := !adminDown && combinedStates.Caches[id].IsAvailable // if the cache is missing, default to false
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "kbps"; filter.UseStat(stat) {
+				v := info.Vitals.KbpsOut
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "loadAvg"; filter.UseStat(stat) {
+				v := info.Vitals.LoadAvg
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "maxKbps"; filter.UseStat(stat) {
+				v := info.Vitals.MaxKbpsOut
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "queryTime"; filter.UseStat(stat) {
+				v := fmt.Sprintf("%d", info.RequestTime.Nanoseconds()/nsPerMs)
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "stateUrl"; filter.UseStat(stat) {
+				v, err := getHealthPollingURL(id, monitorConfig)
+				if err != nil {
+					v = fmt.Sprintf("ERROR: %v", err) // should never happen
+				}
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "status"; filter.UseStat(stat) {
+				v := ""
+				srv, ok := monitorConfig.TrafficServer[string(id)]
+				if !ok {
+					v = fmt.Sprintf("ERROR: cache not found in monitor config") // should never happen
+				}
+				v = srv.Status
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.astatsLoad"; filter.UseStat(stat) {
+				v := info.System.AstatsLoad
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.configReloadRequests"; filter.UseStat(stat) {
+				v := info.System.ConfigLoadRequest
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.configReloads"; filter.UseStat(stat) {
+				v := info.System.ConfigReloads
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.inf.name"; filter.UseStat(stat) {
+				v := info.System.InfName
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.inf.speed"; filter.UseStat(stat) {
+				v := info.System.InfSpeed
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.lastReload"; filter.UseStat(stat) {
+				v := info.System.LastReload
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.lastReloadRequest"; filter.UseStat(stat) {
+				v := info.System.LastReloadRequest
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.proc.loadavg"; filter.UseStat(stat) {
+				v := info.System.ProcLoadavg
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+			if stat := "system.proc.net.dev"; filter.UseStat(stat) {
+				v := info.System.ProcNetDev
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: v, Time: t, Span: 1})
+			}
+		}
+	}
 	return json.Marshal(stats)
 }
 
+func getHealthPollingURL(cache enum.CacheName, monitorConfig to.TrafficMonitorConfigMap) (string, error) {
+	srv, ok := monitorConfig.TrafficServer[string(cache)]
+	if !ok {
+		return "", fmt.Errorf("server not found in monitor config from Traffic Ops")
+	}
+	url := monitorConfig.Profile[srv.Profile].Parameters.HealthPollingURL
+	if url == "" {
+		return "", fmt.Errorf("health polling URL not found in monitor config from Traffic Ops")
+	}
+
+	// TODO abstract replacer, remove duplication with manager/monitorconfig.go
+	url = strings.NewReplacer(
+		"${hostname}", srv.IP,
+		"${interface_name}", srv.InterfaceName,
+		"application=system", "application=plugin.remap",
+		"application=", "application=plugin.remap",
+	).Replace(url)
+	return url, nil
+}
+
 // Handle handles results fetched from a cache, parsing the raw Reader data and passing it along to a chan for further processing.
 func (handler Handler) Handle(id string, r io.Reader, reqTime time.Duration, reqErr error, pollID uint64, pollFinished chan<- uint64) {
 	log.Debugf("poll %v %v handle start\n", pollID, time.Now())

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/ce71d1d8/traffic_monitor/experimental/traffic_monitor/cache/data.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/cache/data.go b/traffic_monitor/experimental/traffic_monitor/cache/data.go
index b7871a5..e22e174 100644
--- a/traffic_monitor/experimental/traffic_monitor/cache/data.go
+++ b/traffic_monitor/experimental/traffic_monitor/cache/data.go
@@ -20,6 +20,7 @@ package cache
  */
 
 import (
+	"encoding/json"
 	"fmt"
 	"time"
 
@@ -87,19 +88,22 @@ type ResultStatValHistory map[string][]ResultStatVal
 // Span is the number of polls this stat has been the same. For example, if History is set to 100, and the last 50 polls had the same value for this stat (but none of the previous 50 were the same), this stat's map value slice will actually contain 51 entries, and the first entry will have the value, the time of the last poll, and a Span of 50. Assuming the poll time is every 8 seconds, users will then know, looking at the Span, that the value was unchanged for the last 50*8=400 seconds.
 // JSON values are all strings, for the TM1.0 /publish/CacheStats API.
 type ResultStatVal struct {
-	Val  interface{} `json:"value,string"`
-	Time TM1Time     `json:"time,string"`
-	Span uint64      `json:"span,string"`
-}
-
-// TM1Time provides a custom MarshalJSON func to serialise a time.Time into milliseconds since the epoch, as served in Traffic Monitor 1.x APIs
-// TODO move somewhere more generic (enum?)
-type TM1Time time.Time
-
-func (t *TM1Time) MarshalJSON() ([]byte, error) {
-	NanosecondsPerMillisecond := int64(1000000)
-	it := (*time.Time)(t).UnixNano() / NanosecondsPerMillisecond
-	return []byte(fmt.Sprintf("%d", it)), nil
+	Val  interface{} `json:"value"`
+	Time time.Time   `json:"time"`
+	Span uint64      `json:"span"`
+}
+
+func (t *ResultStatVal) MarshalJSON() ([]byte, error) {
+	v := struct {
+		Val  string `json:"value"`
+		Time int64  `json:"time"`
+		Span uint64 `json:"span"`
+	}{
+		Val:  fmt.Sprintf("%v", t.Val),
+		Time: t.Time.UnixNano() / 1000000, // ms since the epoch
+		Span: t.Span,
+	}
+	return json.Marshal(&v)
 }
 
 func copyResultStatVals(a []ResultStatVal) []ResultStatVal {
@@ -136,12 +140,12 @@ func (a ResultStatHistory) Add(r Result, limit uint64) {
 		statHistory := a[r.ID][statName]
 		// If the new stat value is the same as the last, update the time and increment the span. Span is the number of polls the latest value has been the same, and hence the length of time it's been the same is span*pollInterval.
 		if len(statHistory) > 0 && statHistory[0].Val == statVal {
-			statHistory[0].Time = TM1Time(r.Time)
+			statHistory[0].Time = r.Time
 			statHistory[0].Span++
 		} else {
 			resultVal := ResultStatVal{
 				Val:  statVal,
-				Time: TM1Time(r.Time),
+				Time: r.Time,
 				Span: 1,
 			}
 			statHistory = pruneStats(append([]ResultStatVal{resultVal}, statHistory...), limit)
@@ -153,6 +157,7 @@ func (a ResultStatHistory) Add(r Result, limit uint64) {
 	}
 }
 
+// TODO determine if anything ever needs more than the latest, and if not, change ResultInfo to not be a slice.
 type ResultInfoHistory map[enum.CacheName][]ResultInfo
 
 // ResultInfo contains all the non-stat result info. This includes the cache ID, any errors, the time of the poll, the request time duration, Astats System (Vitals), Poll ID, and Availability.
@@ -162,6 +167,7 @@ type ResultInfo struct {
 	Time        time.Time
 	RequestTime time.Duration
 	Vitals      Vitals
+	System      AstatsSystem
 	PollID      uint64
 	Available   bool
 }
@@ -175,6 +181,7 @@ func toInfo(r Result) ResultInfo {
 		Vitals:      r.Vitals,
 		PollID:      r.PollID,
 		Available:   r.Available,
+		System:      r.Astats.System,
 	}
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/ce71d1d8/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go b/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
index 1117248..b3beee8 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
@@ -582,13 +582,13 @@ func srvTRStateSelf(localStates peer.CRStatesThreadsafe) ([]byte, error) {
 }
 
 // TODO remove error params, handle by returning an error? How, since we need to return a non-standard code?
-func srvCacheStats(params url.Values, errorCount threadsafe.Uint, path string, toData todata.TODataThreadsafe, statResultHistory threadsafe.ResultStatHistory) ([]byte, int) {
+func srvCacheStats(params url.Values, errorCount threadsafe.Uint, path string, toData todata.TODataThreadsafe, statResultHistory threadsafe.ResultStatHistory, statInfoHistory threadsafe.ResultInfoHistory, monitorConfig TrafficMonitorConfigMapThreadsafe, combinedStates peer.CRStatesThreadsafe, statMaxKbpses threadsafe.CacheKbpses) ([]byte, int) {
 	filter, err := NewCacheStatFilter(path, params, toData.Get().ServerTypes)
 	if err != nil {
 		HandleErr(errorCount, path, err)
 		return []byte(err.Error()), http.StatusBadRequest
 	}
-	bytes, err := cache.StatsMarshall(statResultHistory.Get(), filter, params)
+	bytes, err := cache.StatsMarshall(statResultHistory.Get(), statInfoHistory.Get(), combinedStates.Get(), monitorConfig.Get(), statMaxKbpses.Get(), filter, params)
 	return WrapErrCode(errorCount, path, bytes, err)
 }
 
@@ -742,7 +742,7 @@ func MakeDispatchMap(
 			return WrapErrCode(errorCount, path, bytes, err)
 		}, ContentTypeJSON)),
 		"/publish/CacheStats": wrap(WrapParams(func(params url.Values, path string) ([]byte, int) {
-			return srvCacheStats(params, errorCount, path, toData, statResultHistory)
+			return srvCacheStats(params, errorCount, path, toData, statResultHistory, statInfoHistory, monitorConfig, combinedStates, statMaxKbpses)
 		}, ContentTypeJSON)),
 		"/publish/DsStats": wrap(WrapParams(func(params url.Values, path string) ([]byte, int) {
 			return srvDSStats(params, errorCount, path, toData, dsStats)

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/ce71d1d8/traffic_monitor/experimental/traffic_monitor/manager/stathistory.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/manager/stathistory.go b/traffic_monitor/experimental/traffic_monitor/manager/stathistory.go
index 4782fc3..2a15dd1 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/stathistory.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/stathistory.go
@@ -27,6 +27,7 @@ import (
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/config"
 	ds "github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/deliveryservice"
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/enum"
+	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/health"
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/peer"
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/threadsafe"
 	todata "github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/trafficopsdata"
@@ -78,9 +79,10 @@ func StartStatHistoryManager(
 	tickInterval := cfg.StatFlushInterval
 
 	precomputedData := map[enum.CacheName]cache.PrecomputedData{}
+	lastResults := map[enum.CacheName]cache.Result{}
 
 	process := func(results []cache.Result) {
-		processStatResults(results, statInfoHistory, statResultHistory, statMaxKbpses, combinedStates.Get(), lastStats, toData.Get(), errorCount, dsStats, lastStatEndTimes, lastStatDurations, unpolledCaches, monitorConfig.Get(), precomputedData)
+		processStatResults(results, statInfoHistory, statResultHistory, statMaxKbpses, combinedStates.Get(), lastStats, toData.Get(), errorCount, dsStats, lastStatEndTimes, lastStatDurations, unpolledCaches, monitorConfig.Get(), precomputedData, lastResults)
 	}
 
 	go func() {
@@ -135,6 +137,7 @@ func processStatResults(
 	unpolledCaches threadsafe.UnpolledCaches,
 	mc to.TrafficMonitorConfigMap,
 	precomputedData map[enum.CacheName]cache.PrecomputedData,
+	lastResults map[enum.CacheName]cache.Result,
 ) {
 
 	// setting the statHistory could be put in a goroutine concurrent with `ds.CreateStats`, if it were slow
@@ -148,7 +151,11 @@ func processStatResults(
 			log.Warnf("processStatResults got history count %v for %v, setting to 1\n", maxStats, result.ID)
 			maxStats = 1
 		}
+
 		// TODO determine if we want to add results with errors, or just print the errors now and don't add them.
+		if lastResult, ok := lastResults[result.ID]; ok {
+			health.GetVitals(&result, &lastResult, &mc) // TODO precompute
+		}
 		statInfoHistory.Add(result, maxStats)
 		statResultHistory.Add(result, maxStats)
 		// Don't add errored maxes or precomputed DSStats
@@ -162,6 +169,7 @@ func processStatResults(
 			precomputedData[result.ID] = result.PrecomputedData
 
 		}
+		lastResults[result.ID] = result
 	}
 	statInfoHistoryThreadsafe.Set(statInfoHistory)
 	statResultHistoryThreadsafe.Set(statResultHistory)


[2/3] incubator-trafficcontrol git commit: Add TM2 StatSummary

Posted by el...@apache.org.
Add TM2 StatSummary


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/commit/578d33be
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/tree/578d33be
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/diff/578d33be

Branch: refs/heads/master
Commit: 578d33be372609a6b746fe72cf02fa8c25dc9eff
Parents: 2480241
Author: Robert Butts <ro...@gmail.com>
Authored: Thu Jan 12 16:24:43 2017 -0700
Committer: Jeff Elsloo <je...@cable.comcast.com>
Committed: Tue Jan 17 13:04:16 2017 -0700

----------------------------------------------------------------------
 .../experimental/traffic_monitor/cache/data.go  |   1 +
 .../experimental/traffic_monitor/index.html     |   2 +-
 .../traffic_monitor/manager/datarequest.go      | 139 +++++++++++++++++--
 3 files changed, 133 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/578d33be/traffic_monitor/experimental/traffic_monitor/cache/data.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/cache/data.go b/traffic_monitor/experimental/traffic_monitor/cache/data.go
index 087995a..b7871a5 100644
--- a/traffic_monitor/experimental/traffic_monitor/cache/data.go
+++ b/traffic_monitor/experimental/traffic_monitor/cache/data.go
@@ -93,6 +93,7 @@ type ResultStatVal struct {
 }
 
 // TM1Time provides a custom MarshalJSON func to serialise a time.Time into milliseconds since the epoch, as served in Traffic Monitor 1.x APIs
+// TODO move somewhere more generic (enum?)
 type TM1Time time.Time
 
 func (t *TM1Time) MarshalJSON() ([]byte, error) {

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/578d33be/traffic_monitor/experimental/traffic_monitor/index.html
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/index.html b/traffic_monitor/experimental/traffic_monitor/index.html
index 847982e..5c02902 100644
--- a/traffic_monitor/experimental/traffic_monitor/index.html
+++ b/traffic_monitor/experimental/traffic_monitor/index.html
@@ -442,7 +442,7 @@ under the License.
 				<li class="endpoint"><a href="/publish/CrConfig">CrConfig (json)</a></li>
 				<li class="endpoint"><a href="/publish/PeerStates">PeerStates</a></li>
 				<li class="endpoint"><a href="/publish/Stats">Stats</a></li>
-				<li class="endpoint"><a href="/publish/StatsSummary">StatsSummary</a></li>
+				<li class="endpoint"><a href="/publish/StatSummary">StatSummary</a></li>
 				<li class="endpoint"><a href="/publish/ConfigDoc">ConfigDoc</a></li>
 			</ul>
 

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/578d33be/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go b/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
index ddf8e6b..1117248 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
@@ -135,7 +135,14 @@ func (f *CacheStatFilter) WithinStatHistoryMax(n int) bool {
 // If `wildcard` is empty, `stats` is considered exact.
 // If `type` is empty, all cache types are returned.
 func NewCacheStatFilter(path string, params url.Values, cacheTypes map[enum.CacheName]enum.CacheType) (cache.Filter, error) {
-	validParams := map[string]struct{}{"hc": struct{}{}, "stats": struct{}{}, "wildcard": struct{}{}, "type": struct{}{}, "hosts": struct{}{}}
+	validParams := map[string]struct{}{
+		"hc":       struct{}{},
+		"stats":    struct{}{},
+		"wildcard": struct{}{},
+		"type":     struct{}{},
+		"hosts":    struct{}{},
+		"cache":    struct{}{},
+	}
 	if len(params) > len(validParams) {
 		return nil, fmt.Errorf("invalid query parameters")
 	}
@@ -181,6 +188,12 @@ func NewCacheStatFilter(path string, params url.Values, cacheTypes map[enum.Cach
 			hosts[enum.CacheName(host)] = struct{}{}
 		}
 	}
+	if paramHosts, exists := params["cache"]; exists && len(paramHosts) > 0 {
+		commaHosts := strings.Split(paramHosts[0], ",")
+		for _, host := range commaHosts {
+			hosts[enum.CacheName(host)] = struct{}{}
+		}
+	}
 
 	pathArgument := getPathArgument(path)
 	if pathArgument != "" {
@@ -603,10 +616,6 @@ func srvPeerStates(params url.Values, errorCount threadsafe.Uint, path string, t
 	return WrapErrCode(errorCount, path, bytes, err)
 }
 
-func srvStatSummary() ([]byte, int) {
-	return nil, http.StatusNotImplemented
-}
-
 func srvStats(staticAppData StaticAppData, healthPollInterval time.Duration, lastHealthDurations DurationMapThreadsafe, fetchCount threadsafe.Uint, healthIteration threadsafe.Uint, errorCount threadsafe.Uint) ([]byte, error) {
 	return getStats(staticAppData, healthPollInterval, lastHealthDurations.Get(), fetchCount.Get(), healthIteration.Get(), errorCount.Get())
 }
@@ -744,15 +753,15 @@ func MakeDispatchMap(
 		"/publish/PeerStates": wrap(WrapParams(func(params url.Values, path string) ([]byte, int) {
 			return srvPeerStates(params, errorCount, path, toData, peerStates)
 		}, ContentTypeJSON)),
-		"/publish/StatSummary": wrap(WrapParams(func(params url.Values, path string) ([]byte, int) {
-			return srvStatSummary()
-		}, ContentTypeJSON)),
 		"/publish/Stats": wrap(WrapErr(errorCount, func() ([]byte, error) {
 			return srvStats(staticAppData, healthPollInterval, lastHealthDurations, fetchCount, healthIteration, errorCount)
 		}, ContentTypeJSON)),
 		"/publish/ConfigDoc": wrap(WrapErr(errorCount, func() ([]byte, error) {
 			return srvConfigDoc(opsConfig)
 		}, ContentTypeJSON)),
+		"/publish/StatSummary": wrap(WrapParams(func(params url.Values, path string) ([]byte, int) {
+			return srvStatSummary(params, errorCount, path, toData, statResultHistory)
+		}, ContentTypeJSON)),
 		"/api/cache-count": wrap(WrapBytes(func() []byte {
 			return srvAPICacheCount(localStates)
 		}, ContentTypeJSON)),
@@ -1119,3 +1128,117 @@ func getStats(staticAppData StaticAppData, pollingInterval time.Duration, lastHe
 
 	return json.Marshal(s)
 }
+
+type StatSummary struct {
+	Caches map[enum.CacheName]map[string]StatSummaryStat `json:"caches"`
+	srvhttp.CommonAPIData
+}
+
+type StatSummaryStat struct {
+	DataPointCount int64   `json:"dpCount"`
+	Start          float64 `json:"start"`
+	End            float64 `json:"end"`
+	High           float64 `json:"high"`
+	Low            float64 `json:"low"`
+	Average        float64 `json:"average"`
+	StartTime      int64   `json:"startTime"`
+	EndTime        int64   `json:"endTime"`
+}
+
+// toNumeric returns a float for any numeric type, and false if the interface does not hold a numeric type.
+// This allows converting unknown numeric types (for example, from JSON) in a single line
+func toNumeric(v interface{}) (float64, bool) {
+	switch i := v.(type) {
+	case uint8:
+		return float64(i), true
+	case uint16:
+		return float64(i), true
+	case uint32:
+		return float64(i), true
+	case uint64:
+		return float64(i), true
+	case int8:
+		return float64(i), true
+	case int16:
+		return float64(i), true
+	case int32:
+		return float64(i), true
+	case int64:
+		return float64(i), true
+	case float32:
+		return float64(i), true
+	case float64:
+		return float64(i), true
+	case int:
+		return float64(i), true
+	case uint:
+		return float64(i), true
+	default:
+		return 0.0, false
+	}
+}
+
+func createStatSummary(statResultHistory cache.ResultStatHistory, filter cache.Filter, params url.Values) StatSummary {
+	statPrefix := "ats."
+	ss := StatSummary{
+		Caches:        map[enum.CacheName]map[string]StatSummaryStat{},
+		CommonAPIData: srvhttp.GetCommonAPIData(params, time.Now()),
+	}
+	for cache, stats := range statResultHistory {
+		if !filter.UseCache(cache) {
+			continue
+		}
+		ssStats := map[string]StatSummaryStat{}
+		for statName, statHistory := range stats {
+			if !filter.UseStat(statName) {
+				continue
+			}
+			if len(statHistory) == 0 {
+				continue
+			}
+			ssStat := StatSummaryStat{}
+			msPerNs := int64(1000000)
+			ssStat.StartTime = time.Time(statHistory[len(statHistory)-1].Time).UnixNano() / msPerNs
+			ssStat.EndTime = time.Time(statHistory[0].Time).UnixNano() / msPerNs
+			oldestVal, isOldestValNumeric := toNumeric(statHistory[len(statHistory)-1].Val)
+			newestVal, isNewestValNumeric := toNumeric(statHistory[0].Val)
+			if !isOldestValNumeric || !isNewestValNumeric {
+				continue // skip non-numeric stats
+			}
+			ssStat.Start = oldestVal
+			ssStat.End = newestVal
+			ssStat.High = newestVal
+			ssStat.Low = newestVal
+			for _, val := range statHistory {
+				fVal, ok := toNumeric(val.Val)
+				if !ok {
+					continue // skip non-numeric stats. TODO warn about stat history containing different types?
+				}
+				for i := uint64(0); i < val.Span; i++ {
+					ssStat.DataPointCount++
+					ssStat.Average -= ssStat.Average / float64(ssStat.DataPointCount)
+					ssStat.Average += fVal / float64(ssStat.DataPointCount)
+				}
+				if fVal < ssStat.Low {
+					ssStat.Low = fVal
+				}
+				if fVal > ssStat.High {
+					ssStat.High = fVal
+				}
+			}
+			ssStats[statPrefix+statName] = ssStat
+		}
+		ss.Caches[cache] = ssStats
+	}
+	return ss
+}
+
+func srvStatSummary(params url.Values, errorCount threadsafe.Uint, path string, toData todata.TODataThreadsafe, statResultHistory threadsafe.ResultStatHistory) ([]byte, int) {
+	filter, err := NewCacheStatFilter(path, params, toData.Get().ServerTypes)
+	if err != nil {
+		HandleErr(errorCount, path, err)
+		return []byte(err.Error()), http.StatusBadRequest
+	}
+	bytes, err := json.Marshal(createStatSummary(statResultHistory.Get(), filter, params))
+	return WrapErrCode(errorCount, path, bytes, err)
+}


[3/3] incubator-trafficcontrol git commit: This closes #187.

Posted by el...@apache.org.
This closes #187.


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/commit/8833d3e2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/tree/8833d3e2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/diff/8833d3e2

Branch: refs/heads/master
Commit: 8833d3e2aea87cda14936470bbca111f853af56f
Parents: ce71d1d
Author: Jeff Elsloo <je...@cable.comcast.com>
Authored: Tue Jan 17 13:04:50 2017 -0700
Committer: Jeff Elsloo <je...@cable.comcast.com>
Committed: Tue Jan 17 13:04:50 2017 -0700

----------------------------------------------------------------------

----------------------------------------------------------------------