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

[CELEBORN-1800] Introduce ApplicationTotalCount and ApplicationFallbackCount metric to record the total and fallback count of application #3026

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
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
186 changes: 183 additions & 3 deletions assets/grafana/celeborn-dashboard.json
Original file line number Diff line number Diff line change
Expand Up @@ -1472,7 +1472,7 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "The total count of shuffle including celeborn shuffle and spark built-in shuffle.",
"description": "The total count of shuffle including celeborn shuffle and engine built-in shuffle.",
"fieldConfig": {
"defaults": {
"color": {
Expand Down Expand Up @@ -1652,7 +1652,7 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "The ApplyCompletedIndex of current master node in HA mode.",
"description": "The total count of application running with celeborn shuffle and engine built-in shuffle.",
"fieldConfig": {
"defaults": {
"color": {
Expand Down Expand Up @@ -1710,6 +1710,186 @@
"x": 0,
"y": 90
},
"id": 238,
"options": {
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom",
"showLegend": true
},
"tooltip": {
"maxHeight": 600,
"mode": "single",
"sort": "none"
}
},
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"expr": "metrics_ApplicationTotalCount_Value{instance=~\"${instance}\"}",
"legendFormat": "${baseLegend}",
"range": true,
"refId": "A"
}
],
"title": "metrics_ApplicationTotalCount_Value",
"type": "timeseries"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "The count of application fallbacks.",
"fieldConfig": {
"defaults": {
"color": {
"mode": "palette-classic"
},
"custom": {
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
"fillOpacity": 0,
"gradientMode": "none",
"hideFrom": {
"legend": false,
"tooltip": false,
"viz": false
},
"lineInterpolation": "linear",
"lineWidth": 1,
"pointSize": 5,
"scaleDistribution": {
"type": "linear"
},
"showPoints": "auto",
"spanNulls": false,
"stacking": {
"group": "A",
"mode": "none"
},
"thresholdsStyle": {
"mode": "off"
}
},
"mappings": [],
"thresholds": {
"mode": "absolute",
"steps": [
{
"color": "green"
}
]
}
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 12,
"x": 12,
"y": 90
},
"id": 239,
"options": {
"legend": {
"calcs": [],
"displayMode": "list",
"placement": "bottom",
"showLegend": true
},
"tooltip": {
"maxHeight": 600,
"mode": "single",
"sort": "none"
}
},
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"expr": "metrics_ApplicationFallbackCount_Value{instance=~\"${instance}\"}",
"legendFormat": "${baseLegend}",
"range": true,
"refId": "A"
}
],
"title": "metrics_ApplicationFallbackCount_Value",
"type": "timeseries"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"description": "The ApplyCompletedIndex of current master node in HA mode.",
"fieldConfig": {
"defaults": {
"color": {
"mode": "palette-classic"
},
"custom": {
"axisCenteredZero": false,
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
"fillOpacity": 0,
"gradientMode": "none",
"hideFrom": {
"legend": false,
"tooltip": false,
"viz": false
},
"lineInterpolation": "linear",
"lineWidth": 1,
"pointSize": 5,
"scaleDistribution": {
"type": "linear"
},
"showPoints": "auto",
"spanNulls": false,
"stacking": {
"group": "A",
"mode": "none"
},
"thresholdsStyle": {
"mode": "off"
}
},
"mappings": [],
"thresholds": {
"mode": "absolute",
"steps": [
{
"color": "green"
},
{
"color": "red",
"value": 80
}
]
}
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 12,
"x": 0,
"y": 98
},
"id": 118,
"options": {
"legend": {
Expand Down Expand Up @@ -1802,7 +1982,7 @@
"h": 8,
"w": 12,
"x": 12,
"y": 90
"y": 98
},
"id": 136,
"options": {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,13 +104,17 @@ public void registerJob(JobShuffleContext context) {
}
}

lifecycleManager.applicationCount().increment();
try {
if (nettyShuffleServiceFactory != null) {
Optional<ShuffleFallbackPolicy> shuffleFallbackPolicy =
ShuffleFallbackPolicyRunner.getActivatedFallbackPolicy(context, conf, lifecycleManager);
if (shuffleFallbackPolicy.isPresent()) {
LOG.warn("Fallback to vanilla Flink NettyShuffleMaster for job: {}.", jobID);
jobFallbackPolicies.put(jobID, shuffleFallbackPolicy.get().getClass().getName());
String jobFallbackPolicy = shuffleFallbackPolicy.get().getClass().getName();
jobFallbackPolicies.put(jobID, jobFallbackPolicy);
lifecycleManager.computeFallbackCounts(
lifecycleManager.applicationFallbackCounts(), jobFallbackPolicy);
nettyShuffleMaster().registerJob(context);
return;
}
Expand Down Expand Up @@ -158,9 +162,8 @@ public CompletableFuture<ShuffleDescriptor> registerPartitionWithProducer(
String jobFallbackPolicy = jobFallbackPolicies.get(jobID);
if (jobFallbackPolicy != null) {
try {
lifecycleManager
.shuffleFallbackCounts()
.compute(jobFallbackPolicy, (key, value) -> value == null ? 1L : value + 1L);
lifecycleManager.computeFallbackCounts(
lifecycleManager.shuffleFallbackCounts(), jobFallbackPolicy);
return nettyShuffleMaster()
.registerPartitionWithProducer(jobID, partitionDescriptor, producerDescriptor)
.get();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,7 @@ private void initializeLifecycleManager(String appId) {
if (lifecycleManager == null) {
appUniqueId = celebornConf.appUniqueIdWithUUIDSuffix(appId);
lifecycleManager = new LifecycleManager(appUniqueId, celebornConf);
lifecycleManager.applicationCount().increment();
lifecycleManager.registerCancelShuffleCallback(SparkUtils::cancelShuffle);
if (celebornConf.clientStageRerunEnabled()) {
MapOutputTrackerMaster mapOutputTracker =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.shuffle.celeborn

import java.util.concurrent.ConcurrentHashMap
import java.util.function.BiFunction

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -44,19 +45,27 @@ class CelebornShuffleFallbackPolicyRunner(conf: CelebornConf) extends Logging {
throw new CelebornIOException(
"Fallback to spark built-in shuffle implementation is prohibited.")
} else {
lifecycleManager.shuffleFallbackCounts.compute(
fallbackPolicy.get.getClass.getName,
new BiFunction[String, java.lang.Long, java.lang.Long] {
override def apply(k: String, v: java.lang.Long): java.lang.Long = {
if (v == null) {
1L
} else {
v + 1L
}
}
})
val shuffleFallbackPolicy = fallbackPolicy.get.getClass.getName
computeFallbackCounts(shuffleFallbackPolicy, lifecycleManager.shuffleFallbackCounts)
computeFallbackCounts(shuffleFallbackPolicy, lifecycleManager.applicationFallbackCounts)
}
}
fallbackPolicy.isDefined
}

private def computeFallbackCounts(
fallbackPolicy: String,
fallbackCounts: ConcurrentHashMap[String, java.lang.Long]) = {
fallbackCounts.compute(
fallbackPolicy,
new BiFunction[String, java.lang.Long, java.lang.Long] {
override def apply(k: String, v: java.lang.Long): java.lang.Long = {
if (v == null) {
1L
} else {
v + 1L
}
}
})
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,7 @@ private void initializeLifecycleManager(String appId) {
if (lifecycleManager == null) {
appUniqueId = celebornConf.appUniqueIdWithUUIDSuffix(appId);
lifecycleManager = new LifecycleManager(appUniqueId, celebornConf);
lifecycleManager.applicationCount().increment();
lifecycleManager.registerCancelShuffleCallback(SparkUtils::cancelShuffle);
if (celebornConf.clientStageRerunEnabled()) {
MapOutputTrackerMaster mapOutputTracker =
Expand Down Expand Up @@ -220,6 +221,7 @@ public ShuffleBlockResolver shuffleBlockResolver() {

@Override
public void stop() {
sortShuffleIds.clear();
if (shuffleClient != null) {
shuffleClient.shutdown();
ShuffleClient.reset();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.shuffle.celeborn

import java.util.concurrent.ConcurrentHashMap

import scala.collection.JavaConverters._

import org.apache.spark.ShuffleDependency
Expand All @@ -42,17 +44,25 @@ class CelebornShuffleFallbackPolicyRunner(conf: CelebornConf) extends Logging {
throw new CelebornIOException(
"Fallback to spark built-in shuffle implementation is prohibited.")
} else {
lifecycleManager.shuffleFallbackCounts.compute(
fallbackPolicy.get.getClass.getName,
(_, v) => {
if (v == null) {
1L
} else {
v + 1L
}
})
val shuffleFallbackPolicy = fallbackPolicy.get.getClass.getName
computeFallbackCounts(shuffleFallbackPolicy, lifecycleManager.shuffleFallbackCounts)
computeFallbackCounts(shuffleFallbackPolicy, lifecycleManager.applicationFallbackCounts)
}
}
fallbackPolicy.isDefined
}

private def computeFallbackCounts(
fallbackPolicy: String,
fallbackCounts: ConcurrentHashMap[String, java.lang.Long]) = {
fallbackCounts.compute(
fallbackPolicy,
(_, v) => {
if (v == null) {
1L
} else {
v + 1L
}
})
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,9 @@ class ApplicationHeartbeater(
appId: String,
conf: CelebornConf,
masterClient: MasterClient,
shuffleMetrics: () => ((Long, Long), (Long, Map[String, java.lang.Long])),
shuffleMetrics: () => (
(Long, Long),
(Long, Long, Map[String, java.lang.Long], Map[String, java.lang.Long])),
workerStatusTracker: WorkerStatusTracker,
registeredShuffles: ConcurrentHashMap.KeySetView[Int, java.lang.Boolean],
cancelAllActiveStages: String => Unit) extends Logging {
Expand All @@ -61,10 +63,15 @@ class ApplicationHeartbeater(
require(masterClient != null, "When sending a heartbeat, client shouldn't be null.")
val (
(tmpTotalWritten, tmpTotalFileCount),
(tmpShuffleCount, tmpShuffleFallbackCounts)) = shuffleMetrics()
(
tmpShuffleCount,
tmpApplicationCount,
tmpShuffleFallbackCounts,
tmpApplicationFallbackCounts)) = shuffleMetrics()
logInfo("Send app heartbeat with " +
s"written: ${Utils.bytesToString(tmpTotalWritten)}, file count: $tmpTotalFileCount, " +
s"shuffle count: $tmpShuffleCount, shuffle fallback counts: $tmpShuffleFallbackCounts")
s"shuffle count: $tmpShuffleCount, shuffle fallback counts: $tmpShuffleFallbackCounts, " +
s"application count: $tmpApplicationCount, application fallback counts: $tmpApplicationFallbackCounts")
// UserResourceConsumption and DiskInfo are eliminated from WorkerInfo
// during serialization of HeartbeatFromApplication
val appHeartbeat =
Expand All @@ -73,7 +80,9 @@ class ApplicationHeartbeater(
tmpTotalWritten,
tmpTotalFileCount,
tmpShuffleCount,
tmpApplicationCount,
tmpShuffleFallbackCounts.asJava,
tmpApplicationFallbackCounts.asJava,
workerStatusTracker.getNeedCheckedWorkers().toList.asJava,
ZERO_UUID,
true)
Expand Down
Loading
Loading