diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 2e64403c64..2dc5bf1665 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -622,6 +622,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, diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 2bea501e5c..83029513cd 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -66,6 +66,7 @@ Comet provides the following configuration settings. | spark.comet.explainFallback.enabled | When this setting is enabled, Comet will provide logging explaining the reason(s) why a query stage cannot be executed natively. Set this to false to reduce the amount of logging. | 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 | diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index fe29d8da14..c43116d6fb 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -91,6 +91,10 @@ struct ExecutionContext { pub runtime: Runtime, /// Native metrics pub metrics: Arc, + // The interval in milliseconds to update metrics + pub metrics_update_interval: Option, + // 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 @@ -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>, /// Memory pool config pub memory_pool_config: MemoryPoolConfig, } @@ -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, @@ -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, @@ -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, }); @@ -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)) => { @@ -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 { @@ -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( diff --git a/native/core/src/execution/metrics/utils.rs b/native/core/src/execution/metrics/utils.rs index 0eb4b631dd..a8514cbd20 100644 --- a/native/core/src/execution/metrics/utils.rs +++ b/native/core/src/execution/metrics/utils.rs @@ -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; @@ -34,10 +31,22 @@ pub fn update_comet_metric( env: &mut JNIEnv, metric_node: &JObject, spark_plan: &Arc, - metrics_jstrings: &mut HashMap>, ) -> 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) -> Result { + 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(); @@ -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::>(), - 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>, -) -> 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) } diff --git a/native/core/src/jvm_bridge/comet_metric_node.rs b/native/core/src/jvm_bridge/comet_metric_node.rs index 85386d9b0d..f1f0255845 100644 --- a/native/core/src/jvm_bridge/comet_metric_node.rs +++ b/native/core/src/jvm_bridge/comet_metric_node.rs @@ -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> { @@ -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, }) } diff --git a/native/core/src/jvm_bridge/mod.rs b/native/core/src/jvm_bridge/mod.rs index 5fc0a55e3e..b863268945 100644 --- a/native/core/src/jvm_bridge/mod.rs +++ b/native/core/src/jvm_bridge/mod.rs @@ -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) -> ())?; @@ -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; diff --git a/native/proto/build.rs b/native/proto/build.rs index e707f0c3b9..ba3d12b382 100644 --- a/native/proto/build.rs +++ b/native/proto/build.rs @@ -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", ], diff --git a/native/proto/src/lib.rs b/native/proto/src/lib.rs index 266bf62dbc..ed24440360 100644 --- a/native/proto/src/lib.rs +++ b/native/proto/src/lib.rs @@ -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")); +} diff --git a/native/proto/src/proto/metric.proto b/native/proto/src/proto/metric.proto new file mode 100644 index 0000000000..f026e505ae --- /dev/null +++ b/native/proto/src/proto/metric.proto @@ -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 metrics = 1; + repeated NativeMetricNode children = 2; +} diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 7a70181401..fa77323e61 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -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 /** @@ -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), diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index c293905c96..2be0bf0066 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -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. @@ -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, diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala index 53370a03b7..41490607d6 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala @@ -19,11 +19,15 @@ package org.apache.spark.sql.comet +import scala.collection.JavaConverters._ + import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.comet.serde.Metric + /** * A node carrying SQL metrics from SparkPlan, and metrics of its children. Native code will call * [[getChildNode]] and [[set]] to update the metrics. @@ -65,6 +69,20 @@ case class CometMetricNode(metrics: Map[String, SQLMetric], children: Seq[CometM logDebug(s"Non-existing metric: $metricName. Ignored") } } + + private def set_all(metricNode: Metric.NativeMetricNode): Unit = { + metricNode.getMetricsMap.forEach((name, value) => { + set(name, value) + }) + metricNode.getChildrenList.asScala.zip(children).foreach { case (child, childNode) => + childNode.set_all(child) + } + } + + def set_all_from_bytes(bytes: Array[Byte]): Unit = { + val metricNode = Metric.NativeMetricNode.parseFrom(bytes) + set_all(metricNode) + } } object CometMetricNode {