-
Notifications
You must be signed in to change notification settings - Fork 28.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-26285][CORE] accumulator metrics sources for LongAccumulator and Doub… #23242
Changes from 6 commits
45cfada
de531c8
fa45015
1fd7ebe
cf015cd
7f60730
682cfd7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You 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 | ||
* | ||
* http://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 org.apache.spark.metrics.source | ||
|
||
import com.codahale.metrics.{Gauge, MetricRegistry} | ||
|
||
import org.apache.spark.SparkContext | ||
import org.apache.spark.annotation.Experimental | ||
import org.apache.spark.util.{AccumulatorV2, DoubleAccumulator, LongAccumulator} | ||
|
||
/** | ||
* AccumulatorSource is a Spark metric Source that reports the current value | ||
* of the accumulator as a gauge. | ||
* | ||
* It is restricted to the LongAccumulator and the DoubleAccumulator, as those | ||
* are the current built-in numerical accumulators with Spark, and excludes | ||
* the CollectionAccumulator, as that is a List of values (hard to report, | ||
* to a metrics system) | ||
*/ | ||
private[spark] class AccumulatorSource extends Source { | ||
private val registry = new MetricRegistry | ||
protected def register[T](accumulators: Map[String, AccumulatorV2[_, T]]): Unit = { | ||
accumulators.foreach { | ||
case (name, accumulator) => | ||
val gauge = new Gauge[T] { | ||
override def getValue: T = accumulator.value | ||
} | ||
registry.register(MetricRegistry.name(name), gauge) | ||
} | ||
} | ||
|
||
override def sourceName: String = "AccumulatorSource" | ||
override def metricRegistry: MetricRegistry = registry | ||
} | ||
|
||
@Experimental | ||
class LongAccumulatorSource extends AccumulatorSource | ||
|
||
@Experimental | ||
class DoubleAccumulatorSource extends AccumulatorSource | ||
|
||
/** | ||
* :: Experimental :: | ||
* Metrics source specifically for LongAccumulators. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. perhaps we should expand this and say user can register accumulators that show up in the spark driver metrics |
||
* Register LongAccumulators using: | ||
* LongAccumulatorSource.register(sc, {"name" -> longAccumulator}) | ||
*/ | ||
@Experimental | ||
object LongAccumulatorSource { | ||
def register(sc: SparkContext, accumulators: Map[String, LongAccumulator]): Unit = { | ||
val source = new LongAccumulatorSource | ||
source.register(accumulators) | ||
sc.env.metricsSystem.registerSource(source) | ||
} | ||
} | ||
|
||
/** | ||
* :: Experimental :: | ||
* Metrics source specifically for DoubleAccumulators. | ||
* Register DoubleAccumulators using: | ||
* DoubleAccumulatorSource.register(sc, {"name" -> doubleAccumulator}) | ||
*/ | ||
@Experimental | ||
object DoubleAccumulatorSource { | ||
def register(sc: SparkContext, accumulators: Map[String, DoubleAccumulator]): Unit = { | ||
val source = new DoubleAccumulatorSource | ||
source.register(accumulators) | ||
sc.env.metricsSystem.registerSource(source) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You 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 | ||
* | ||
* http://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 org.apache.spark.metrics.source | ||
|
||
import com.codahale.metrics.MetricRegistry | ||
import org.mockito.ArgumentCaptor | ||
import org.mockito.Mockito.{mock, never, spy, times, verify, when} | ||
|
||
import org.apache.spark.{SparkContext, SparkEnv, SparkFunSuite} | ||
import org.apache.spark.metrics.MetricsSystem | ||
import org.apache.spark.util.{DoubleAccumulator, LongAccumulator} | ||
|
||
class AccumulatorSourceSuite extends SparkFunSuite { | ||
test("that that accumulators register against the metric system's register") { | ||
val acc1 = new LongAccumulator() | ||
val acc2 = new LongAccumulator() | ||
val mockContext = mock(classOf[SparkContext]) | ||
val mockEnvironment = mock(classOf[SparkEnv]) | ||
val mockMetricSystem = mock(classOf[MetricsSystem]) | ||
when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) | ||
when(mockContext.env) thenReturn (mockEnvironment) | ||
val accs = Map("my-accumulator-1" -> acc1, | ||
"my-accumulator-2" -> acc2) | ||
LongAccumulatorSource.register(mockContext, accs) | ||
val captor = new ArgumentCaptor[AccumulatorSource]() | ||
verify(mockMetricSystem, times(1)).registerSource(captor.capture()) | ||
val source = captor.getValue() | ||
val gauges = source.metricRegistry.getGauges() | ||
assert (gauges.size == 2) | ||
assert (gauges.firstKey == "my-accumulator-1") | ||
assert (gauges.lastKey == "my-accumulator-2") | ||
} | ||
|
||
test("the accumulators value property is checked when the gauge's value is requested") { | ||
val acc1 = new LongAccumulator() | ||
acc1.add(123) | ||
val acc2 = new LongAccumulator() | ||
acc2.add(456) | ||
val mockContext = mock(classOf[SparkContext]) | ||
val mockEnvironment = mock(classOf[SparkEnv]) | ||
val mockMetricSystem = mock(classOf[MetricsSystem]) | ||
when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) | ||
when(mockContext.env) thenReturn (mockEnvironment) | ||
val accs = Map("my-accumulator-1" -> acc1, | ||
"my-accumulator-2" -> acc2) | ||
LongAccumulatorSource.register(mockContext, accs) | ||
val captor = new ArgumentCaptor[AccumulatorSource]() | ||
verify(mockMetricSystem, times(1)).registerSource(captor.capture()) | ||
val source = captor.getValue() | ||
val gauges = source.metricRegistry.getGauges() | ||
assert(gauges.get("my-accumulator-1").getValue() == 123) | ||
assert(gauges.get("my-accumulator-2").getValue() == 456) | ||
} | ||
|
||
test("the double accumulators value propety is checked when the gauge's value is requested") { | ||
val acc1 = new DoubleAccumulator() | ||
acc1.add(123.123) | ||
val acc2 = new DoubleAccumulator() | ||
acc2.add(456.456) | ||
val mockContext = mock(classOf[SparkContext]) | ||
val mockEnvironment = mock(classOf[SparkEnv]) | ||
val mockMetricSystem = mock(classOf[MetricsSystem]) | ||
when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) | ||
when(mockContext.env) thenReturn (mockEnvironment) | ||
val accs = Map( | ||
"my-accumulator-1" -> acc1, | ||
"my-accumulator-2" -> acc2) | ||
DoubleAccumulatorSource.register(mockContext, accs) | ||
val captor = new ArgumentCaptor[AccumulatorSource]() | ||
verify(mockMetricSystem, times(1)).registerSource(captor.capture()) | ||
val source = captor.getValue() | ||
val gauges = source.metricRegistry.getGauges() | ||
assert(gauges.get("my-accumulator-1").getValue() == 123.123) | ||
assert(gauges.get("my-accumulator-2").getValue() == 456.456) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You 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 | ||
* | ||
* http://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. | ||
*/ | ||
|
||
// scalastyle:off println | ||
package org.apache.spark.examples | ||
|
||
import org.apache.spark.metrics.source.{DoubleAccumulatorSource, LongAccumulatorSource} | ||
import org.apache.spark.sql.SparkSession | ||
|
||
/** | ||
* Usage: AccumulatorMetricsTest [numElem] | ||
* | ||
* This example shows how to register accumulators against the accumulator source. | ||
* A simple RDD is created, and during the map, the accumulators are incremented. | ||
* | ||
* The only argument, numElem, sets the number elements in the collection to parallize. | ||
* | ||
* The result is output to stdout in the driver with the values of the accumulators. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. perhaps we should expand this more saying using ConsoleSink so the metrics going to stdout show up as metrics like AccumulatorSource.my-double-metric |
||
* For the long accumulator, it should equal numElem the double accumulator should be | ||
* roughly 1.1 x numElem (within double precision.) | ||
*/ | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. add a description on what this does |
||
object AccumulatorMetricsTest { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Example named as Test is a bit confusing i think... thoughts? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @redsanket yes at first I thought that, but there are other many other "examples" here with the suffix Test. |
||
def main(args: Array[String]) { | ||
|
||
val spark = SparkSession | ||
.builder() | ||
.config("spark.metrics.conf.*.sink.console.class", | ||
"org.apache.spark.metrics.sink.ConsoleSink") | ||
.getOrCreate() | ||
|
||
val sc = spark.sparkContext | ||
|
||
val acc = sc.longAccumulator("my-long-metric") | ||
LongAccumulatorSource.register(sc, List(("my-long-metric" -> acc)).toMap) | ||
|
||
val acc2 = sc.doubleAccumulator("my-double-metric") | ||
DoubleAccumulatorSource.register(sc, List(("my-double-metric" -> acc2)).toMap) | ||
|
||
val num = if (args.length > 0) args(0).toInt else 1000000 | ||
|
||
val startTime = System.nanoTime | ||
|
||
val accumulatorTest = sc.parallelize(1 to num).foreach(_ => { | ||
acc.add(1) | ||
acc2.add(1.1) | ||
}) | ||
|
||
// Print a footer with test time and accumulator values | ||
println("Test took %.0f milliseconds".format((System.nanoTime - startTime) / 1E6)) | ||
println("Accumulator values:") | ||
println("*** Long accumulator (my-long-metric): " + acc.value) | ||
println("*** Double accumulator (my-double-metric): " + acc2.value) | ||
|
||
spark.stop() | ||
} | ||
} | ||
// scalastyle:on println |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it might be good to mark all of these are Experimental for now, that gives us an out if we need to change something with api until we get some usage of it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also add small description for scala docs for Long and Double Accumulator Source