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

[06/20] incubator-trafficcontrol git commit: Add TM2 dynamic param health thresholds

Add TM2 dynamic param health thresholds


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

Branch: refs/heads/master
Commit: 8a7bed40c856dbafe441e599b4ca335d4247e1a2
Parents: 9c6dbea
Author: Robert Butts <ro...@gmail.com>
Authored: Fri Jan 20 10:00:52 2017 -0700
Committer: Dave Neuman <ne...@apache.org>
Committed: Wed Jan 25 10:29:46 2017 -0700

----------------------------------------------------------------------
 .../experimental/traffic_monitor/cache/cache.go | 198 +++++++++----------
 .../experimental/traffic_monitor/cache/data.go  |   8 +-
 .../experimental/traffic_monitor/enum/enum.go   |  35 ++++
 .../traffic_monitor/health/cache_health.go      | 123 +++++++++---
 .../traffic_monitor/manager/datarequest.go      |  41 +---
 .../traffic_monitor/manager/healthresult.go     | 112 ++++++-----
 .../traffic_monitor/manager/manager.go          |  11 +-
 .../traffic_monitor/manager/peer.go             |   6 +-
 .../traffic_monitor/manager/stathistory.go      |  62 +++++-
 .../traffic_monitor/threadsafe/events.go        |   3 +-
 traffic_ops/client/traffic_monitor_config.go    |  96 +++++++--
 11 files changed, 452 insertions(+), 243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/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 0aea799..541edaa 100644
