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-18364][YARN] Expose metrics for YarnShuffleService #22485

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.server.api.*;
import org.apache.spark.network.util.LevelDBProvider;
Expand Down Expand Up @@ -168,6 +170,15 @@ protected void serviceInit(Configuration conf) throws Exception {
TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf));
blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile);

// register metrics on the block handler into the Node Manager's metrics system.
YarnShuffleServiceMetrics serviceMetrics =
new YarnShuffleServiceMetrics(blockHandler.getAllMetrics());

MetricsSystemImpl metricsSystem = (MetricsSystemImpl) DefaultMetricsSystem.instance();
metricsSystem.register(
"sparkShuffleService", "Metrics on the Spark Shuffle Service", serviceMetrics);
logger.info("Registered metrics with Hadoop's DefaultMetricsSystem");

// If authentication is enabled, set up the shuffle server to use a
// special RPC handler that filters out unauthenticated fetch requests
List<TransportServerBootstrap> bootstraps = Lists.newArrayList();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
/*
* 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.network.yarn;

import java.util.Map;

import com.codahale.metrics.*;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.MetricsSource;
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 newline between hadoop and spark

Copy link
Contributor Author

Choose a reason for hiding this comment

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

spark import removed because of checkstyle wont allow to have import in javadoc when you have {@link className} syntax. I think checkstyle rules should be corrected.


/**
* Forward {@link org.apache.spark.network.shuffle.ExternalShuffleBlockHandler.ShuffleMetrics}
* to hadoop metrics system.
* NodeManager by default exposes JMX endpoint where can be collected.
*/
class YarnShuffleServiceMetrics implements MetricsSource {

private final MetricSet metricSet;

YarnShuffleServiceMetrics(MetricSet metricSet) {
this.metricSet = metricSet;
}

/**
* Get metrics from the source
*
* @param collector to contain the resulting metrics snapshot
* @param all if true, return all metrics even if unchanged.
*/
@Override
public void getMetrics(MetricsCollector collector, boolean all) {
MetricsRecordBuilder metricsRecordBuilder = collector.addRecord("sparkShuffleService");

for (Map.Entry<String, Metric> entry : metricSet.getMetrics().entrySet()) {
collectMetric(metricsRecordBuilder, entry.getKey(), entry.getValue());
}
}

/**
* The metric types used in
* {@link org.apache.spark.network.shuffle.ExternalShuffleBlockHandler.ShuffleMetrics}.
* Visible for testing.
*/
public static void collectMetric(
MetricsRecordBuilder metricsRecordBuilder, String name, Metric metric) {

if (metric instanceof Timer) {
Timer t = (Timer) metric;
metricsRecordBuilder
.addCounter(new ShuffleServiceMetricsInfo(name + "_count", "Count of timer " + name),
t.getCount())
.addGauge(
new ShuffleServiceMetricsInfo(name + "_rate15", "15 minute rate of timer " + name),
t.getFifteenMinuteRate())
.addGauge(
new ShuffleServiceMetricsInfo(name + "_rate5", "5 minute rate of timer " + name),
t.getFiveMinuteRate())
.addGauge(
new ShuffleServiceMetricsInfo(name + "_rate1", "1 minute rate of timer " + name),
t.getOneMinuteRate())
.addGauge(new ShuffleServiceMetricsInfo(name + "_rateMean", "Mean rate of timer " + name),
t.getMeanRate());
} else if (metric instanceof Meter) {
Meter m = (Meter) metric;
metricsRecordBuilder
.addCounter(new ShuffleServiceMetricsInfo(name + "_count", "Count of meter " + name),
m.getCount())
.addGauge(
new ShuffleServiceMetricsInfo(name + "_rate15", "15 minute rate of meter " + name),
m.getFifteenMinuteRate())
.addGauge(
new ShuffleServiceMetricsInfo(name + "_rate5", "5 minute rate of meter " + name),
m.getFiveMinuteRate())
.addGauge(
new ShuffleServiceMetricsInfo(name + "_rate1", "1 minute rate of meter " + name),
m.getOneMinuteRate())
.addGauge(new ShuffleServiceMetricsInfo(name + "_rateMean", "Mean rate of meter " + name),
m.getMeanRate());
} else if (metric instanceof Gauge) {
final Object gaugeValue = ((Gauge) metric).getValue();
if (gaugeValue instanceof Integer) {
metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Integer) gaugeValue);
} else if (gaugeValue instanceof Long) {
metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Long) gaugeValue);
} else if (gaugeValue instanceof Float) {
metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Float) gaugeValue);
} else if (gaugeValue instanceof Double) {
metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Double) gaugeValue);
} else {
throw new IllegalStateException(
"Not supported class type of metric[" + name + "] for value " + gaugeValue);
}
}
}

private static MetricsInfo getShuffleServiceMetricsInfo(String name) {
return new ShuffleServiceMetricsInfo(name, "Value of gauge " + name);
}

private static class ShuffleServiceMetricsInfo implements MetricsInfo {

private final String name;
private final String description;

ShuffleServiceMetricsInfo(String name, String description) {
this.name = name;
this.description = description;
}

@Override
public String name() {
return name;
}

@Override
public String description() {
return description;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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.network.yarn

import scala.collection.JavaConverters._

import org.apache.hadoop.metrics2.MetricsRecordBuilder
import org.mockito.Matchers._
import org.mockito.Mockito.{mock, times, verify, when}
import org.scalatest.Matchers

import org.apache.spark.SparkFunSuite
import org.apache.spark.network.server.OneForOneStreamManager
import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleBlockResolver}

class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers {

val streamManager = mock(classOf[OneForOneStreamManager])
val blockResolver = mock(classOf[ExternalShuffleBlockResolver])
when(blockResolver.getRegisteredExecutorsSize).thenReturn(42)

val metrics = new ExternalShuffleBlockHandler(streamManager, blockResolver).getAllMetrics

test("metrics named as expected") {
val allMetrics = Set(
"openBlockRequestLatencyMillis", "registerExecutorRequestLatencyMillis",
"blockTransferRateBytes", "registeredExecutorsSize")

metrics.getMetrics.keySet().asScala should be (allMetrics)
}

// these three metrics have the same effect on the collector
for (testname <- Seq("openBlockRequestLatencyMillis",
"registerExecutorRequestLatencyMillis",
"blockTransferRateBytes")) {
test(s"$testname - collector receives correct types") {
val builder = mock(classOf[MetricsRecordBuilder])
when(builder.addCounter(any(), anyLong())).thenReturn(builder)
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't see where these when's are used?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

these metrics are Timer or Meter so its called in YarnShuffleServiceMetrics.collectMetrics.
https://github.com/apache/spark/pull/22485/files#diff-04b0327ae6bce4204c03442b0e6d8718R63

when(builder.addGauge(any(), anyDouble())).thenReturn(builder)

YarnShuffleServiceMetrics.collectMetric(builder, testname,
Copy link
Contributor

Choose a reason for hiding this comment

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

you are calling a private function here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

metrics.getMetrics.get(testname))

verify(builder).addCounter(anyObject(), anyLong())
verify(builder, times(4)).addGauge(anyObject(), anyDouble())
}
}

// this metric writes only one gauge to the collector
test("registeredExecutorsSize - collector receives correct types") {
val builder = mock(classOf[MetricsRecordBuilder])

YarnShuffleServiceMetrics.collectMetric(builder, "registeredExecutorsSize",
metrics.getMetrics.get("registeredExecutorsSize"))

// only one
verify(builder).addGauge(anyObject(), anyInt())
}
}