vitess.io/vitess@v0.16.2/go/vt/vttablet/tabletmanager/vreplication/stats.go (about)

     1  /*
     2  Copyright 2019 The Vitess Authors.
     3  
     4  Licensed under the Apache License, Version 2.0 (the "License");
     5  you may not use this file except in compliance with the License.
     6  You may obtain a copy of the License at
     7  
     8      http://www.apache.org/licenses/LICENSE-2.0
     9  
    10  Unless required by applicable law or agreed to in writing, software
    11  distributed under the License is distributed on an "AS IS" BASIS,
    12  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    13  See the License for the specific language governing permissions and
    14  limitations under the License.
    15  */
    16  
    17  package vreplication
    18  
    19  import (
    20  	"fmt"
    21  	"sort"
    22  	"strings"
    23  	"sync"
    24  	"time"
    25  
    26  	"vitess.io/vitess/go/vt/binlog/binlogplayer"
    27  
    28  	"vitess.io/vitess/go/stats"
    29  	"vitess.io/vitess/go/vt/servenv"
    30  )
    31  
    32  var (
    33  	globalStats = &vrStats{}
    34  )
    35  
    36  func init() {
    37  	globalStats.register()
    38  }
    39  
    40  // StatusSummary returns the summary status of vreplication.
    41  func StatusSummary() (maxReplicationLagSeconds int64, binlogPlayersCount int32) {
    42  	return globalStats.maxReplicationLagSeconds(), int32(globalStats.numControllers())
    43  }
    44  
    45  // AddStatusPart adds the vreplication status to the status page.
    46  func AddStatusPart() {
    47  	servenv.AddStatusPart("VReplication", vreplicationTemplate, func() any {
    48  		return globalStats.status()
    49  	})
    50  }
    51  
    52  // vrStats exports the stats for Engine. It's a separate structure to
    53  // prevent deadlocks with the mutex in Engine. The Engine pushes changes
    54  // to this struct whenever there is a relevant change.
    55  // This is a singleton.
    56  type vrStats struct {
    57  	mu          sync.Mutex
    58  	isOpen      bool
    59  	controllers map[int]*controller
    60  }
    61  
    62  func (st *vrStats) register() {
    63  	stats.NewGaugeFunc("VReplicationStreamCount", "Number of vreplication streams", st.numControllers)
    64  	stats.NewGaugeFunc("VReplicationLagSecondsMax", "Max vreplication seconds behind primary", st.maxReplicationLagSeconds)
    65  	stats.Publish("VReplicationStreamState", stats.StringMapFunc(func() map[string]string {
    66  		st.mu.Lock()
    67  		defer st.mu.Unlock()
    68  		result := make(map[string]string, len(st.controllers))
    69  		for _, ct := range st.controllers {
    70  			result[ct.workflow+"."+fmt.Sprintf("%v", ct.id)] = ct.blpStats.State.Get()
    71  		}
    72  		return result
    73  	}))
    74  	stats.NewGaugesFuncWithMultiLabels(
    75  		"VReplicationLagSeconds",
    76  		"vreplication seconds behind primary per stream",
    77  		[]string{"source_keyspace", "source_shard", "workflow", "counts"},
    78  		func() map[string]int64 {
    79  			st.mu.Lock()
    80  			defer st.mu.Unlock()
    81  			result := make(map[string]int64, len(st.controllers))
    82  			for _, ct := range st.controllers {
    83  				result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)] = ct.blpStats.ReplicationLagSeconds.Get()
    84  			}
    85  			return result
    86  		})
    87  
    88  	stats.NewCounterFunc(
    89  		"VReplicationLagSecondsTotal",
    90  		"vreplication seconds behind primary aggregated across all streams",
    91  		func() int64 {
    92  			st.mu.Lock()
    93  			defer st.mu.Unlock()
    94  			result := int64(0)
    95  			for _, ct := range st.controllers {
    96  				result += ct.blpStats.ReplicationLagSeconds.Get()
    97  			}
    98  			return result
    99  		})
   100  
   101  	stats.NewRateFunc(
   102  		"VReplicationQPS",
   103  		"vreplication operations per second aggregated across all streams",
   104  		func() map[string][]float64 {
   105  			st.mu.Lock()
   106  			defer st.mu.Unlock()
   107  			result := make(map[string][]float64)
   108  			for _, ct := range st.controllers {
   109  				for k, v := range ct.blpStats.Rates.Get() {
   110  					result[k] = v
   111  				}
   112  			}
   113  			return result
   114  		})
   115  
   116  	stats.NewRateFunc(
   117  		"VReplicationLag",
   118  		"vreplication lag per stream",
   119  		func() map[string][]float64 {
   120  			st.mu.Lock()
   121  			defer st.mu.Unlock()
   122  			result := make(map[string][]float64)
   123  			for _, ct := range st.controllers {
   124  				for k, v := range ct.blpStats.VReplicationLagRates.Get() {
   125  					result[k] = v
   126  				}
   127  			}
   128  			return result
   129  		})
   130  
   131  	stats.Publish("VReplicationSource", stats.StringMapFunc(func() map[string]string {
   132  		st.mu.Lock()
   133  		defer st.mu.Unlock()
   134  		result := make(map[string]string, len(st.controllers))
   135  		for _, ct := range st.controllers {
   136  			result[fmt.Sprintf("%v", ct.id)] = ct.source.Keyspace + "/" + ct.source.Shard
   137  		}
   138  		return result
   139  	}))
   140  	stats.Publish("VReplicationSourceTablet", stats.StringMapFunc(func() map[string]string {
   141  		st.mu.Lock()
   142  		defer st.mu.Unlock()
   143  		result := make(map[string]string, len(st.controllers))
   144  		for _, ct := range st.controllers {
   145  			result[fmt.Sprintf("%v", ct.id)] = ct.sourceTablet.Get()
   146  		}
   147  		return result
   148  	}))
   149  	stats.Publish("VReplicationMessages", stats.StringMapFunc(func() map[string]string {
   150  		st.mu.Lock()
   151  		defer st.mu.Unlock()
   152  		result := make(map[string]string, len(st.controllers))
   153  		for _, ct := range st.controllers {
   154  			var messages []string
   155  			for _, rec := range ct.blpStats.History.Records() {
   156  				hist := rec.(*binlogplayer.StatsHistoryRecord)
   157  				messages = append(messages, fmt.Sprintf("%s:%s", hist.Time.Format(time.RFC3339Nano), hist.Message))
   158  			}
   159  			result[fmt.Sprintf("%v", ct.id)] = strings.Join(messages, "; ")
   160  		}
   161  		return result
   162  	}))
   163  	stats.NewGaugesFuncWithMultiLabels(
   164  		"VReplicationPhaseTimings",
   165  		"vreplication per phase timings per stream",
   166  		[]string{"source_keyspace", "source_shard", "workflow", "counts", "phase"},
   167  		func() map[string]int64 {
   168  			st.mu.Lock()
   169  			defer st.mu.Unlock()
   170  			result := make(map[string]int64, len(st.controllers))
   171  			for _, ct := range st.controllers {
   172  				for phase, t := range ct.blpStats.PhaseTimings.Histograms() {
   173  					result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)+"."+phase] = t.Total()
   174  				}
   175  			}
   176  			return result
   177  		})
   178  	stats.NewCounterFunc(
   179  		"VReplicationPhaseTimingsTotal",
   180  		"vreplication per phase timings aggregated across all phases and streams",
   181  		func() int64 {
   182  			st.mu.Lock()
   183  			defer st.mu.Unlock()
   184  			result := int64(0)
   185  			for _, ct := range st.controllers {
   186  				for _, t := range ct.blpStats.PhaseTimings.Histograms() {
   187  					result += t.Total()
   188  				}
   189  			}
   190  			return result
   191  		})
   192  
   193  	stats.NewGaugesFuncWithMultiLabels(
   194  		"VReplicationPhaseTimingsCounts",
   195  		"vreplication per phase count of timings per stream",
   196  		[]string{"source_keyspace", "source_shard", "workflow", "counts", "phase"},
   197  		func() map[string]int64 {
   198  			st.mu.Lock()
   199  			defer st.mu.Unlock()
   200  			result := make(map[string]int64, len(st.controllers))
   201  			for _, ct := range st.controllers {
   202  				for phase, t := range ct.blpStats.PhaseTimings.Counts() {
   203  					result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)+"."+phase] = t
   204  				}
   205  			}
   206  			return result
   207  		})
   208  
   209  	stats.NewGaugesFuncWithMultiLabels(
   210  		"VReplicationQueryCount",
   211  		"vreplication query counts per stream",
   212  		[]string{"source_keyspace", "source_shard", "workflow", "counts", "phase"},
   213  		func() map[string]int64 {
   214  			st.mu.Lock()
   215  			defer st.mu.Unlock()
   216  			result := make(map[string]int64, len(st.controllers))
   217  			for _, ct := range st.controllers {
   218  				for label, count := range ct.blpStats.QueryCount.Counts() {
   219  					if label == "" {
   220  						continue
   221  					}
   222  					result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)+"."+label] = count
   223  				}
   224  			}
   225  			return result
   226  		})
   227  
   228  	stats.NewCounterFunc(
   229  		"VReplicationQueryCountTotal",
   230  		"vreplication query counts aggregated across all streams",
   231  		func() int64 {
   232  			st.mu.Lock()
   233  			defer st.mu.Unlock()
   234  			result := int64(0)
   235  			for _, ct := range st.controllers {
   236  				for _, count := range ct.blpStats.QueryCount.Counts() {
   237  					result += count
   238  				}
   239  			}
   240  			return result
   241  		})
   242  
   243  	stats.NewGaugesFuncWithMultiLabels(
   244  		"VReplicationNoopQueryCount",
   245  		"vreplication noop query counts per stream",
   246  		[]string{"source_keyspace", "source_shard", "workflow", "counts", "phase"},
   247  		func() map[string]int64 {
   248  			st.mu.Lock()
   249  			defer st.mu.Unlock()
   250  			result := make(map[string]int64, len(st.controllers))
   251  			for _, ct := range st.controllers {
   252  				for label, count := range ct.blpStats.NoopQueryCount.Counts() {
   253  					if label == "" {
   254  						continue
   255  					}
   256  					result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)+"."+label] = count
   257  				}
   258  			}
   259  			return result
   260  		})
   261  
   262  	stats.NewCounterFunc(
   263  		"VReplicationNoopQueryCountTotal",
   264  		"vreplication query noop counts aggregated across all streams",
   265  		func() int64 {
   266  			st.mu.Lock()
   267  			defer st.mu.Unlock()
   268  			result := int64(0)
   269  			for _, ct := range st.controllers {
   270  				for _, count := range ct.blpStats.NoopQueryCount.Counts() {
   271  					result += count
   272  				}
   273  			}
   274  			return result
   275  		})
   276  	stats.NewGaugesFuncWithMultiLabels(
   277  		"VReplicationCopyRowCount",
   278  		"vreplication rows copied in copy phase per stream",
   279  		[]string{"source_keyspace", "source_shard", "workflow", "counts"},
   280  		func() map[string]int64 {
   281  			st.mu.Lock()
   282  			defer st.mu.Unlock()
   283  			result := make(map[string]int64, len(st.controllers))
   284  			for _, ct := range st.controllers {
   285  				result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)] = ct.blpStats.CopyRowCount.Get()
   286  			}
   287  			return result
   288  		})
   289  
   290  	stats.NewCounterFunc(
   291  		"VReplicationCopyRowCountTotal",
   292  		"vreplication rows copied in copy phase aggregated across all streams",
   293  		func() int64 {
   294  			st.mu.Lock()
   295  			defer st.mu.Unlock()
   296  			result := int64(0)
   297  			for _, ct := range st.controllers {
   298  				result += ct.blpStats.CopyRowCount.Get()
   299  			}
   300  			return result
   301  		})
   302  
   303  	stats.NewGaugesFuncWithMultiLabels(
   304  		"VReplicationCopyLoopCount",
   305  		"Number of times the copy phase looped per stream",
   306  		[]string{"source_keyspace", "source_shard", "workflow", "counts"},
   307  		func() map[string]int64 {
   308  			st.mu.Lock()
   309  			defer st.mu.Unlock()
   310  			result := make(map[string]int64, len(st.controllers))
   311  			for _, ct := range st.controllers {
   312  				result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)] = ct.blpStats.CopyLoopCount.Get()
   313  			}
   314  			return result
   315  		})
   316  
   317  	stats.NewCounterFunc(
   318  		"VReplicationCopyLoopCountTotal",
   319  		"Number of times the copy phase looped aggregated across streams",
   320  		func() int64 {
   321  			st.mu.Lock()
   322  			defer st.mu.Unlock()
   323  			result := int64(0)
   324  			for _, ct := range st.controllers {
   325  				result += ct.blpStats.CopyLoopCount.Get()
   326  			}
   327  			return result
   328  		})
   329  	stats.NewCountersFuncWithMultiLabels(
   330  		"VReplicationErrors",
   331  		"Errors during vreplication",
   332  		[]string{"workflow", "id", "type"},
   333  		func() map[string]int64 {
   334  			st.mu.Lock()
   335  			defer st.mu.Unlock()
   336  			result := make(map[string]int64)
   337  			for _, ct := range st.controllers {
   338  				for key, val := range ct.blpStats.ErrorCounts.Counts() {
   339  					result[fmt.Sprintf("%s.%d.%s", ct.workflow, ct.id, key)] = val
   340  				}
   341  			}
   342  			return result
   343  		})
   344  	stats.NewGaugesFuncWithMultiLabels(
   345  		"VReplicationHeartbeat",
   346  		"Time when last heartbeat was received from a vstreamer",
   347  		[]string{"source_keyspace", "source_shard", "workflow", "time"},
   348  		func() map[string]int64 {
   349  			st.mu.Lock()
   350  			defer st.mu.Unlock()
   351  			result := make(map[string]int64, len(st.controllers))
   352  			for _, ct := range st.controllers {
   353  				result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)] = ct.blpStats.Heartbeat()
   354  			}
   355  			return result
   356  		})
   357  
   358  	stats.NewGaugesFuncWithMultiLabels(
   359  		"VReplicationTableCopyRowCounts",
   360  		"vreplication rows copied in copy phase per table per stream",
   361  		[]string{"source_keyspace", "source_shard", "workflow", "counts", "table"},
   362  		func() map[string]int64 {
   363  			st.mu.Lock()
   364  			defer st.mu.Unlock()
   365  			result := make(map[string]int64, len(st.controllers))
   366  			for _, ct := range st.controllers {
   367  				for table, count := range ct.blpStats.TableCopyRowCounts.Counts() {
   368  					if table == "" {
   369  						continue
   370  					}
   371  					result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)+"."+table] = count
   372  				}
   373  			}
   374  			return result
   375  		})
   376  	stats.NewGaugesFuncWithMultiLabels(
   377  		"VReplicationTableCopyTimings",
   378  		"vreplication copy phase timings per table per stream",
   379  		[]string{"source_keyspace", "source_shard", "workflow", "counts", "table"},
   380  		func() map[string]int64 {
   381  			st.mu.Lock()
   382  			defer st.mu.Unlock()
   383  			result := make(map[string]int64, len(st.controllers))
   384  			for _, ct := range st.controllers {
   385  				for table, t := range ct.blpStats.TableCopyTimings.Histograms() {
   386  					result[ct.source.Keyspace+"."+ct.source.Shard+"."+ct.workflow+"."+fmt.Sprintf("%v", ct.id)+"."+table] = t.Total()
   387  				}
   388  			}
   389  			return result
   390  		})
   391  }
   392  
   393  func (st *vrStats) numControllers() int64 {
   394  	st.mu.Lock()
   395  	defer st.mu.Unlock()
   396  	return int64(len(st.controllers))
   397  }
   398  
   399  func (st *vrStats) maxReplicationLagSeconds() int64 {
   400  	st.mu.Lock()
   401  	defer st.mu.Unlock()
   402  	max := int64(0)
   403  	for _, ct := range st.controllers {
   404  		if cur := ct.blpStats.ReplicationLagSeconds.Get(); cur > max {
   405  			max = cur
   406  		}
   407  	}
   408  	return max
   409  }
   410  
   411  func (st *vrStats) status() *EngineStatus {
   412  	st.mu.Lock()
   413  	defer st.mu.Unlock()
   414  
   415  	status := &EngineStatus{}
   416  	status.IsOpen = st.isOpen
   417  
   418  	status.Controllers = make([]*ControllerStatus, len(st.controllers))
   419  	i := 0
   420  	for _, ct := range st.controllers {
   421  		status.Controllers[i] = &ControllerStatus{
   422  			Index:                 ct.id,
   423  			Source:                ct.source.String(),
   424  			StopPosition:          ct.stopPos,
   425  			LastPosition:          ct.blpStats.LastPosition().String(),
   426  			Heartbeat:             ct.blpStats.Heartbeat(),
   427  			ReplicationLagSeconds: ct.blpStats.ReplicationLagSeconds.Get(),
   428  			Counts:                ct.blpStats.Timings.Counts(),
   429  			Rates:                 ct.blpStats.Rates.Get(),
   430  			State:                 ct.blpStats.State.Get(),
   431  			SourceTablet:          ct.sourceTablet.Get(),
   432  			Messages:              ct.blpStats.MessageHistory(),
   433  			QueryCounts:           ct.blpStats.QueryCount.Counts(),
   434  			PhaseTimings:          ct.blpStats.PhaseTimings.Counts(),
   435  			CopyRowCount:          ct.blpStats.CopyRowCount.Get(),
   436  			CopyLoopCount:         ct.blpStats.CopyLoopCount.Get(),
   437  			NoopQueryCounts:       ct.blpStats.NoopQueryCount.Counts(),
   438  			TableCopyTimings:      ct.blpStats.TableCopyTimings.Counts(),
   439  		}
   440  		i++
   441  	}
   442  	sort.Slice(status.Controllers, func(i, j int) bool { return status.Controllers[i].Index < status.Controllers[j].Index })
   443  	return status
   444  }
   445  
   446  // EngineStatus contains a renderable status of the Engine.
   447  type EngineStatus struct {
   448  	IsOpen      bool
   449  	Controllers []*ControllerStatus
   450  }
   451  
   452  // ControllerStatus contains a renderable status of a controller.
   453  type ControllerStatus struct {
   454  	Index                 uint32
   455  	Source                string
   456  	SourceShard           string
   457  	StopPosition          string
   458  	LastPosition          string
   459  	Heartbeat             int64
   460  	ReplicationLagSeconds int64
   461  	Counts                map[string]int64
   462  	Rates                 map[string][]float64
   463  	State                 string
   464  	SourceTablet          string
   465  	Messages              []string
   466  	QueryCounts           map[string]int64
   467  	PhaseTimings          map[string]int64
   468  	CopyRowCount          int64
   469  	CopyLoopCount         int64
   470  	NoopQueryCounts       map[string]int64
   471  	TableCopyTimings      map[string]int64
   472  }
   473  
   474  var vreplicationTemplate = `
   475  {{if .IsOpen}}VReplication state: Open</br>
   476  <table>
   477    <tr>
   478      <th>Index</th>
   479      <th>Source</th>
   480      <th>Source Tablet</th>
   481      <th>State</th>
   482      <th>Stop Position</th>
   483      <th>Last Position</th>
   484      <th>VReplication Lag</th>
   485      <th>Counts</th>
   486      <th>Rates</th>
   487      <th>Last Message</th>
   488    </tr>
   489    {{range .Controllers}}<tr>
   490        <td>{{.Index}}</td>
   491        <td>{{.Source}}</td>
   492        <td>{{.SourceTablet}}</td>
   493        <td>{{.State}}</td>
   494        <td>{{.StopPosition}}</td>
   495        <td>{{.LastPosition}}</td>
   496        <td>{{.ReplicationLagSeconds}}</td>
   497        <td>{{range $key, $value := .Counts}}<b>{{$key}}</b>: {{$value}}<br>{{end}}</td>
   498        <td>{{range $key, $values := .Rates}}<b>{{$key}}</b>: {{range $values}}{{.}} {{end}}<br>{{end}}</td>
   499        <td>{{range $index, $value := .Messages}}{{$value}}<br>{{end}}</td>
   500      </tr>{{end}}
   501  <div id="vreplication_qps_chart" style="height: 500px; width: 900px">QPS All Streams </div>
   502  
   503  <script src="https://www.gstatic.com/charts/loader.js"></script>
   504  <script type="text/javascript">
   505  
   506  function drawVReplicationQPSChart() {
   507    var div = document.getElementById('vreplication_qps_chart')
   508    var chart = new google.visualization.LineChart(div);
   509  
   510    var options = {
   511      title: "VReplication QPS across all streams",
   512      focusTarget: 'category',
   513      vAxis: {
   514        viewWindow: {min: 0},
   515      }
   516    };
   517  
   518    // If we're accessing status through a proxy that requires a URL prefix,
   519    // add the prefix to the vars URL.
   520    var vars_url = '/debug/vars';
   521    var pos = window.location.pathname.lastIndexOf('/debug/status');
   522    if (pos > 0) {
   523      vars_url = window.location.pathname.substring(0, pos) + vars_url;
   524    }
   525  
   526    const redraw = () => fetch(vars_url)
   527    .then(async (response) => {
   528  	const input_data = await response.json();
   529  	var now = new Date();
   530        var qps = input_data.VReplicationQPS;
   531        var planTypes = Object.keys(qps);
   532        if (planTypes.length === 0) {
   533          planTypes = ["All"];
   534          qps["All"] = [];
   535        }
   536  
   537        var data = [["Time"].concat(planTypes)];
   538  
   539        // Create data points, starting with the most recent timestamp.
   540        // (On the graph this means going from right to left.)
   541        // Time span: 15 minutes in 5 second intervals.
   542        for (var i = 0; i < 15*60/5; i++) {
   543          var datum = [sampleDate(now, i)];
   544          for (var j = 0; j < planTypes.length; j++) {
   545            if (i < qps[planTypes[j]].length) {
   546            	// Rates are ordered from least recent to most recent.
   547            	// Therefore, we have to start reading from the end of the array.
   548            	var idx = qps[planTypes[j]].length - i - 1;
   549              datum.push(+qps[planTypes[j]][idx].toFixed(2));
   550            } else {
   551              // Assume 0.0 QPS for older, non-existent data points.
   552              datum.push(0);
   553            }
   554          }
   555          data.push(datum)
   556        }
   557        chart.draw(google.visualization.arrayToDataTable(data), options);
   558    })
   559  
   560    redraw();
   561  
   562    // redraw every 2.5 seconds.
   563    window.setInterval(redraw, 2500);
   564  }
   565  google.setOnLoadCallback(drawVReplicationQPSChart);
   566  </script>
   567  </table>{{else}}VReplication is closed.{{end}}
   568  `