--- a/traffic_monitor/experimental/traffic_monitor/cache/cache.go
+++ b/traffic_monitor/experimental/traffic_monitor/cache/cache.go
@@ -117,6 +117,84 @@ type Filter interface {
 
 const nsPerMs = 1000000
 
+type StatComputeFunc func(resultInfo ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{}
+
+// ComputedStats returns a map of cache stats which are computed by Traffic Monitor (rather than returned literally from ATS), mapped to the func to compute them.
+func ComputedStats() map[string]StatComputeFunc {
+	return map[string]StatComputeFunc{
+		"availableBandwidthInKbps": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.Vitals.MaxKbpsOut - info.Vitals.KbpsOut
+		},
+
+		"availableBandwidthInMbps": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return (info.Vitals.MaxKbpsOut - info.Vitals.KbpsOut) / 1000
+		},
+		"bandwidth": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.Vitals.KbpsOut
+		},
+		"error-string": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			if info.Error != nil {
+				return info.Error.Error()
+			}
+			return "false"
+		},
+		"isAvailable": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return combinedState.IsAvailable // if the cache is missing, default to false
+		},
+		"isHealthy": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			if enum.CacheStatusFromString(serverInfo.Status) == enum.CacheStatusAdminDown {
+				return true
+			}
+			return combinedState.IsAvailable
+		},
+		"kbps": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.Vitals.KbpsOut
+		},
+		"loadavg": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.Vitals.LoadAvg
+		},
+		"maxKbps": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.Vitals.MaxKbpsOut
+		},
+		"queryTime": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.RequestTime.Nanoseconds() / nsPerMs
+		},
+		"stateUrl": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return serverProfile.Parameters.HealthPollingURL
+		},
+		"status": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return serverInfo.Status
+		},
+		"system.astatsLoad": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.AstatsLoad
+		},
+		"system.configReloadRequests": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.ConfigLoadRequest
+		},
+		"system.configReloads": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.ConfigReloads
+		},
+		"system.inf.name": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.InfName
+		},
+		"system.inf.speed": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.InfSpeed
+		},
+		"system.lastReload": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.LastReload
+		},
+		"system.lastReloadRequest": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.LastReloadRequest
+		},
+		"system.proc.loadavg": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.ProcLoadavg
+		},
+		"system.proc.net.dev": func(info ResultInfo, serverInfo to.TrafficServer, serverProfile to.TMProfile, combinedState peer.IsAvailable) interface{} {
+			return info.System.ProcNetDev
+		},
+	}
+}
+
 // 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, statInfo ResultInfoHistory, combinedStates peer.Crstates, monitorConfig to.TrafficMonitorConfigMap, statMaxKbpses Kbpses, filter Filter, params url.Values) ([]byte, error) {
 	stats := Stats{
@@ -124,6 +202,8 @@ func StatsMarshall(statResultHistory ResultStatHistory, statInfo ResultInfoHisto
 		Caches:        map[enum.CacheName]map[string][]ResultStatVal{},
 	}
 
+	computedStats := ComputedStats()
+
 	// TODO in 1.0, stats are divided into 'location', 'cache', and 'type'. 'cache' are hidden by default.
 
 	for id, history := range statResultHistory {
@@ -153,7 +233,18 @@ func StatsMarshall(statResultHistory ResultStatHistory, statInfo ResultInfoHisto
 		if !filter.UseCache(id) {
 			continue
 		}
-		for i, info := range infos {
+
+		serverInfo, ok := monitorConfig.TrafficServer[string(id)]
+		if !ok {
+			log.Warnf("cache.StatsMarshall server %s missing from monitorConfig\n", id)
+		}
+
+		serverProfile, ok := monitorConfig.Profile[serverInfo.Profile]
+		if !ok {
+			log.Warnf("cache.StatsMarshall server %s missing profile in monitorConfig\n", id)
+		}
+
+		for i, resultInfo := range infos {
 			if !filter.WithinStatHistoryMax(i + 1) {
 				break
 			}
@@ -161,108 +252,13 @@ func StatsMarshall(statResultHistory ResultStatHistory, statInfo ResultInfoHisto
 				stats.Caches[id] = map[string][]ResultStatVal{}
 			}
 
-			t := info.Time
+			t := resultInfo.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
+			for stat, statValF := range computedStats {
+				if !filter.UseStat(stat) {
+					continue
 				}
-				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})
+				stats.Caches[id][stat] = append(stats.Caches[id][stat], ResultStatVal{Val: statValF(resultInfo, serverInfo, serverProfile, combinedStates.Caches[id]), Time: t, Span: 1}) // combinedState will default to unavailable
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/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 811babb..ee21907 100644
--- a/traffic_monitor/experimental/traffic_monitor/cache/data.go
+++ b/traffic_monitor/experimental/traffic_monitor/cache/data.go
@@ -36,6 +36,8 @@ type AvailableStatus struct {
 	Available bool
 	Status    string
 	Why       string
+	// UnavailableStat is the stat whose threshold made the cache unavailable. If this is the empty string, the cache is unavailable for a non-threshold reason. This exists so a poller (health, stat) won't mark an unavailable cache as available if the stat whose threshold was reached isn't available on that poller.
+	UnavailableStat string
 }
 
 // CacheAvailableStatuses is the available status of each cache.
@@ -161,7 +163,7 @@ type ResultInfo struct {
 	Available   bool
 }
 
-func toInfo(r Result) ResultInfo {
+func ToInfo(r Result) ResultInfo {
 	return ResultInfo{
 		ID:          r.ID,
 		Error:       r.Error,
@@ -177,7 +179,7 @@ func toInfo(r Result) ResultInfo {
 func toInfos(rs []Result) []ResultInfo {
 	infos := make([]ResultInfo, len(rs), len(rs))
 	for i, r := range rs {
-		infos[i] = toInfo(r)
+		infos[i] = ToInfo(r)
 	}
 	return infos
 }
@@ -204,7 +206,7 @@ func pruneInfos(history []ResultInfo, limit uint64) []ResultInfo {
 }
 
 func (a ResultInfoHistory) Add(r Result, limit uint64) {
-	a[r.ID] = pruneInfos(append([]ResultInfo{toInfo(r)}, a[r.ID]...), limit)
+	a[r.ID] = pruneInfos(append([]ResultInfo{ToInfo(r)}, a[r.ID]...), limit)
 }
 
 // Kbpses is the kbps values of each cache.

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/traffic_monitor/experimental/traffic_monitor/enum/enum.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/enum/enum.go b/traffic_monitor/experimental/traffic_monitor/enum/enum.go
index 853236a..44184df 100644
--- a/traffic_monitor/experimental/traffic_monitor/enum/enum.go
+++ b/traffic_monitor/experimental/traffic_monitor/enum/enum.go
@@ -175,3 +175,38 @@ func CacheStatusFromString(s string) CacheStatus {
 		return CacheStatusInvalid
 	}
 }
+
+// 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
+// TODO move to 'util' or 'tools' package?
+// TODO try to parse string stats as numbers?
+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
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/traffic_monitor/experimental/traffic_monitor/health/cache_health.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/health/cache_health.go b/traffic_monitor/experimental/traffic_monitor/health/cache_health.go
index 7e9af51..15d5b9a 100644
--- a/traffic_monitor/experimental/traffic_monitor/health/cache_health.go
+++ b/traffic_monitor/experimental/traffic_monitor/health/cache_health.go
@@ -28,7 +28,8 @@ import (
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/common/log"
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/cache"
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/enum"
-	traffic_ops "github.com/apache/incubator-trafficcontrol/traffic_ops/client"
+	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/peer"
+	to "github.com/apache/incubator-trafficcontrol/traffic_ops/client"
 )
 
 func setError(newResult *cache.Result, err error) {
@@ -37,7 +38,7 @@ func setError(newResult *cache.Result, err error) {
 }
 
 // GetVitals Gets the vitals to decide health on in the right format
-func GetVitals(newResult *cache.Result, prevResult *cache.Result, mc *traffic_ops.TrafficMonitorConfigMap) {
+func GetVitals(newResult *cache.Result, prevResult *cache.Result, mc *to.TrafficMonitorConfigMap) {
 	if newResult.Error != nil {
 		log.Errorf("cache_health.GetVitals() called with an errored Result!")
 		return
@@ -89,7 +90,7 @@ func GetVitals(newResult *cache.Result, prevResult *cache.Result, mc *traffic_op
 	// inf.speed -- value looks like "10000" (without the quotes) so it is in Mbps.
 	// TODO JvD: Should we really be running this code every second for every cache polled????? I don't think so.
 	interfaceBandwidth := newResult.Astats.System.InfSpeed
-	newResult.Vitals.MaxKbpsOut = int64(interfaceBandwidth)*1000 - mc.Profile[mc.TrafficServer[string(newResult.ID)].Profile].Parameters.MinFreeKbps
+	newResult.Vitals.MaxKbpsOut = int64(interfaceBandwidth) * 1000
 
 	// log.Infoln(newResult.Id, "BytesOut", newResult.Vitals.BytesOut, "BytesIn", newResult.Vitals.BytesIn, "Kbps", newResult.Vitals.KbpsOut, "max", newResult.Vitals.MaxKbpsOut)
 }
@@ -123,13 +124,25 @@ func cacheCapacityKbps(result cache.Result) int64 {
 	return int64(result.Astats.System.InfSpeed) * kbpsInMbps
 }
 
-// EvalCache returns whether the given cache should be marked available, and a string describing why
-func EvalCache(result cache.Result, mc *traffic_ops.TrafficMonitorConfigMap) (bool, string) {
-	toServer := mc.TrafficServer[string(result.ID)]
-	status := enum.CacheStatusFromString(toServer.Status)
-	params := mc.Profile[toServer.Profile].Parameters
-	kbpsThreshold, hasKbpsThreshold := getKbpsThreshold(params.HealthThresholdAvailableBandwidthInKbps)
-	queryTimeThreshold, hasQueryTimeThreshold := getQueryThreshold(int64(params.HealthThresholdQueryTime))
+// EvalCache returns whether the given cache should be marked available, a string describing why, and which stat exceeded a threshold. The `stats` may be nil, for pollers which don't poll stats.
+// The availability of EvalCache MAY NOT be used to directly set the cache's local availability, because the threshold stats may not be part of the poller which produced the result. Rather, if the cache was previously unavailable from a threshold, it must be verified that threshold stat is in the results before setting the cache to available.
+// TODO change to return a `cache.AvailableStatus`
+func EvalCache(result cache.ResultInfo, resultStats cache.ResultStatValHistory, mc *to.TrafficMonitorConfigMap) (bool, string, string) {
+	serverInfo, ok := mc.TrafficServer[string(result.ID)]
+	if !ok {
+		log.Errorf("Cache %v missing from from Traffic Ops Monitor Config - treating as OFFLINE\n", result.ID)
+		return false, "ERROR - server missing in Traffic Ops monitor config", ""
+	}
+	serverProfile, ok := mc.Profile[serverInfo.Profile]
+	if !ok {
+		log.Errorf("Cache %v profile %v missing from from Traffic Ops Monitor Config - treating as OFFLINE\n", result.ID, serverInfo.Profile)
+		return false, "ERROR - server profile missing in Traffic Ops monitor config", ""
+	}
+
+	status := enum.CacheStatusFromString(serverInfo.Status)
+	if status == enum.CacheStatusInvalid {
+		log.Errorf("Cache %v got invalid status from Traffic Ops '%v' - treating as Reported\n", result.ID, serverInfo.Status)
+	}
 
 	availability := "available"
 	if !result.Available {
@@ -138,25 +151,87 @@ func EvalCache(result cache.Result, mc *traffic_ops.TrafficMonitorConfigMap) (bo
 
 	switch {
 	case status == enum.CacheStatusInvalid:
-		log.Errorf("Cache %v got invalid status from Traffic Ops '%v' - treating as OFFLINE\n", result.ID, toServer.Status)
-		return false, getEventDescription(status, availability+"; invalid status")
+		log.Errorf("Cache %v got invalid status from Traffic Ops '%v' - treating as OFFLINE\n", result.ID, serverInfo.Status)
+		return false, getEventDescription(status, availability+"; invalid status"), ""
 	case status == enum.CacheStatusAdminDown:
-		return false, getEventDescription(status, availability)
+		return false, getEventDescription(status, availability), ""
 	case status == enum.CacheStatusOffline:
-		log.Errorf("Cache %v set to OFFLINE, but still polled\n", result.ID)
-		return false, getEventDescription(status, availability)
+		log.Errorf("Cache %v set to offline, but still polled\n", result.ID)
+		return false, getEventDescription(status, availability), ""
 	case status == enum.CacheStatusOnline:
-		return true, getEventDescription(status, availability)
+		return true, getEventDescription(status, availability), ""
 	case result.Error != nil:
-		return false, getEventDescription(status, fmt.Sprintf("%v", result.Error))
-	case result.Vitals.LoadAvg > params.HealthThresholdLoadAvg && params.HealthThresholdLoadAvg != 0:
-		return false, getEventDescription(status, fmt.Sprintf("loadavg too high (%.5f > %.5f)", result.Vitals.LoadAvg, params.HealthThresholdLoadAvg))
-	case hasKbpsThreshold && cacheCapacityKbps(result)-result.Vitals.KbpsOut < kbpsThreshold:
-		return false, getEventDescription(status, fmt.Sprintf("availableBandwidthInKbps too low (%d < %d)", cacheCapacityKbps(result)-result.Vitals.KbpsOut, kbpsThreshold))
-	case hasQueryTimeThreshold && result.RequestTime > queryTimeThreshold:
-		return false, getEventDescription(status, fmt.Sprintf("queryTime too high (%.5f > %.5f)", float64(result.RequestTime.Nanoseconds())/1e6, float64(queryTimeThreshold.Nanoseconds())/1e6))
+		return false, getEventDescription(status, fmt.Sprintf("%v", result.Error)), ""
+	}
+
+	computedStats := cache.ComputedStats()
+
+	for stat, threshold := range serverProfile.Parameters.Thresholds {
+		resultStat := interface{}(nil)
+		if computedStatF, ok := computedStats[stat]; ok {
+			dummyCombinedstate := peer.IsAvailable{} // the only stats which use combinedState are things like isAvailable, which don't make sense to ever be thresholds.
+			resultStat = computedStatF(result, serverInfo, serverProfile, dummyCombinedstate)
+		} else {
+			if resultStats == nil {
+				continue
+			}
+			resultStatHistory, ok := resultStats[stat]
+			if !ok {
+				continue
+			}
+			if len(resultStatHistory) < 1 {
+				continue
+			}
+			resultStat = resultStatHistory[0].Val
+		}
+
+		resultStatNum, ok := enum.ToNumeric(resultStat)
+		if !ok {
+			log.Errorf("health.EvalCache threshold stat %s was not a number: %v", stat, resultStat)
+			continue
+		}
+
+		if !InThreshold(threshold, resultStatNum) {
+			return false, getEventDescription(status, ExceedsThresholdMsg(stat, threshold, resultStatNum)), stat
+		}
+	}
+
+	return result.Available, getEventDescription(status, availability), ""
+}
+
+// ExceedsThresholdMsg returns a human-readable message for why the given value exceeds the threshold. It does NOT check whether the value actually exceeds the threshold; call `InThreshold` to check first.
+func ExceedsThresholdMsg(stat string, threshold to.HealthThreshold, val float64) string {
+	switch threshold.Comparator {
+	case "=":
+		return fmt.Sprintf("%s not equal (%f != %f)", stat, val, threshold.Val)
+	case ">":
+		return fmt.Sprintf("%s too low (%f < %f)", stat, val, threshold.Val)
+	case "<":
+		return fmt.Sprintf("%s too high (%f > %f)", stat, val, threshold.Val)
+	case ">=":
+		return fmt.Sprintf("%s too low (%f <= %f)", stat, val, threshold.Val)
+	case "<=":
+		return fmt.Sprintf("%s too high (%f >= %f)", stat, val, threshold.Val)
+	default:
+		return fmt.Sprintf("ERROR: Invalid Threshold: %+v", threshold)
+	}
+}
+
+func InThreshold(threshold to.HealthThreshold, val float64) bool {
+	switch threshold.Comparator {
+	case "=":
+		return val == threshold.Val
+	case ">":
+		return val > threshold.Val
+	case "<":
+		return val < threshold.Val
+	case ">=":
+		return val >= threshold.Val
+	case "<=":
+		return val <= threshold.Val
 	default:
-		return result.Available, getEventDescription(status, availability)
+		log.Errorf("Invalid Threshold: %+v", threshold)
+		return true // for safety, if a threshold somehow gets corrupted, don't start marking caches down.
 	}
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/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 c4385f3..2f42d74 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/datarequest.go
@@ -902,7 +902,7 @@ func createCacheStatuses(
 ) map[enum.CacheName]CacheStatus {
 	conns := createCacheConnections(statResultHistory)
 	statii := map[enum.CacheName]CacheStatus{}
-	localCacheStatus := localCacheStatusThreadsafe.Get()
+	localCacheStatus := localCacheStatusThreadsafe.Get().Copy() // TODO test whether copy is necessary
 	maxKbpses := statMaxKbpses.Get()
 
 	for cacheName, cacheType := range cacheTypes {
@@ -1162,39 +1162,6 @@ type StatSummaryStat struct {
 	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{
@@ -1217,8 +1184,8 @@ func createStatSummary(statResultHistory cache.ResultStatHistory, filter cache.F
 			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)
+			oldestVal, isOldestValNumeric := enum.ToNumeric(statHistory[len(statHistory)-1].Val)
+			newestVal, isNewestValNumeric := enum.ToNumeric(statHistory[0].Val)
 			if !isOldestValNumeric || !isNewestValNumeric {
 				continue // skip non-numeric stats
 			}
@@ -1227,7 +1194,7 @@ func createStatSummary(statResultHistory cache.ResultStatHistory, filter cache.F
 			ssStat.High = newestVal
 			ssStat.Low = newestVal
 			for _, val := range statHistory {
-				fVal, ok := toNumeric(val.Val)
+				fVal, ok := enum.ToNumeric(val.Val)
 				if !ok {
 					continue // skip non-numeric stats. TODO warn about stat history containing different types?
 				}

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/traffic_monitor/experimental/traffic_monitor/manager/healthresult.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/manager/healthresult.go b/traffic_monitor/experimental/traffic_monitor/manager/healthresult.go
index ed9f9fc..f99155d 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/healthresult.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/healthresult.go
@@ -87,9 +87,9 @@ func StartHealthResultManager(
 	errorCount threadsafe.Uint,
 	cfg config.Config,
 	events threadsafe.Events,
-) (DurationMapThreadsafe, threadsafe.CacheAvailableStatus, threadsafe.ResultHistory) {
+	localCacheStatus threadsafe.CacheAvailableStatus,
+) (DurationMapThreadsafe, threadsafe.ResultHistory) {
 	lastHealthDurations := NewDurationMapThreadsafe()
-	localCacheStatus := threadsafe.NewCacheAvailableStatus()
 	healthHistory := threadsafe.NewResultHistory()
 	go healthResultManagerListen(
 		cacheHealthChan,
@@ -106,7 +106,7 @@ func StartHealthResultManager(
 		localCacheStatus,
 		cfg,
 	)
-	return lastHealthDurations, localCacheStatus, healthHistory
+	return lastHealthDurations, healthHistory
 }
 
 func healthResultManagerListen(
@@ -127,6 +127,27 @@ func healthResultManagerListen(
 	lastHealthEndTimes := map[enum.CacheName]time.Time{}
 	// This reads at least 1 value from the cacheHealthChan. Then, we loop, and try to read from the channel some more. If there's nothing to read, we hit `default` and process. If there is stuff to read, we read it, then inner-loop trying to read more. If we're continuously reading and the channel is never empty, and we hit the tick time, process anyway even though the channel isn't empty, to prevent never processing (starvation).
 	var ticker *time.Ticker
+
+	process := func(results []cache.Result) {
+		processHealthResult(
+			cacheHealthChan,
+			toData,
+			localStates,
+			lastHealthDurations,
+			monitorConfig,
+			peerStates,
+			combinedStates,
+			fetchCount,
+			errorCount,
+			events,
+			localCacheStatus,
+			lastHealthEndTimes,
+			healthHistory,
+			results,
+			cfg,
+		)
+	}
+
 	for {
 		var results []cache.Result
 		results = append(results, <-cacheHealthChan)
@@ -139,46 +160,14 @@ func healthResultManagerListen(
 			select {
 			case <-ticker.C:
 				log.Warnf("Health Result Manager flushing queued results\n")
-				processHealthResult(
-					cacheHealthChan,
-					toData,
-					localStates,
-					lastHealthDurations,
-					monitorConfig,
-					peerStates,
-					combinedStates,
-					fetchCount,
-					errorCount,
-					events,
-					localCacheStatus,
-					lastHealthEndTimes,
-					healthHistory,
-					results,
-					cfg,
-				)
+				process(results)
 				break innerLoop
 			default:
 				select {
 				case r := <-cacheHealthChan:
 					results = append(results, r)
 				default:
-					processHealthResult(
-						cacheHealthChan,
-						toData,
-						localStates,
-						lastHealthDurations,
-						monitorConfig,
-						peerStates,
-						combinedStates,
-						fetchCount,
-						errorCount,
-						events,
-						localCacheStatus,
-						lastHealthEndTimes,
-						healthHistory,
-						results,
-						cfg,
-					)
+					process(results)
 					break innerLoop
 				}
 			}
@@ -207,6 +196,13 @@ func processHealthResult(
 	if len(results) == 0 {
 		return
 	}
+	defer func() {
+		for _, r := range results {
+			log.Debugf("poll %v %v finish\n", r.PollID, time.Now())
+			r.PollFinished <- r.PollID
+		}
+	}()
+
 	toDataCopy := toData.Get() // create a copy, so the same data used for all processing of this cache health result
 	localCacheStatus := localCacheStatusThreadsafe.Get().Copy()
 	monitorConfigCopy := monitorConfig.Get()
@@ -232,20 +228,30 @@ func processHealthResult(
 
 		healthHistoryCopy[healthResult.ID] = pruneHistory(append([]cache.Result{healthResult}, healthHistoryCopy[healthResult.ID]...), maxHistory)
 
-		isAvailable, whyAvailable := health.EvalCache(healthResult, &monitorConfigCopy)
+		isAvailable, whyAvailable, unavailableStat := health.EvalCache(cache.ToInfo(healthResult), nil, &monitorConfigCopy)
+		whyAvailable += "(healthpoll)" // debug
 		if available, ok := localStates.GetCache(healthResult.ID); !ok || available.IsAvailable != isAvailable {
-			log.Infof("Changing state for %s was: %t now: %t because %s error: %v", healthResult.ID, prevResult.Available, isAvailable, whyAvailable, healthResult.Error)
-			events.Add(health.Event{Time: healthResult.Time, Unix: healthResult.Time.Unix(), Description: whyAvailable, Name: healthResult.ID.String(), Hostname: healthResult.ID.String(), Type: toDataCopy.ServerTypes[healthResult.ID].String(), Available: isAvailable})
+			log.Infof("Changing state for %s was: %t now: %t because %s error: %v", healthResult.ID, available.IsAvailable, isAvailable, whyAvailable, healthResult.Error)
+			events.Add(health.Event{Time: time.Now(), Description: whyAvailable, Name: string(healthResult.ID), Hostname: string(healthResult.ID), Type: toDataCopy.ServerTypes[healthResult.ID].String(), Available: isAvailable})
+		}
+
+		// if the cache is now Available, and was previously unavailable due to a threshold, make sure this poller contains the stat which exceeded the threshold.
+		if previousStatus, hasPreviousStatus := localCacheStatus[healthResult.ID]; isAvailable && hasPreviousStatus && !previousStatus.Available && previousStatus.UnavailableStat != "" {
+			if !resultHasStat(previousStatus.UnavailableStat, healthResult) {
+				// TODO determine if it's ok to add the result data (but not availability). Or will making them not align cause issues?
+				continue
+			}
 		}
 
 		localCacheStatus[healthResult.ID] = cache.AvailableStatus{
-			Available: isAvailable,
-			Status:    monitorConfigCopy.TrafficServer[string(healthResult.ID)].Status,
-			Why:       whyAvailable,
+			Available:       isAvailable,
+			Status:          monitorConfigCopy.TrafficServer[string(healthResult.ID)].Status,
+			Why:             whyAvailable,
+			UnavailableStat: unavailableStat,
 		} // TODO move within localStates?
 		localStates.SetCache(healthResult.ID, peer.IsAvailable{IsAvailable: isAvailable})
 	}
-	calculateDeliveryServiceState(toDataCopy.DeliveryServiceServers, localStates)
+	CalculateDeliveryServiceState(toDataCopy.DeliveryServiceServers, localStates)
 	healthHistory.Set(healthHistoryCopy)
 	localCacheStatusThreadsafe.Set(localCacheStatus)
 	// TODO determine if we should combineCrStates() here
@@ -257,15 +263,25 @@ func processHealthResult(
 			lastHealthDurations[healthResult.ID] = d
 		}
 		lastHealthEndTimes[healthResult.ID] = time.Now()
-
-		log.Debugf("poll %v %v finish\n", healthResult.PollID, time.Now())
-		healthResult.PollFinished <- healthResult.PollID
 	}
 	lastHealthDurationsThreadsafe.Set(lastHealthDurations)
 }
 
+// resultHasStat returns whether the given stat is in the Result.
+// TODO move to cache?
+func resultHasStat(stat string, result cache.Result) bool {
+	computedStats := cache.ComputedStats()
+	if _, ok := computedStats[stat]; ok {
+		return true // health poll has all computed stats
+	}
+	if _, ok := result.Astats.Ats[stat]; ok {
+		return true
+	}
+	return false
+}
+
 // calculateDeliveryServiceState calculates the state of delivery services from the new cache state data `cacheState` and the CRConfig data `deliveryServiceServers` and puts the calculated state in the outparam `deliveryServiceStates`
-func calculateDeliveryServiceState(deliveryServiceServers map[enum.DeliveryServiceName][]enum.CacheName, states peer.CRStatesThreadsafe) {
+func CalculateDeliveryServiceState(deliveryServiceServers map[enum.DeliveryServiceName][]enum.CacheName, states peer.CRStatesThreadsafe) {
 	deliveryServices := states.GetDeliveryServices()
 	for deliveryServiceName, deliveryServiceState := range deliveryServices {
 		if _, ok := deliveryServiceServers[deliveryServiceName]; !ok {

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/traffic_monitor/experimental/traffic_monitor/manager/manager.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/manager/manager.go b/traffic_monitor/experimental/traffic_monitor/manager/manager.go
index 26ebb37..dba10ad 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/manager.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/manager.go
@@ -102,16 +102,17 @@ func Start(opsConfigFile string, cfg config.Config, staticAppData StaticAppData)
 		staticAppData,
 	)
 
-	combinedStates := StartPeerManager(
+	combinedStates, events := StartPeerManager(
 		peerHandler.ResultChannel,
 		localStates,
 		peerStates,
 		events,
-		cfg.PeerOptimistic,
+		cfg.PeerOptimistic, // TODO remove
 		toData,
+		cfg,
 	)
 
-	statInfoHistory, statResultHistory, statMaxKbpses, _, lastKbpsStats, dsStats, unpolledCaches := StartStatHistoryManager(
+	statInfoHistory, statResultHistory, statMaxKbpses, _, lastKbpsStats, dsStats, unpolledCaches, localCacheStatus := StartStatHistoryManager(
 		cacheStatHandler.ResultChannel,
 		localStates,
 		combinedStates,
@@ -120,9 +121,10 @@ func Start(opsConfigFile string, cfg config.Config, staticAppData StaticAppData)
 		errorCount,
 		cfg,
 		monitorConfig,
+		events,
 	)
 
-	lastHealthDurations, localCacheStatus, healthHistory := StartHealthResultManager(
+	lastHealthDurations, healthHistory := StartHealthResultManager(
 		cacheHealthHandler.ResultChannel,
 		toData,
 		localStates,
@@ -133,6 +135,7 @@ func Start(opsConfigFile string, cfg config.Config, staticAppData StaticAppData)
 		errorCount,
 		cfg,
 		events,
+		localCacheStatus,
 	)
 
 	StartOpsConfigManager(

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/traffic_monitor/experimental/traffic_monitor/manager/peer.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/manager/peer.go b/traffic_monitor/experimental/traffic_monitor/manager/peer.go
index 2f8cbdb..13b8411 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/peer.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/peer.go
@@ -27,6 +27,7 @@ import (
 
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/common/log"
 	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/common/util"
+	"github.com/apache/incubator-trafficcontrol/traffic_monitor/experimental/traffic_monitor/config"
 	"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"
@@ -42,7 +43,8 @@ func StartPeerManager(
 	events threadsafe.Events,
 	peerOptimistic bool,
 	toData todata.TODataThreadsafe,
-) peer.CRStatesThreadsafe {
+	cfg config.Config,
+) (peer.CRStatesThreadsafe, threadsafe.Events) {
 	combinedStates := peer.NewCRStatesThreadsafe()
 	overrideMap := map[enum.CacheName]bool{}
 
@@ -54,7 +56,7 @@ func StartPeerManager(
 			peerResult.PollFinished <- peerResult.PollID
 		}
 	}()
-	return combinedStates
+	return combinedStates, events
 }
 
 func comparePeerState(events threadsafe.Events, result peer.Result, peerStates peer.CRStatesPeersThreadsafe) {

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/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 d28713d..bf2d1dc 100644
--- a/traffic_monitor/experimental/traffic_monitor/manager/stathistory.go
+++ b/traffic_monitor/experimental/traffic_monitor/manager/stathistory.go
@@ -67,7 +67,8 @@ func StartStatHistoryManager(
 	errorCount threadsafe.Uint,
 	cfg config.Config,
 	monitorConfig TrafficMonitorConfigMapThreadsafe,
-) (threadsafe.ResultInfoHistory, threadsafe.ResultStatHistory, threadsafe.CacheKbpses, DurationMapThreadsafe, threadsafe.LastStats, threadsafe.DSStatsReader, threadsafe.UnpolledCaches) {
+	events threadsafe.Events,
+) (threadsafe.ResultInfoHistory, threadsafe.ResultStatHistory, threadsafe.CacheKbpses, DurationMapThreadsafe, threadsafe.LastStats, threadsafe.DSStatsReader, threadsafe.UnpolledCaches, threadsafe.CacheAvailableStatus) {
 	statInfoHistory := threadsafe.NewResultInfoHistory()
 	statResultHistory := threadsafe.NewResultStatHistory()
 	statMaxKbpses := threadsafe.NewCacheKbpses()
@@ -77,12 +78,13 @@ func StartStatHistoryManager(
 	dsStats := threadsafe.NewDSStats()
 	unpolledCaches := threadsafe.NewUnpolledCaches()
 	tickInterval := cfg.StatFlushInterval
+	localCacheStatus := threadsafe.NewCacheAvailableStatus()
 
 	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, lastResults)
+		processStatResults(results, statInfoHistory, statResultHistory, statMaxKbpses, combinedStates.Get(), lastStats, toData.Get(), errorCount, dsStats, lastStatEndTimes, lastStatDurations, unpolledCaches, monitorConfig.Get(), precomputedData, lastResults, localStates, events, localCacheStatus)
 	}
 
 	go func() {
@@ -118,7 +120,7 @@ func StartStatHistoryManager(
 			}
 		}
 	}()
-	return statInfoHistory, statResultHistory, statMaxKbpses, lastStatDurations, lastStats, &dsStats, unpolledCaches
+	return statInfoHistory, statResultHistory, statMaxKbpses, lastStatDurations, lastStats, &dsStats, unpolledCaches, localCacheStatus
 }
 
 // processStatResults processes the given results, creating and setting DSStats, LastStats, and other stats. Note this is NOT threadsafe, and MUST NOT be called from multiple threads.
@@ -138,14 +140,27 @@ func processStatResults(
 	mc to.TrafficMonitorConfigMap,
 	precomputedData map[enum.CacheName]cache.PrecomputedData,
 	lastResults map[enum.CacheName]cache.Result,
+	localStates peer.CRStatesThreadsafe,
+	events threadsafe.Events,
+	localCacheStatusThreadsafe threadsafe.CacheAvailableStatus,
 ) {
+	if len(results) == 0 {
+		return
+	}
+	defer func() {
+		for _, r := range results {
+			// log.Debugf("poll %v %v statfinish\n", result.PollID, endTime)
+			r.PollFinished <- r.PollID
+		}
+	}()
 
 	// setting the statHistory could be put in a goroutine concurrent with `ds.CreateStats`, if it were slow
 	statInfoHistory := statInfoHistoryThreadsafe.Get().Copy()
 	statResultHistory := statResultHistoryThreadsafe.Get().Copy()
 	statMaxKbpses := statMaxKbpsesThreadsafe.Get().Copy()
+	localCacheStatus := localCacheStatusThreadsafe.Get().Copy()
 
-	for _, result := range results {
+	for i, result := range results {
 		maxStats := uint64(mc.Profile[mc.TrafficServer[string(result.ID)].Profile].Parameters.HistoryCount)
 		if maxStats < 1 {
 			log.Warnf("processStatResults got history count %v for %v, setting to 1\n", maxStats, result.ID)
@@ -153,8 +168,13 @@ func processStatResults(
 		}
 
 		// 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 {
+		if lastResult, ok := lastResults[result.ID]; ok && result.Error == nil {
 			health.GetVitals(&result, &lastResult, &mc) // TODO precompute
+			if result.Error == nil {
+				results[i] = result
+			} else {
+				log.Errorf("stat poll getting vitals for %v: %v\n", result.ID, result.Error)
+			}
 		}
 		statInfoHistory.Add(result, maxStats)
 		statResultHistory.Add(result, maxStats)
@@ -193,6 +213,35 @@ func processStatResults(
 		lastStats.Set(newLastStats)
 	}
 
+	// TODO test
+	// TODO abstract setting availability logic (duplicated in healthresult.go)
+	for _, result := range results {
+		isAvailable, whyAvailable, unavailableStat := health.EvalCache(cache.ToInfo(result), statResultHistory[result.ID], &mc)
+		whyAvailable += "(statpoll)" // debug
+
+		if available, ok := localStates.GetCache(result.ID); !ok || available.IsAvailable != isAvailable {
+			log.Infof("Changing state for %s was: %t now: %t because %s error: %v", result.ID, available.IsAvailable, isAvailable, whyAvailable, result.Error)
+			events.Add(health.Event{Time: time.Now(), Description: whyAvailable, Name: string(result.ID), Hostname: string(result.ID), Type: toData.ServerTypes[result.ID].String(), Available: isAvailable})
+		}
+
+		// if the cache is now Available, and was previously unavailable due to a threshold, make sure this poller contains the stat which exceeded the threshold.
+		if previousStatus, hasPreviousStatus := localCacheStatus[result.ID]; isAvailable && hasPreviousStatus && !previousStatus.Available && previousStatus.UnavailableStat != "" {
+			if !resultHasStat(previousStatus.UnavailableStat, result) {
+				// TODO determine if it's ok to add the result data (but not availability). Or will making them not align cause issues?
+				continue
+			}
+		}
+		localCacheStatus[result.ID] = cache.AvailableStatus{
+			Available:       isAvailable,
+			Status:          mc.TrafficServer[string(result.ID)].Status,
+			Why:             whyAvailable,
+			UnavailableStat: unavailableStat,
+		} // TODO move within localStates?
+		localStates.SetCache(result.ID, peer.IsAvailable{IsAvailable: isAvailable})
+		CalculateDeliveryServiceState(toData.DeliveryServiceServers, localStates)
+		localCacheStatusThreadsafe.Set(localCacheStatus)
+	}
+
 	endTime := time.Now()
 	lastStatDurations := lastStatDurationsThreadsafe.Get().Copy()
 	for _, result := range results {
@@ -201,9 +250,6 @@ func processStatResults(
 			lastStatDurations[result.ID] = d
 		}
 		lastStatEndTimes[result.ID] = endTime
-
-		// log.Debugf("poll %v %v statfinish\n", result.PollID, endTime)
-		result.PollFinished <- result.PollID
 	}
 	lastStatDurationsThreadsafe.Set(lastStatDurations)
 	unpolledCaches.SetPolled(results, lastStats.Get())

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/traffic_monitor/experimental/traffic_monitor/threadsafe/events.go
----------------------------------------------------------------------
diff --git a/traffic_monitor/experimental/traffic_monitor/threadsafe/events.go b/traffic_monitor/experimental/traffic_monitor/threadsafe/events.go
index fe78ba5..9cd0320 100644
--- a/traffic_monitor/experimental/traffic_monitor/threadsafe/events.go
+++ b/traffic_monitor/experimental/traffic_monitor/threadsafe/events.go
@@ -57,13 +57,14 @@ func (o *Events) Get() []health.Event {
 func (o *Events) Add(e health.Event) {
 	// host="hostname", type=EDGE, available=true, msg="REPORTED - available"
 	log.Eventf(e.Time, "host=\"%s\", type=%s, available=%t, msg=\"%s\"", e.Hostname, e.Type, e.Available, e.Description)
+	o.m.Lock() // TODO test removing
 	events := copyEvents(*o.events)
 	e.Index = *o.nextIndex
 	events = append([]health.Event{e}, events...)
 	if len(events) > int(o.max) {
 		events = (events)[:o.max-1]
 	}
-	o.m.Lock()
+	// o.m.Lock()
 	*o.events = events
 	*o.nextIndex++
 	o.m.Unlock()

http://git-wip-us.apache.org/repos/asf/incubator-trafficcontrol/blob/8a7bed40/traffic_ops/client/traffic_monitor_config.go
----------------------------------------------------------------------
diff --git a/traffic_ops/client/traffic_monitor_config.go b/traffic_ops/client/traffic_monitor_config.go
index cf1208b..f5ece4d 100644
--- a/traffic_ops/client/traffic_monitor_config.go
+++ b/traffic_ops/client/traffic_monitor_config.go
@@ -75,14 +75,86 @@ type TMProfile struct {
 }
 
 // TMParameters ...
+// TODO change TO to return this struct, so a custom UnmarshalJSON isn't necessary.
 type TMParameters struct {
-	HealthConnectionTimeout                 int     `json:"health.connection.timeout"`
-	HealthPollingURL                        string  `json:"health.polling.url"`
-	HealthThresholdQueryTime                int     `json:"health.threshold.queryTime"`
-	HistoryCount                            int     `json:"history.count"`
-	HealthThresholdAvailableBandwidthInKbps string  `json:"health.threshold.availableBandwidthInKbps"`
-	HealthThresholdLoadAvg                  float64 `json:"health.threshold.loadavg,string"`
-	MinFreeKbps                             int64
+	HealthConnectionTimeout int    `json:"health.connection.timeout"`
+	HealthPollingURL        string `json:"health.polling.url"`
+	HistoryCount            int    `json:"history.count"`
+	MinFreeKbps             int64
+	Thresholds              map[string]HealthThreshold `json:"health_threshold"`
+}
+
+const DefaultHealthThresholdComparator = "<"
+
+type HealthThreshold struct {
+	Val        float64
+	Comparator string // TODO change to enum?
+}
+
+// strToThreshold takes a string like ">=42" and returns a HealthThreshold with a Val of `42` and a Comparator of `">="`. If no comparator exists, `DefaultHealthThresholdComparator` is used. If the string is not of the form "(>|<|)(=|)\d+" an error is returned
+func strToThreshold(s string) (HealthThreshold, error) {
+	comparators := []string{"=", ">", "<", ">=", "<="}
+	for _, comparator := range comparators {
+		if strings.HasPrefix(s, comparator) {
+			valStr := s[len(comparator):]
+			val, err := strconv.ParseFloat(valStr, 64)
+			if err != nil {
+				return HealthThreshold{}, fmt.Errorf("invalid threshold: NaN")
+			}
+			return HealthThreshold{Val: val, Comparator: comparator}, nil
+		}
+	}
+	val, err := strconv.ParseFloat(s, 64)
+	if err != nil {
+		return HealthThreshold{}, fmt.Errorf("invalid threshold: NaN")
+	}
+	return HealthThreshold{Val: val, Comparator: DefaultHealthThresholdComparator}, nil
+}
+
+func (params *TMParameters) UnmarshalJSON(bytes []byte) (err error) {
+	raw := map[string]interface{}{}
+	if err := json.Unmarshal(bytes, &raw); err != nil {
+		return err
+	}
+
+	if vi, ok := raw["health.connection.timeout"]; ok {
+		if v, ok := vi.(float64); !ok {
+			return fmt.Errorf("Unmarshalling TMParameters health.connection.timeout expected integer, got %v", vi)
+		} else {
+			params.HealthConnectionTimeout = int(v)
+		}
+	}
+
+	if vi, ok := raw["health.polling.url"]; ok {
+		if v, ok := vi.(string); !ok {
+			return fmt.Errorf("Unmarshalling TMParameters health.polling.url expected string, got %v", vi)
+		} else {
+			params.HealthPollingURL = v
+		}
+	}
+
+	if vi, ok := raw["history.count"]; ok {
+		if v, ok := vi.(float64); !ok {
+			return fmt.Errorf("Unmarshalling TMParameters history.count expected integer, got %v", vi)
+		} else {
+			params.HistoryCount = int(v)
+		}
+	}
+
+	params.Thresholds = map[string]HealthThreshold{}
+	thresholdPrefix := "health.threshold."
+	for k, v := range raw {
+		if strings.HasPrefix(k, thresholdPrefix) {
+			stat := k[len(thresholdPrefix):]
+			vStr := fmt.Sprintf("%v", v) // allows string or numeric JSON types. TODO check if a type switch is faster.
+			if t, err := strToThreshold(vStr); err != nil {
+				return fmt.Errorf("Unmarshalling TMParameters `health.threshold.` parameter value not of the form `(>|)(=|)\\d+`: stat '%s' value '%v'", k, v)
+			} else {
+				params.Thresholds[stat] = t
+			}
+		}
+	}
+	return nil
 }
 
 // TrafficMonitorConfigMap ...
@@ -146,14 +218,8 @@ func trafficMonitorTransformToMap(tmConfig *TrafficMonitorConfig) (*TrafficMonit
 	}
 
 	for _, profile := range tmConfig.Profiles {
-		bwThresholdString := profile.Parameters.HealthThresholdAvailableBandwidthInKbps
-		if strings.HasPrefix(bwThresholdString, ">") {
-			var err error
-			profile.Parameters.MinFreeKbps, err = strconv.ParseInt(bwThresholdString[1:len(bwThresholdString)], 10, 64)
-			if err != nil {
-				return nil, err
-			}
-		}
+		bwThreshold := profile.Parameters.Thresholds["availableBandwidthInKbps"]
+		profile.Parameters.MinFreeKbps = int64(bwThreshold.Val)
 		tm.Profile[profile.Name] = profile
 	}