Add job start latency metrics and add namespace tag in metrics (#852)

* Add new metric for job start latency

* Add job latency histogram metric and namespace tag

Job start latency is defined as the time difference between when the
job is submitted by the user and when the job is in a running or any of
the terminal states. We use histogram with configurable boundaries
because users can provide different boundaries that they are interested
of. They can use one of them as their SLO/SLA and use the histogram
values to compute the percentage of number of jobs that meet the SLA. We
also added the namespace label into all the metrics when applicable when
the users specify it in the command line option. In addition, we fixed
the controller state machine diagram.

* Add start latency metrics doc, fix based on review

Added start latency summary and histogram metrics doc in
quick-start-guide.md. Added fixes based on the code review comments in
the PR.

Co-authored-by: Vaishnavi Giridaran <vgiridaran@salesforce.com>
This commit is contained in:
Shiqi Sun 2020-04-01 16:13:36 -04:00 committed by GitHub
parent 5afcce2919
commit 0e0867f0b4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 153 additions and 37 deletions

View File

@ -188,7 +188,9 @@ If enabled, the operator generates the following metrics:
| `spark_app_failure_count` | Total number of SparkApplication which failed to complete. |
| `spark_app_running_count` | Total number of SparkApplication which are currently running.|
| `spark_app_success_execution_time_microseconds` | Execution time for applications which succeeded.|
| `spark_app_failure_execution_time_microseconds` |Execution time for applications which failed. |
| `spark_app_failure_execution_time_microseconds` | Execution time for applications which failed. |
| `spark_app_start_latency_microseconds` | Start latency of SparkApplication as type of [Prometheus Summary](https://prometheus.io/docs/concepts/metric_types/#summary). |
| `spark_app_start_latency_seconds` | Start latency of SparkApplication as type of [Prometheus Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram). |
| `spark_app_executor_success_count` | Total number of Spark Executors which completed successfully. |
| `spark_app_executor_failure_count` | Total number of Spark Executors which failed. |
| `spark_app_executor_running_count` | Total number of Spark Executors which are currently running. |

26
main.go
View File

@ -60,10 +60,6 @@ var (
namespace = flag.String("namespace", apiv1.NamespaceAll, "The Kubernetes namespace to manage. Will manage custom resource objects of the managed CRD types for the whole cluster if unset.")
enableWebhook = flag.Bool("enable-webhook", false, "Whether to enable the mutating admission webhook for admitting and patching Spark pods.")
enableResourceQuotaEnforcement = flag.Bool("enable-resource-quota-enforcement", false, "Whether to enable ResourceQuota enforcement for SparkApplication resources. Requires the webhook to be enabled.")
enableMetrics = flag.Bool("enable-metrics", false, "Whether to enable the metrics endpoint.")
metricsPort = flag.String("metrics-port", "10254", "Port for the metrics endpoint.")
metricsEndpoint = flag.String("metrics-endpoint", "/metrics", "Metrics endpoint.")
metricsPrefix = flag.String("metrics-prefix", "", "Prefix for the metrics.")
ingressURLFormat = flag.String("ingress-url-format", "", "Ingress URL format.")
enableLeaderElection = flag.Bool("leader-election", false, "Enable Spark operator leader election.")
leaderElectionLockNamespace = flag.String("leader-election-lock-namespace", "spark-operator", "Namespace in which to create the ConfigMap for leader election.")
@ -71,13 +67,20 @@ var (
leaderElectionLeaseDuration = flag.Duration("leader-election-lease-duration", 15*time.Second, "Leader election lease duration.")
leaderElectionRenewDeadline = flag.Duration("leader-election-renew-deadline", 14*time.Second, "Leader election renew deadline.")
leaderElectionRetryPeriod = flag.Duration("leader-election-retry-period", 4*time.Second, "Leader election retry period.")
enableBatchScheduler = flag.Bool("enable-batch-scheduler", false,
fmt.Sprintf("Enable batch schedulers for pods' scheduling, the available batch schedulers are: (%s).", strings.Join(batchscheduler.GetRegisteredNames(), ",")))
enableBatchScheduler = flag.Bool("enable-batch-scheduler", false, fmt.Sprintf("Enable batch schedulers for pods' scheduling, the available batch schedulers are: (%s).", strings.Join(batchscheduler.GetRegisteredNames(), ",")))
enableMetrics = flag.Bool("enable-metrics", false, "Whether to enable the metrics endpoint.")
metricsPort = flag.String("metrics-port", "10254", "Port for the metrics endpoint.")
metricsEndpoint = flag.String("metrics-endpoint", "/metrics", "Metrics endpoint.")
metricsPrefix = flag.String("metrics-prefix", "", "Prefix for the metrics.")
metricsLabels util.ArrayFlags
metricsJobStartLatencyBuckets util.HistogramBuckets = util.DefaultJobStartLatencyBuckets
)
func main() {
var metricsLabels util.ArrayFlags
flag.Var(&metricsLabels, "metrics-labels", "Labels for the metrics")
flag.Var(&metricsJobStartLatencyBuckets, "metrics-job-start-latency-buckets",
"Comma-separated boundary values (in seconds) for the job start latency histogram bucket; "+
"it accepts any numerical values that can be parsed into a 64-bit floating point")
flag.Parse()
// Create the client config. Use kubeConfig if given, otherwise assume in-cluster.
@ -159,10 +162,11 @@ func main() {
var metricConfig *util.MetricConfig
if *enableMetrics {
metricConfig = &util.MetricConfig{
MetricsEndpoint: *metricsEndpoint,
MetricsPort: *metricsPort,
MetricsPrefix: *metricsPrefix,
MetricsLabels: metricsLabels,
MetricsEndpoint: *metricsEndpoint,
MetricsPort: *metricsPort,
MetricsPrefix: *metricsPrefix,
MetricsLabels: metricsLabels,
MetricsJobStartLatencyBuckets: metricsJobStartLatencyBuckets,
}
glog.Info("Enabling metrics collecting and exporting to Prometheus")

View File

@ -122,7 +122,7 @@ func newSparkApplicationController(
}
if metricsConfig != nil {
controller.metrics = newSparkAppMetrics(metricsConfig.MetricsPrefix, metricsConfig.MetricsLabels)
controller.metrics = newSparkAppMetrics(metricsConfig)
controller.metrics.registerMetrics()
}
@ -455,20 +455,20 @@ func shouldRetry(app *v1beta2.SparkApplication) bool {
// State Machine for SparkApplication:
//+--------------------------------------------------------------------------------------------------------------------+
//| |
//| +---------+ |
//| | | |
//| | + |
//| |Submission |
//| +----> Failed +-----+------------------------------------------------------------------+ |
//| | | | | | |
//| | | | | | |
//| | +----^----+ | | |
//| | | | | |
//| | | | | |
//| +----+----+ | +-----v----+ +----------+ +-----------+ +----v-----+ |
//| +---------------------------------------------------------------------------------------------+ |
//| | +----------+ | |
//| | | | | |
//| | | | | |
//| | |Submission| | |
//| | +----> Failed +----+------------------------------------------------------------------+ | |
//| | | | | | | | |
//| | | | | | | | |
//| | | +----^-----+ | +-----------------------------------------+ | | |
//| | | | | | | | | |
//| | | | | | | | | |
//| +-+--+----+ | +-----v--+-+ +----------+ +-----v-----+ +----v--v--+ |
//| | | | | | | | | | | | |
//| | | | | | | | | | | | |
//| | | | | | | | | | | | |
//| | New +---------> Submitted+----------> Running +-----------> Failing +----------> Failed | |
//| | | | | | | | | | | | |
//| | | | | | | | | | | | |

View File

@ -36,15 +36,19 @@ type sparkAppMetrics struct {
sparkAppFailedSubmissionCount *prometheus.CounterVec
sparkAppRunningCount *util.PositiveGauge
sparkAppSuccessExecutionTime *prometheus.SummaryVec
sparkAppFailureExecutionTime *prometheus.SummaryVec
sparkAppSuccessExecutionTime *prometheus.SummaryVec
sparkAppFailureExecutionTime *prometheus.SummaryVec
sparkAppStartLatency *prometheus.SummaryVec
sparkAppStartLatencyHistogram *prometheus.HistogramVec
sparkAppExecutorRunningCount *util.PositiveGauge
sparkAppExecutorFailureCount *prometheus.CounterVec
sparkAppExecutorSuccessCount *prometheus.CounterVec
}
func newSparkAppMetrics(prefix string, labels []string) *sparkAppMetrics {
func newSparkAppMetrics(metricsConfig *util.MetricConfig) *sparkAppMetrics {
prefix := metricsConfig.MetricsPrefix
labels := metricsConfig.MetricsLabels
validLabels := make([]string, len(labels))
for i, label := range labels {
validLabels[i] = util.CreateValidMetricNameLabel("", label)
@ -92,6 +96,21 @@ func newSparkAppMetrics(prefix string, labels []string) *sparkAppMetrics {
},
validLabels,
)
sparkAppStartLatency := prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: util.CreateValidMetricNameLabel(prefix, "spark_app_start_latency_microseconds"),
Help: "Spark App Start Latency via the Operator",
},
validLabels,
)
sparkAppStartLatencyHistogram := prometheus.NewHistogramVec(
prometheus.HistogramOpts{
Name: util.CreateValidMetricNameLabel(prefix, "spark_app_start_latency_seconds"),
Help: "Spark App Start Latency counts in buckets via the Operator",
Buckets: metricsConfig.MetricsJobStartLatencyBuckets,
},
validLabels,
)
sparkAppExecutorSuccessCount := prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: util.CreateValidMetricNameLabel(prefix, "spark_app_executor_success_count"),
@ -121,6 +140,8 @@ func newSparkAppMetrics(prefix string, labels []string) *sparkAppMetrics {
sparkAppFailedSubmissionCount: sparkAppFailedSubmissionCount,
sparkAppSuccessExecutionTime: sparkAppSuccessExecutionTime,
sparkAppFailureExecutionTime: sparkAppFailureExecutionTime,
sparkAppStartLatency: sparkAppStartLatency,
sparkAppStartLatencyHistogram: sparkAppStartLatencyHistogram,
sparkAppExecutorRunningCount: sparkAppExecutorRunningCount,
sparkAppExecutorSuccessCount: sparkAppExecutorSuccessCount,
sparkAppExecutorFailureCount: sparkAppExecutorFailureCount,
@ -133,6 +154,8 @@ func (sm *sparkAppMetrics) registerMetrics() {
util.RegisterMetric(sm.sparkAppFailureCount)
util.RegisterMetric(sm.sparkAppSuccessExecutionTime)
util.RegisterMetric(sm.sparkAppFailureExecutionTime)
util.RegisterMetric(sm.sparkAppStartLatency)
util.RegisterMetric(sm.sparkAppStartLatencyHistogram)
util.RegisterMetric(sm.sparkAppExecutorSuccessCount)
util.RegisterMetric(sm.sparkAppExecutorFailureCount)
sm.sparkAppRunningCount.Register()
@ -140,7 +163,7 @@ func (sm *sparkAppMetrics) registerMetrics() {
}
func (sm *sparkAppMetrics) exportMetrics(oldApp, newApp *v1beta2.SparkApplication) {
metricLabels := fetchMetricLabels(newApp.Labels, sm.labels)
metricLabels := fetchMetricLabels(newApp, sm.labels)
glog.V(2).Infof("Exporting metrics for %s; old status: %v new status: %v", newApp.Name,
oldApp.Status, newApp.Status)
@ -156,6 +179,7 @@ func (sm *sparkAppMetrics) exportMetrics(oldApp, newApp *v1beta2.SparkApplicatio
}
case v1beta2.RunningState:
sm.sparkAppRunningCount.Inc(metricLabels)
sm.exportJobStartLatencyMetrics(newApp, metricLabels)
case v1beta2.SucceedingState:
if !newApp.Status.LastSubmissionAttemptTime.Time.IsZero() && !newApp.Status.TerminationTime.Time.IsZero() {
d := newApp.Status.TerminationTime.Time.Sub(newApp.Status.LastSubmissionAttemptTime.Time)
@ -196,6 +220,18 @@ func (sm *sparkAppMetrics) exportMetrics(oldApp, newApp *v1beta2.SparkApplicatio
}
}
// In the event that state transitions happened too quickly and the spark app skipped the RUNNING state, the job
// start latency should still be captured.
// Note: There is an edge case that a Submitted state can go directly to a Failing state if the driver pod is
// deleted. This is very unlikely if not being done intentionally, so we choose not to handle it.
if newState != oldState {
if (newState == v1beta2.FailingState || newState == v1beta2.SucceedingState) && oldState == v1beta2.SubmittedState {
// TODO: remove this log once we've gathered some data in prod fleets.
glog.V(2).Infof("Calculating job start latency metrics for edge case transition from %v to %v in app %v in namespace %v.", oldState, newState, newApp.Name, newApp.Namespace)
sm.exportJobStartLatencyMetrics(newApp, metricLabels)
}
}
// Potential Executor status updates
for executor, newExecState := range newApp.Status.ExecutorState {
switch newExecState {
@ -231,7 +267,27 @@ func (sm *sparkAppMetrics) exportMetrics(oldApp, newApp *v1beta2.SparkApplicatio
}
}
func fetchMetricLabels(specLabels map[string]string, labels []string) map[string]string {
func (sm *sparkAppMetrics) exportJobStartLatencyMetrics(app *v1beta2.SparkApplication, labels map[string]string) {
// Expose the job start latency related metrics of an SparkApp only once when it runs for the first time
if app.Status.ExecutionAttempts == 1 {
latency := time.Now().Sub(app.CreationTimestamp.Time)
if m, err := sm.sparkAppStartLatency.GetMetricWith(labels); err != nil {
glog.Errorf("Error while exporting metrics: %v", err)
} else {
m.Observe(float64(latency / time.Microsecond))
}
if m, err := sm.sparkAppStartLatencyHistogram.GetMetricWith(labels); err != nil {
glog.Errorf("Error while exporting metrics: %v", err)
} else {
m.Observe(float64(latency / time.Second))
}
}
}
func fetchMetricLabels(app *v1beta2.SparkApplication, labels []string) map[string]string {
specLabels := app.Labels
// Transform spec labels since our labels names might be not same as specLabels if we removed invalid characters.
validSpecLabels := make(map[string]string)
for labelKey, v := range specLabels {
@ -243,6 +299,8 @@ func fetchMetricLabels(specLabels map[string]string, labels []string) map[string
for _, label := range labels {
if value, ok := validSpecLabels[label]; ok {
metricLabels[label] = value
} else if label == "namespace" { // if the "namespace" label is in the metrics config, use it
metricLabels[label] = app.Namespace
} else {
metricLabels[label] = "Unknown"
}

View File

@ -17,6 +17,7 @@ limitations under the License.
package sparkapplication
import (
"github.com/GoogleCloudPlatform/spark-on-k8s-operator/pkg/util"
"net/http"
"sync"
"testing"
@ -27,8 +28,13 @@ import (
func TestSparkAppMetrics(t *testing.T) {
http.DefaultServeMux = new(http.ServeMux)
// Test with label containing "-". Expect them to be converted to "_".
metrics := newSparkAppMetrics("", []string{"app-id"})
app1 := map[string]string{"app_id": "test1"}
metricsConfig := &util.MetricConfig{
MetricsPrefix: "",
MetricsLabels: []string{"app-id", "namespace"},
MetricsJobStartLatencyBuckets: []float64{30, 60, 90, 120},
}
metrics := newSparkAppMetrics(metricsConfig)
app1 := map[string]string{"app_id": "test1", "namespace": "default"}
var wg sync.WaitGroup
wg.Add(1)
@ -41,6 +47,8 @@ func TestSparkAppMetrics(t *testing.T) {
metrics.sparkAppFailedSubmissionCount.With(app1).Inc()
metrics.sparkAppSuccessExecutionTime.With(app1).Observe(float64(100 * i))
metrics.sparkAppFailureExecutionTime.With(app1).Observe(float64(500 * i))
metrics.sparkAppStartLatency.With(app1).Observe(float64(10 * i))
metrics.sparkAppStartLatencyHistogram.With(app1).Observe(float64(10 * i))
metrics.sparkAppExecutorRunningCount.Inc(app1)
metrics.sparkAppExecutorSuccessCount.With(app1).Inc()
metrics.sparkAppExecutorFailureCount.With(app1).Inc()

View File

@ -0,0 +1,43 @@
/*
Copyright 2018 Google LLC
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
https://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package util
import (
"fmt"
"strconv"
"strings"
)
var DefaultJobStartLatencyBuckets = []float64{30, 60, 90, 120, 150, 180, 210, 240, 270, 300}
type HistogramBuckets []float64
func (hb *HistogramBuckets) String() string {
return fmt.Sprint(*hb)
}
func (hb *HistogramBuckets) Set(value string) error {
*hb = nil
for _, boundaryStr := range strings.Split(value, ",") {
boundary, err := strconv.ParseFloat(strings.TrimSpace(boundaryStr), 64)
if err != nil {
return err
}
*hb = append(*hb, boundary)
}
return nil
}

View File

@ -49,10 +49,11 @@ func RegisterMetric(metric prometheus.Collector) {
// MetricConfig is a container of configuration properties for the collection and exporting of
// application metrics to Prometheus.
type MetricConfig struct {
MetricsEndpoint string
MetricsPort string
MetricsPrefix string
MetricsLabels []string
MetricsEndpoint string
MetricsPort string
MetricsPrefix string
MetricsLabels []string
MetricsJobStartLatencyBuckets []float64
}
// A variant of Prometheus Gauge that only holds non-negative values.