You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficcontrol.apache.org by ra...@apache.org on 2018/11/26 21:27:03 UTC

[trafficcontrol] branch master updated: Add TM Stat Poll Buffer Config

This is an automated email from the ASF dual-hosted git repository.

rawlin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/trafficcontrol.git


The following commit(s) were added to refs/heads/master by this push:
     new 604e61a  Add TM Stat Poll Buffer Config
604e61a is described below

commit 604e61aa908f20eb0a96febc02ccd7d2762eaf04
Author: Robert Butts <ro...@apache.org>
AuthorDate: Wed Nov 7 12:47:10 2018 -0700

    Add TM Stat Poll Buffer Config
    
    Adds a Monitor Stat Buffer config, which buffers all results up to the
    given interval. It defaults to 0, which behaves exactly like the
    current configuration.
    
    This gives operators more flexibility, to trade knowing that a cache
    is unhealthy ASAP, for more CPU.
---
 docs/source/admin/traffic_monitor.rst |  17 +++++
 traffic_monitor/config/config.go      |   8 +++
 traffic_monitor/manager/stat.go       | 121 +++++++++++++++++++++++++++++-----
 3 files changed, 128 insertions(+), 18 deletions(-)

diff --git a/docs/source/admin/traffic_monitor.rst b/docs/source/admin/traffic_monitor.rst
index bc590b4..8a73639 100755
--- a/docs/source/admin/traffic_monitor.rst
+++ b/docs/source/admin/traffic_monitor.rst
@@ -55,6 +55,23 @@ Configuration Overview
 
 Traffic Monitor is configured via two JSON configuration files, ``traffic_ops.cfg`` and ``traffic_monitor.cfg``, by default located in the ``conf`` directory in the install location. ``traffic_ops.cfg`` contains Traffic Ops connection information. Specify the URL, username, and password for the instance of Traffic Ops of which this Traffic Monitor is a member. ``traffic_monitor.cfg`` contains log file locations, as well as detailed application configuration variables such as processing f [...]
 
+Stat and Health Flush Configuration
+-----------------------------------
+
+The Monitor has a health flush interval, a stat flush interval, and a stat buffer interval. Recall that the monitor polls both stats and health. The health poll is so small and fast, a buffer is largely unnecessary. However, on a large CDN, the stat poll may involve thousands of caches with thousands of stats each, or more, and CPU may be a bottleneck.
+
+The flush intervals, ``health_flush_interval_ms`` and ``stat_flush_interval_ms``, indicate how often to flush stats or health, if results are continuously coming in with no break. This prevents starvation. Ideally, if there is enough CPU, the flushes should never occur.
+
+The default flush times are 200 milliseconds, which is suggested as a reasonable starting point, from which operators may adjust them higher or lower, depending on the need to get health data and stop serving to unhealthy caches as quickly as possible, balanced by the need to reduce CPU usage.
+
+The stat buffer interval, ``stat_buffer_interval_ms``, also provides a temporal buffer for stat processing. Stats will not be processed except after this interval, whereupon all pending stats will be processed, unless the flush interval occurs as a starvation safety. The stat buffer and flush intervals may be thought of as a state machine with two states: the "buffer state" accepts results until the buffer interval has elapsed, whereupon the "flush state" is entered, and results are acce [...]
+
+Note that this means the stat buffer interval acts as "bufferbloat," increasing the average and maximum time a cache may be down before it is processed and marked as unhealthy. If the stat buffer interval is non-zero, the average time a cache may be down before being marked unavailable is half the poll time plus half the stat buffer interval, and the maximum time is the poll time plus the stat buffer interval. For example, if the stat poll time is 6 seconds, and the stat buffer interval  [...]
+
+The default stat buffer interval is 0, which results in all stats being processed as quickly as possible. This is recommended, if a CDN operator is okay with the CPU usage and processing time, to minimize the time a cache may be unhealthy before being marked unavailable. If operators feel the need to set the stat buffer interval to a nonzero value, a recommended starting point is 5 milliseconds.
+
+It is not recommended to set either flush interval to 0, irrespective of the stat buffer interval. This will cause new results to be immediately processed, with little to no processing of multiple results concurrently. Result processing does not scale linearly, for example, it is not significantly more CPU or time to process 100 results than 10 at once. Thus, a flush interval which is too low will cause increased CPU usage, and potentially increased overall poll times, with little or no  [...]
+
 
 Troubleshooting and Log Files
 =============================
