Skip to content

Commit

Permalink
perf: improve performance of update metrics (#1329)
Browse files Browse the repository at this point in the history
  • Loading branch information
wForget authored Feb 8, 2025
1 parent 26b8d57 commit a1e6a39
Show file tree
Hide file tree
Showing 12 changed files with 142 additions and 81 deletions.
8 changes: 8 additions & 0 deletions common/src/main/scala/org/apache/comet/CometConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -632,6 +632,14 @@ object CometConf extends ShimCometConf {
.booleanConf
.createWithDefault(false)

val COMET_METRICS_UPDATE_INTERVAL: ConfigEntry[Long] =
conf("spark.comet.metrics.updateInterval")
.doc(
"The interval in milliseconds to update metrics. If interval is negative," +
" metrics will be updated upon task completion.")
.longConf
.createWithDefault(3000L)

/** Create a config to enable a specific operator */
private def createExecEnabledConfig(
exec: String,
Expand Down
1 change: 1 addition & 0 deletions docs/source/user-guide/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ Comet provides the following configuration settings.
| spark.comet.expression.allowIncompatible | Comet is not currently fully compatible with Spark for all expressions. Set this config to true to allow them anyway. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html). | false |
| spark.comet.memory.overhead.factor | Fraction of executor memory to be allocated as additional non-heap memory per executor process for Comet. | 0.2 |
| spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB. | 402653184b |
| spark.comet.metrics.updateInterval | The interval in milliseconds to update metrics. If interval is negative, metrics will be updated upon task completion. | 3000 |
| spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false |
| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. | false |
| spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. | false |
Expand Down
43 changes: 33 additions & 10 deletions native/core/src/execution/jni_api.rs
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,10 @@ struct ExecutionContext {
pub runtime: Runtime,
/// Native metrics
pub metrics: Arc<GlobalRef>,
// The interval in milliseconds to update metrics
pub metrics_update_interval: Option<Duration>,
// The last update time of metrics
pub metrics_last_update_time: Instant,
/// The time it took to create the native plan and configure the context
pub plan_creation_time: Duration,
/// DataFusion SessionContext
Expand All @@ -99,8 +103,6 @@ struct ExecutionContext {
pub debug_native: bool,
/// Whether to write native plans with metrics to stdout
pub explain_native: bool,
/// Map of metrics name -> jstring object to cache jni_NewStringUTF calls.
pub metrics_jstrings: HashMap<String, Arc<GlobalRef>>,
/// Memory pool config
pub memory_pool_config: MemoryPoolConfig,
}
Expand Down Expand Up @@ -160,6 +162,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
serialized_query: jbyteArray,
partition_count: jint,
metrics_node: JObject,
metrics_update_interval: jlong,
comet_task_memory_manager_obj: JObject,
batch_size: jint,
use_unified_memory_manager: jboolean,
Expand Down Expand Up @@ -222,6 +225,12 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(

let plan_creation_time = start.elapsed();

let metrics_update_interval = if metrics_update_interval > 0 {
Some(Duration::from_millis(metrics_update_interval as u64))
} else {
None
};

let exec_context = Box::new(ExecutionContext {
id,
task_attempt_id,
Expand All @@ -233,11 +242,12 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
stream: None,
runtime,
metrics,
metrics_update_interval,
metrics_last_update_time: Instant::now(),
plan_creation_time,
session_ctx: Arc::new(session),
debug_native: debug_native == 1,
explain_native: explain_native == 1,
metrics_jstrings: HashMap::new(),
memory_pool_config,
});

Expand Down Expand Up @@ -508,8 +518,14 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_executePlan(
let next_item = exec_context.stream.as_mut().unwrap().next();
let poll_output = exec_context.runtime.block_on(async { poll!(next_item) });

// Update metrics
update_metrics(&mut env, exec_context)?;
// update metrics at interval
if let Some(interval) = exec_context.metrics_update_interval {
let now = Instant::now();
if now - exec_context.metrics_last_update_time >= interval {
update_metrics(&mut env, exec_context)?;
exec_context.metrics_last_update_time = now;
}
}

match poll_output {
Poll::Ready(Some(output)) => {
Expand Down Expand Up @@ -561,8 +577,12 @@ pub extern "system" fn Java_org_apache_comet_Native_releasePlan(
_class: JClass,
exec_context: jlong,
) {
try_unwrap_or_throw(&e, |_| unsafe {
try_unwrap_or_throw(&e, |mut env| unsafe {
let execution_context = get_execution_context(exec_context);

// Update metrics
update_metrics(&mut env, execution_context)?;

if execution_context.memory_pool_config.pool_type == MemoryPoolType::FairSpillTaskShared
|| execution_context.memory_pool_config.pool_type == MemoryPoolType::GreedyTaskShared
{
Expand All @@ -586,10 +606,13 @@ pub extern "system" fn Java_org_apache_comet_Native_releasePlan(

/// Updates the metrics of the query plan.
fn update_metrics(env: &mut JNIEnv, exec_context: &mut ExecutionContext) -> CometResult<()> {
let native_query = exec_context.root_op.as_ref().unwrap();
let metrics = exec_context.metrics.as_obj();
let metrics_jstrings = &mut exec_context.metrics_jstrings;
update_comet_metric(env, metrics, native_query, metrics_jstrings)
if exec_context.root_op.is_some() {
let native_query = exec_context.root_op.as_ref().unwrap();
let metrics = exec_context.metrics.as_obj();
update_comet_metric(env, metrics, native_query)
} else {
Ok(())
}
}

fn convert_datatype_arrays(
Expand Down
94 changes: 32 additions & 62 deletions native/core/src/execution/metrics/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,14 +16,11 @@
// under the License.

use crate::execution::spark_plan::SparkPlan;
use crate::jvm_bridge::jni_new_global_ref;
use crate::{
errors::CometError,
jvm_bridge::{jni_call, jni_new_string},
};
use crate::{errors::CometError, jvm_bridge::jni_call};
use datafusion::physical_plan::metrics::MetricValue;
use jni::objects::{GlobalRef, JString};
use datafusion_comet_proto::spark_metric::NativeMetricNode;
use jni::{objects::JObject, JNIEnv};
use prost::Message;
use std::collections::HashMap;
use std::sync::Arc;

Expand All @@ -34,10 +31,22 @@ pub fn update_comet_metric(
env: &mut JNIEnv,
metric_node: &JObject,
spark_plan: &Arc<SparkPlan>,
metrics_jstrings: &mut HashMap<String, Arc<GlobalRef>>,
) -> Result<(), CometError> {
// combine all metrics from all native plans for this SparkPlan
let metrics = if spark_plan.additional_native_plans.is_empty() {
unsafe {
let native_metric = to_native_metric_node(spark_plan);
let jbytes = env.byte_array_from_slice(&native_metric?.encode_to_vec())?;
jni_call!(env, comet_metric_node(metric_node).set_all_from_bytes(&jbytes) -> ())?;
}
Ok(())
}

pub fn to_native_metric_node(spark_plan: &Arc<SparkPlan>) -> Result<NativeMetricNode, CometError> {
let mut native_metric_node = NativeMetricNode {
metrics: HashMap::new(),
children: Vec::new(),
};

let node_metrics = if spark_plan.additional_native_plans.is_empty() {
spark_plan.native_plan.metrics()
} else {
let mut metrics = spark_plan.native_plan.metrics().unwrap_or_default();
Expand All @@ -55,60 +64,21 @@ pub fn update_comet_metric(
Some(metrics.aggregate_by_name())
};

update_metrics(
env,
metric_node,
&metrics
.unwrap_or_default()
.iter()
.map(|m| m.value())
.map(|m| (m.name(), m.as_usize() as i64))
.collect::<Vec<_>>(),
metrics_jstrings,
)?;
// add metrics
node_metrics
.unwrap_or_default()
.iter()
.map(|m| m.value())
.map(|m| (m.name(), m.as_usize() as i64))
.for_each(|(name, value)| {
native_metric_node.metrics.insert(name.to_string(), value);
});

unsafe {
for (i, child_plan) in spark_plan.children().iter().enumerate() {
let child_metric_node: JObject = jni_call!(env,
comet_metric_node(metric_node).get_child_node(i as i32) -> JObject
)?;
if child_metric_node.is_null() {
continue;
}
update_comet_metric(env, &child_metric_node, child_plan, metrics_jstrings)?;
}
// add children
for child_plan in spark_plan.children() {
let child_node = to_native_metric_node(child_plan)?;
native_metric_node.children.push(child_node);
}
Ok(())
}

#[inline]
fn update_metrics(
env: &mut JNIEnv,
metric_node: &JObject,
metric_values: &[(&str, i64)],
metrics_jstrings: &mut HashMap<String, Arc<GlobalRef>>,
) -> Result<(), CometError> {
unsafe {
for &(name, value) in metric_values {
// Perform a lookup in the jstrings cache.
if let Some(map_global_ref) = metrics_jstrings.get(name) {
// Cache hit. Extract the jstring from the global ref.
let jobject = map_global_ref.as_obj();
let jstring = JString::from_raw(**jobject);
// Update the metrics using the jstring as a key.
jni_call!(env, comet_metric_node(metric_node).set(&jstring, value) -> ())?;
} else {
// Cache miss. Allocate a new string, promote to global ref, and insert into cache.
let local_jstring = jni_new_string!(env, &name)?;
let global_ref = jni_new_global_ref!(env, local_jstring)?;
let arc_global_ref = Arc::new(global_ref);
metrics_jstrings.insert(name.to_string(), Arc::clone(&arc_global_ref));
let jobject = arc_global_ref.as_obj();
let jstring = JString::from_raw(**jobject);
// Update the metrics using the jstring as a key.
jni_call!(env, comet_metric_node(metric_node).set(&jstring, value) -> ())?;
}
}
}
Ok(())
Ok(native_metric_node)
}
8 changes: 8 additions & 0 deletions native/core/src/jvm_bridge/comet_metric_node.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@ pub struct CometMetricNode<'a> {
pub method_get_child_node_ret: ReturnType,
pub method_set: JMethodID,
pub method_set_ret: ReturnType,
pub method_set_all_from_bytes: JMethodID,
pub method_set_all_from_bytes_ret: ReturnType,
}

impl<'a> CometMetricNode<'a> {
Expand All @@ -47,6 +49,12 @@ impl<'a> CometMetricNode<'a> {
method_get_child_node_ret: ReturnType::Object,
method_set: env.get_method_id(Self::JVM_CLASS, "set", "(Ljava/lang/String;J)V")?,
method_set_ret: ReturnType::Primitive(Primitive::Void),
method_set_all_from_bytes: env.get_method_id(
Self::JVM_CLASS,
"set_all_from_bytes",
"([B)V",
)?,
method_set_all_from_bytes_ret: ReturnType::Primitive(Primitive::Void),
class,
})
}
Expand Down
8 changes: 0 additions & 8 deletions native/core/src/jvm_bridge/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,13 +46,6 @@ macro_rules! jvalues {
}}
}

/// Macro for create a new JNI string.
macro_rules! jni_new_string {
($env:expr, $value:expr) => {{
$crate::jvm_bridge::jni_map_error!($env, $env.new_string($value))
}};
}

/// Macro for calling a JNI method.
/// The syntax is:
/// jni_call!(env, comet_metric_node(metric_node).add(jname, value) -> ())?;
Expand Down Expand Up @@ -173,7 +166,6 @@ macro_rules! jni_new_global_ref {
pub(crate) use jni_call;
pub(crate) use jni_map_error;
pub(crate) use jni_new_global_ref;
pub(crate) use jni_new_string;
pub(crate) use jni_static_call;
pub(crate) use jvalues;

Expand Down
1 change: 1 addition & 0 deletions native/proto/build.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ fn main() -> Result<()> {
prost_build::Config::new().out_dir(out_dir).compile_protos(
&[
"src/proto/expr.proto",
"src/proto/metric.proto",
"src/proto/partitioning.proto",
"src/proto/operator.proto",
],
Expand Down
6 changes: 6 additions & 0 deletions native/proto/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,3 +36,9 @@ pub mod spark_partitioning {
pub mod spark_operator {
include!(concat!("generated", "/spark.spark_operator.rs"));
}

// Include generated modules from .proto files.
#[allow(missing_docs)]
pub mod spark_metric {
include!(concat!("generated", "/spark.spark_metric.rs"));
}
29 changes: 29 additions & 0 deletions native/proto/src/proto/metric.proto
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
// 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.



syntax = "proto3";

package spark.spark_metric;

option java_package = "org.apache.comet.serde";

message NativeMetricNode {
map<string, int64> metrics = 1;
repeated NativeMetricNode children = 2;
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.comet.CometMetricNode
import org.apache.spark.sql.vectorized._

import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_POOL_TYPE, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS}
import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_POOL_TYPE, COMET_EXPLAIN_NATIVE_ENABLED, COMET_METRICS_UPDATE_INTERVAL, COMET_WORKER_THREADS}
import org.apache.comet.vector.NativeUtil

/**
Expand Down Expand Up @@ -72,6 +72,7 @@ class CometExecIterator(
protobufQueryPlan,
numParts,
nativeMetrics,
metricsUpdateInterval = COMET_METRICS_UPDATE_INTERVAL.get(),
new CometTaskMemoryManager(id),
batchSize = COMET_BATCH_SIZE.get(),
use_unified_memory_manager = conf.getBoolean("spark.memory.offHeap.enabled", false),
Expand Down
4 changes: 4 additions & 0 deletions spark/src/main/scala/org/apache/comet/Native.scala
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,9 @@ class Native extends NativeBase {
* the bytes of serialized SparkPlan.
* @param metrics
* the native metrics of SparkPlan.
* @param metricsUpdateInterval
* the interval in milliseconds to update metrics, if interval is negative, metrics will be
* updated upon task completion.
* @param taskMemoryManager
* the task-level memory manager that is responsible for tracking memory usage across JVM and
* native side.
Expand All @@ -53,6 +56,7 @@ class Native extends NativeBase {
plan: Array[Byte],
partitionCount: Int,
metrics: CometMetricNode,
metricsUpdateInterval: Long,
taskMemoryManager: CometTaskMemoryManager,
batchSize: Int,
use_unified_memory_manager: Boolean,
Expand Down
Loading

0 comments on commit a1e6a39

Please sign in to comment.