Skip to content
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

Closed
Closed
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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.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)
*/
class AccumulatorSource extends Source {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should make this private[spark] for now unless you have a use case that would need to public extend this

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
}

class LongAccumulatorSource extends AccumulatorSource
Copy link
Contributor

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.

Copy link
Contributor

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

class DoubleAccumulatorSource extends AccumulatorSource

object LongAccumulatorSource {
def register(sc: SparkContext, accumulators: Map[String, LongAccumulator]): Unit = {
val source = new LongAccumulatorSource
source.register(accumulators)
sc.env.metricsSystem.registerSource(source)
}
}

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,92 @@
/*
* 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,68 @@
/*
* 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 [partitions] [numElem] [blockSize]
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add a description on what this does

object AccumulatorMetricsTest {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Example named as Test is a bit confusing i think... thoughts?

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 blockSize = if (args.length > 2) args(2) else "4096"

val spark = SparkSession
.builder()
.config("spark.broadcast.blockSize", blockSize)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we don't need this, assume copy from the BroadcastTest one

.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 slices = if (args.length > 0) args(0).toInt else 2
val num = if (args.length > 1) args(1).toInt else 1000000

val arr1 = (0 until num).toArray

for (i <- 0 until 3) {
println(s"Iteration $i")
println("===========")
val startTime = System.nanoTime
val barr1 = sc.broadcast(arr1)
val observedSizes = sc.parallelize(1 to 10, slices).map(_ => {
acc.add(1)
acc2.add(1.1)
barr1.value.length
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we could simplify the test by remove the broadcast stuff as that isn't really what we are testing, just leave the accumulator updates and do simple map. Print the accumulators at the end. Also perhaps we should put more information the register part and how someone running this might see those metrics output

})
// Collect the small RDD so we can print the observed sizes locally.
observedSizes.repartition(100).collect().foreach(i => println(i))
println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6))
}

spark.stop()
}
}
// scalastyle:on println