Skip to content

Commit 1ed9209

Browse files
[release-2.2] #2525 spark metrics in depends on prometheus (#2543)
Signed-off-by: Hossein Torabi <[email protected]> Co-authored-by: Hossein <[email protected]>
1 parent 7e5b9ab commit 1ed9209

File tree

3 files changed

+44
-18
lines changed

3 files changed

+44
-18
lines changed

internal/controller/sparkapplication/monitoring_config.go

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ import (
3535

3636
func configPrometheusMonitoring(app *v1beta2.SparkApplication, client client.Client) error {
3737
port := common.DefaultPrometheusJavaAgentPort
38-
if app.Spec.Monitoring.Prometheus.Port != nil {
38+
if app.Spec.Monitoring.Prometheus != nil && app.Spec.Monitoring.Prometheus.Port != nil {
3939
port = *app.Spec.Monitoring.Prometheus.Port
4040
}
4141

@@ -63,12 +63,14 @@ func configPrometheusMonitoring(app *v1beta2.SparkApplication, client client.Cli
6363

6464
var javaOption string
6565

66-
javaOption = fmt.Sprintf(
67-
"-javaagent:%s=%d:%s/%s",
68-
app.Spec.Monitoring.Prometheus.JmxExporterJar,
69-
port,
70-
common.PrometheusConfigMapMountPath,
71-
common.PrometheusConfigKey)
66+
if app.Spec.Monitoring.Prometheus != nil {
67+
javaOption = fmt.Sprintf(
68+
"-javaagent:%s=%d:%s/%s",
69+
app.Spec.Monitoring.Prometheus.JmxExporterJar,
70+
port,
71+
common.PrometheusConfigMapMountPath,
72+
common.PrometheusConfigKey)
73+
}
7274

7375
if util.HasPrometheusConfigFile(app) {
7476
configFile := *app.Spec.Monitoring.Prometheus.ConfigFile
@@ -133,7 +135,7 @@ func buildPrometheusConfigMap(app *v1beta2.SparkApplication, prometheusConfigMap
133135
configMapData[common.MetricsPropertiesKey] = metricsProperties
134136
}
135137

136-
if !util.HasPrometheusConfigFile(app) {
138+
if app.Spec.Monitoring.Prometheus != nil && !util.HasPrometheusConfigFile(app) {
137139
prometheusConfig := common.DefaultPrometheusConfiguration
138140
if app.Spec.Monitoring.Prometheus.Configuration != nil {
139141
prometheusConfig = *app.Spec.Monitoring.Prometheus.Configuration

internal/controller/sparkapplication/monitoring_config_test.go

Lines changed: 32 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -56,30 +56,30 @@ func TestConfigPrometheusMonitoring(t *testing.T) {
5656
err = fakeClient.Get(context.TODO(), client.ObjectKeyFromObject(configMap), configMap)
5757
assert.NoError(t, err, "failed to get ConfigMap %s", configMapName)
5858

59-
if test.app.Spec.Monitoring.Prometheus.ConfigFile == nil &&
59+
if test.app.Spec.Monitoring.Prometheus != nil && test.app.Spec.Monitoring.Prometheus.ConfigFile == nil &&
6060
test.app.Spec.Monitoring.MetricsPropertiesFile == nil {
6161
assert.Len(t, configMap.Data, 2, "expected 2 data items")
6262
}
6363

64-
if test.app.Spec.Monitoring.Prometheus.ConfigFile != nil &&
64+
if test.app.Spec.Monitoring.Prometheus != nil && test.app.Spec.Monitoring.Prometheus.ConfigFile != nil &&
6565
test.app.Spec.Monitoring.MetricsPropertiesFile == nil {
6666
assert.Len(t, configMap.Data, 1, "expected 1 data item")
6767
}
6868

69-
if test.app.Spec.Monitoring.Prometheus.ConfigFile == nil &&
69+
if test.app.Spec.Monitoring.Prometheus != nil && test.app.Spec.Monitoring.Prometheus.ConfigFile == nil &&
7070
test.app.Spec.Monitoring.MetricsPropertiesFile != nil {
7171
assert.Len(t, configMap.Data, 1, "expected 1 data item")
7272
}
7373

74-
if test.app.Spec.Monitoring.MetricsPropertiesFile == nil {
74+
if test.app.Spec.Monitoring.Prometheus != nil && test.app.Spec.Monitoring.MetricsPropertiesFile == nil {
7575
assert.Equal(t, test.metricsProperties, configMap.Data[common.MetricsPropertiesKey], "metrics.properties mismatch")
7676
}
7777

78-
if test.app.Spec.Monitoring.Prometheus.ConfigFile == nil {
78+
if test.app.Spec.Monitoring.Prometheus != nil && test.app.Spec.Monitoring.Prometheus.ConfigFile == nil {
7979
assert.Equal(t, test.prometheusConfig, configMap.Data[common.PrometheusConfigKey], "prometheus.yaml mismatch")
8080
}
8181

82-
if test.app.Spec.Monitoring.ExposeDriverMetrics {
82+
if test.app.Spec.Monitoring.Prometheus != nil && test.app.Spec.Monitoring.ExposeDriverMetrics {
8383
assert.Len(t, test.app.Spec.Driver.Annotations, 3, "expected 3 driver annotations")
8484
assert.Equal(t, test.port, test.app.Spec.Driver.Annotations[common.PrometheusPortAnnotation], "java agent port mismatch")
8585
assert.Equal(t, test.driverJavaOptions, *test.app.Spec.Driver.JavaOptions, "driver Java options mismatch")
@@ -309,6 +309,32 @@ func TestConfigPrometheusMonitoring(t *testing.T) {
309309
driverJavaOptions: "-javaagent:/prometheus/exporter.jar=1000:/etc/metrics/conf/prometheus.yaml",
310310
executorJavaOptions: "-javaagent:/prometheus/exporter.jar=1000:/etc/metrics/conf/prometheus.yaml",
311311
},
312+
{
313+
app: &v1beta2.SparkApplication{
314+
ObjectMeta: metav1.ObjectMeta{
315+
Name: "app2",
316+
Namespace: "default",
317+
},
318+
Spec: v1beta2.SparkApplicationSpec{
319+
Driver: v1beta2.DriverSpec{
320+
JavaOptions: util.StringPtr("-XX:+PrintGCDetails -XX:+PrintGCTimeStamps"),
321+
},
322+
Executor: v1beta2.ExecutorSpec{
323+
JavaOptions: util.StringPtr("-XX:+PrintGCDetails -XX:+PrintGCTimeStamps"),
324+
},
325+
Monitoring: &v1beta2.MonitoringSpec{
326+
ExposeDriverMetrics: false,
327+
ExposeExecutorMetrics: false,
328+
MetricsProperties: util.StringPtr("testcase2dummy"),
329+
},
330+
},
331+
},
332+
metricsProperties: "testcase2dummy",
333+
prometheusConfig: "",
334+
port: "8090",
335+
driverJavaOptions: "-XX:+PrintGCDetails -XX:+PrintGCTimeStamps ",
336+
executorJavaOptions: "-XX:+PrintGCDetails -XX:+PrintGCTimeStamps ",
337+
},
312338
}
313339

314340
for _, test := range testcases {

pkg/util/sparkapplication.go

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -252,15 +252,13 @@ func HasPrometheusConfigFile(app *v1beta2.SparkApplication) bool {
252252

253253
// HasPrometheusConfig returns if Prometheus monitoring defines metricsProperties in the spec.
254254
func HasMetricsProperties(app *v1beta2.SparkApplication) bool {
255-
return PrometheusMonitoringEnabled(app) &&
256-
app.Spec.Monitoring.MetricsProperties != nil &&
255+
return app.Spec.Monitoring.MetricsProperties != nil &&
257256
*app.Spec.Monitoring.MetricsProperties != ""
258257
}
259258

260259
// HasPrometheusConfigFile returns if Monitoring defines metricsPropertiesFile in the spec.
261260
func HasMetricsPropertiesFile(app *v1beta2.SparkApplication) bool {
262-
return PrometheusMonitoringEnabled(app) &&
263-
app.Spec.Monitoring.MetricsPropertiesFile != nil &&
261+
return app.Spec.Monitoring.MetricsPropertiesFile != nil &&
264262
*app.Spec.Monitoring.MetricsPropertiesFile != ""
265263
}
266264

0 commit comments

Comments
 (0)