diff --git a/traffic_monitor/config/config.go b/traffic_monitor/config/config.go
index c64d250..8017950 100644
--- a/traffic_monitor/config/config.go
+++ b/traffic_monitor/config/config.go
@@ -55,6 +55,7 @@ type Config struct {
 	MaxHealthHistory             uint64        `json:"max_health_history"`
 	HealthFlushInterval          time.Duration `json:"-"`
 	StatFlushInterval            time.Duration `json:"-"`
+	StatBufferInterval           time.Duration `json:"-"`
 	LogLocationError             string        `json:"log_location_error"`
 	LogLocationWarning           string        `json:"log_location_warning"`
 	LogLocationInfo              string        `json:"log_location_info"`
@@ -88,6 +89,7 @@ var DefaultConfig = Config{
 	MaxHealthHistory:             5,
 	HealthFlushInterval:          200 * time.Millisecond,
 	StatFlushInterval:            200 * time.Millisecond,
+	StatBufferInterval:           0,
 	LogLocationError:             LogLocationStderr,
 	LogLocationWarning:           LogLocationStdout,
 	LogLocationInfo:              LogLocationNull,
@@ -115,6 +117,7 @@ func (c *Config) MarshalJSON() ([]byte, error) {
 		PeerOptimistic                 bool   `json:"peer_optimistic"`
 		HealthFlushIntervalMs          uint64 `json:"health_flush_interval_ms"`
 		StatFlushIntervalMs            uint64 `json:"stat_flush_interval_ms"`
+		StatBufferIntervalMs           uint64 `json:"stat_buffer_interval_ms"`
 		ServeReadTimeoutMs             uint64 `json:"serve_read_timeout_ms"`
 		ServeWriteTimeoutMs            uint64 `json:"serve_write_timeout_ms"`
 		*Alias
@@ -127,6 +130,7 @@ func (c *Config) MarshalJSON() ([]byte, error) {
 		PeerOptimistic:                 bool(true),
 		HealthFlushIntervalMs:          uint64(c.HealthFlushInterval / time.Millisecond),
 		StatFlushIntervalMs:            uint64(c.StatFlushInterval / time.Millisecond),
+		StatBufferIntervalMs:           uint64(c.StatBufferInterval / time.Millisecond),
 		Alias:                          (*Alias)(c),
 	})
 }
@@ -143,6 +147,7 @@ func (c *Config) UnmarshalJSON(data []byte) error {
 		PeerOptimistic                 *bool   `json:"peer_optimistic"`
 		HealthFlushIntervalMs          *uint64 `json:"health_flush_interval_ms"`
 		StatFlushIntervalMs            *uint64 `json:"stat_flush_interval_ms"`
+		StatBufferIntervalMs           *uint64 `json:"stat_buffer_interval_ms"`
 		ServeReadTimeoutMs             *uint64 `json:"serve_read_timeout_ms"`
 		ServeWriteTimeoutMs            *uint64 `json:"serve_write_timeout_ms"`
 		TrafficOpsMinRetryIntervalMs   *uint64 `json:"traffic_ops_min_retry_interval_ms"`
@@ -177,6 +182,9 @@ func (c *Config) UnmarshalJSON(data []byte) error {
 	if aux.StatFlushIntervalMs != nil {
 		c.StatFlushInterval = time.Duration(*aux.StatFlushIntervalMs) * time.Millisecond
 	}
+	if aux.StatBufferIntervalMs != nil {
+		c.StatBufferInterval = time.Duration(*aux.StatBufferIntervalMs) * time.Millisecond
+	}
 	if aux.ServeReadTimeoutMs != nil {
 		c.ServeReadTimeout = time.Duration(*aux.ServeReadTimeoutMs) * time.Millisecond
 	}
diff --git a/traffic_monitor/manager/stat.go b/traffic_monitor/manager/stat.go
index 2bde97d..fd14f57 100644
--- a/traffic_monitor/manager/stat.go
+++ b/traffic_monitor/manager/stat.go
@@ -76,49 +76,134 @@ func StartStatHistoryManager(
 	lastStats := threadsafe.NewLastStats()
 	dsStats := threadsafe.NewDSStats()
 	unpolledCaches := threadsafe.NewUnpolledCaches()
-	tickInterval := cfg.StatFlushInterval
 	localCacheStatus := threadsafe.NewCacheAvailableStatus()
 
 	precomputedData := map[tc.CacheName]cache.PrecomputedData{}
+
 	lastResults := map[tc.CacheName]cache.Result{}
 	overrideMap := map[tc.CacheName]bool{}
 
+	haveCachesChanged := func() bool {
+		select {
+		case <-cachesChanged:
+			return true
+		default:
+			return false
+		}
+	}
+
 	process := func(results []cache.Result) {
+		if haveCachesChanged() {
+			unpolledCaches.SetNewCaches(getNewCaches(localStates, monitorConfig))
+		}
 		processStatResults(results, statInfoHistory, statResultHistory, statMaxKbpses, combinedStates, lastStats, toData.Get(), errorCount, dsStats, lastStatEndTimes, lastStatDurations, unpolledCaches, monitorConfig.Get(), precomputedData, lastResults, localStates, events, localCacheStatus, overrideMap, combineState)
 	}
 
 	go func() {
-		var ticker *time.Ticker
-		<-cachesChanged // wait for the signal that localStates have been set
-		unpolledCaches.SetNewCaches(getNewCaches(localStates, monitorConfig))
+		flushTimer := time.NewTimer(cfg.StatFlushInterval)
+		// Note! bufferTimer MAY be uninitialized! If there is no cfg.StatBufferInterval, the timer WILL NOT be created with time.NewTimer(), and thus is NOT initialized, and MUST NOT have functions called, such as timer.Stop()! Those functions WILL panic.
+		bufferTimer := &time.Timer{}
+		bufferFakeChan := make(chan time.Time, 1) // fake chan, if there is no stat buffer interval. Unused, if cfg.StatBufferInterval != nil. Buffer 1, so don't need a separate goroutine to write.
+		if cfg.StatBufferInterval == 0 {
+			// if there is no stat buffer interval, make a timer which has already expired.
+			bufferFakeChan <- time.Now()
+			bufferTimer.C = bufferFakeChan
+		} else {
+			bufferTimer = time.NewTimer(cfg.StatBufferInterval)
+		}
 
-		for {
-			var results []cache.Result
-			results = append(results, <-cacheStatChan)
-			if ticker != nil {
-				ticker.Stop()
+		// resetBufferTimer resets the Buffer timer. It MUST have expired and been read.
+		// If the buffer loop is changed to allow finishing without being expired and read, this MUST be changed to stop and drain the channel (with a select/default, if it's possible to expire but not be read (like flush is now). Otherwise, it will deadlock and/or leak resources.
+		resetBufferTimer := func() {
+			if cfg.StatBufferInterval == 0 {
+				bufferFakeChan <- time.Now()
+			} else {
+				bufferTimer.Reset(cfg.StatBufferInterval)
+			}
+		}
+
+		// resetFlushTimer resets the Flush timer. It may or may not have been read or expired.
+		resetFlushTimer := func() {
+			if !flushTimer.Stop() {
+				select { // need to select/default because we don't know whether the flush timer was read
+				case <-flushTimer.C:
+				default:
+				}
 			}
-			ticker = time.NewTicker(tickInterval)
-		innerLoop:
+			flushTimer.Reset(cfg.StatFlushInterval)
+		}
+
+		// There are 2 timers: the Buffer, and the Flush.
+		// The Buffer says "never process until this much time has elapsed"
+		// The Flush says "if we're continuously getting new stats, with no break, and this much time has elasped, go ahead and process anyway to prevent starvation"
+		//
+		// So, we continuously read from the stat channel, until Buffer has elasped. Even if the channel is empty, wait and keep trying to read.
+		// Then, we move to State 2: continuously read from the stat channel, while there are things to read. If at any point there's nothing more to read, then process. Otherwise, if there are always thing to read, then after Flush time has elapsed, then go ahead and read anyway, and go to State 1.
+		//
+		// Note that either the Buffer or Flush may be configured to be 0.
+		// If the Buffer is 0, we immediately move to phase 2: process as fast as we can, only flush to prevent starvation. This optimizes the Monitor for getting health as quickly as possible, at the cost of CPU. (Having a buffer itself puts CPU above getting health results quickly, and the buffer interval is a factor of that)
+		// If the Flush is 0, then the Monitor will process every Buffer interval, regardless whether results are still coming in. This attempts to optimize for stability, attempting to ensure a poll every (Buffer + Poll Time) interval. Note this attempt may fail, and in particular, if the Monitor is unable to keep up with the given poll time and buffer, it will continuously back up. For this reason, setting a Flush of 0 is not recommended.
+		//
+		// Note the Flush and Buffer times are cumulative. That is, the total "maximum time a cache can be unhealthy before we know" is the Poll+Flush+Buffer. So, the buffer time elapses, then we start a new flush interval. They don't run concurrently.
+
+		results := []cache.Result{}
+
+		// flush loop - breaks after processing - processes when there are no pending results, or the flush time elapses.
+		flushLoop := func() {
+			log.Infof("StatHistoryManager starting flushLoop with %+v results\n", len(results))
+			resetFlushTimer()
 			for {
 				select {
-				case <-cachesChanged:
-					unpolledCaches.SetNewCaches(getNewCaches(localStates, monitorConfig))
-				case <-ticker.C:
-					log.Infof("StatHistoryManager flushing queued results\n")
+				case <-flushTimer.C: // first, make sure the flushTimer hasn't expired, by itself (because GO selects aren't ordered, so it could starve if we were reading <-cacheStatChan at the same level
+					log.Infof("StatHistoryManager flushLoop: flush timer fired, processing %+v results\n", len(results))
 					process(results)
-					break innerLoop
-				default:
+					return
+				default: // flushTimer hadn't expired: read cacheStatChan at the same level now.
+					// This extra level is necessary, because Go selects aren't ordered, so even after the Flush timer expires, the "case" could still never get hit,
+					// if there were continuously results from <-cacheStatChan at the same level.
 					select {
 					case r := <-cacheStatChan:
 						results = append(results, r)
+						// we're still processing as much as possible, and flushing, don't break to the outer Buffer loop, until we process.
 					default:
+						log.Infof("StatHistoryManager flushLoop: stat chan is empty, processing %+v results\n", len(results))
+						// Buffer expired (above), and the cacheStatChan is empty, so process
 						process(results)
-						break innerLoop
+						return
 					}
 				}
 			}
 		}
+
+		results = []cache.Result{}
+		results = append(results, <-cacheStatChan) // no point doing anything, until we read at least one stat.
+
+		// buffer loop - never breaks - calls flushLoop to actually process, when the buffer time elapses.
+		for {
+			// select only the bufferTimer first, to make sure it doesn't starve.
+			select {
+			case <-bufferTimer.C:
+				// buffer expired, move to State 2 (Flush)
+				flushLoop()
+				log.Infof("StatHistoryManager bufferLoop exiting flush loop, resetting buffer timer\n")
+				resetBufferTimer()
+				results = []cache.Result{}
+				results = append(results, <-cacheStatChan) // no point doing anything, until we read at least one stat.
+			default:
+				// buffer time hadn't elapsed, so we know we aren't starving. Go ahead and read the stat chan + buffer now.
+				select {
+				case r := <-cacheStatChan:
+					results = append(results, r)
+				case <-bufferTimer.C: // TODO protect against bufferTimer starvation
+					// buffer expired, move to State 2 (Flush): process until there's nothing to process, or the Flush elapses.
+					flushLoop()
+					log.Infof("StatHistoryManager bufferLoop (within stat select) exiting flush loop, resetting buffer timer\n")
+					resetBufferTimer()
+					results = []cache.Result{}
+					results = append(results, <-cacheStatChan) // no point doing anything, until we read at least one stat.
+				}
+			}
+		}
 	}()
 	return statInfoHistory, statResultHistory, statMaxKbpses, lastStatDurations, lastStats, &dsStats, unpolledCaches, localCacheStatus
 }