Spark Datafusion Comet 向量化Rust Native--创建Datafusion计划

背景

Apache Datafusion Comet 是苹果公司开源的加速Spark运行的向量化项目。

本项目采用了 Spark插件化 + Protobuf + Arrow + DataFusion 架构形式

其中

  • Spark插件是 利用 SparkPlugin 插件,其中分为 DriverPlugin 和 ExecutorPlugin ,这两个插件在driver和 Executor启动的时候就会调用
  • Protobuf 是用来序列化 spark对应的表达式以及计划,用来传递给 native 引擎去执行,利用了 体积小,速度快的特性
  • Arrow 是用来 spark 和 native 引擎进行高效的数据交换(native执行的结果或者spark执行的数据结果),主要在JNI中利用Arrow IPC 列式存储以及零拷贝等特点进行进程间数据交换
  • DataFusion 主要是利用Rust native以及Arrow内存格式实现的向量化执行引擎,Spark中主要offload对应的算子到该引擎中去执行

本文基于 datafusion comet 截止到2026年1月13号的main分支的最新代码(对应的commit为 eef5f28a0727d9aef043fa2b87d6747ff68b827a)
主要分析 Rust Native 创建物理执行计划,这一步也是Comet Native Physical Plan (CometNativeExec)执行doExecuteColumnar方法会涉及到的部分

native createPlan

此处的代码主要是在CometExecIterator类中,该类会CometNativeShuffleWriterCometNativeExec调用

  • CometNativeShuffleWriter 主要是在用native shuffle的时候,会构造native的writer plan,从而写入shuffle中间文件,相比JVM写文件相比,相率更高
  • CometNativeExec 主要是 Native算子执行的时候,用来执行native单个算子,后续会从native中获取对应的结果。

Java侧:

复制代码
 private val plan = {
    val conf = SparkEnv.get.conf
    val localDiskDirs = SparkEnv.get.blockManager.getLocalDiskDirs

    // serialize Comet related Spark configs in protobuf format
    val protobufSparkConfigs = CometExecIterator.serializeCometSQLConfs()

    // Create keyUnwrapper if encryption is enabled
    val keyUnwrapper = if (encryptedFilePaths.nonEmpty) {
      val unwrapper = new CometFileKeyUnwrapper()
      val hadoopConf: Configuration = broadcastedHadoopConfForEncryption.get.value.value

      encryptedFilePaths.foreach(filePath =>
        unwrapper.storeDecryptionKeyRetriever(filePath, hadoopConf))

      unwrapper
    } else {
      null
    }

    val memoryConfig = CometExecIterator.getMemoryConfig(conf)

    nativeLib.createPlan(
      id,
      cometBatchIterators,
      protobufQueryPlan,
      protobufSparkConfigs,
      numParts,
      nativeMetrics,
      metricsUpdateInterval = COMET_METRICS_UPDATE_INTERVAL.get(),
      cometTaskMemoryManager,
      localDiskDirs,
      batchSize = COMET_BATCH_SIZE.get(),
      memoryConfig.offHeapMode,
      memoryConfig.memoryPoolType,
      memoryConfig.memoryLimit,
      memoryConfig.memoryLimitPerTask,
      taskAttemptId,
      taskCPUs,
      keyUnwrapper)
  }

注意这些代码是在Spark Executor端执行的,

  • protobufSparkConfigs 是获取以spark.comet开头的配置项,并且以 ProtoBuf的格式序列化
  • CometExecIterator.getMemoryConfig 获取内存相关配置以及每个Task的内存配置
    我们这里是offHeapMode,且spark.comet.exec.memoryPoolfair_unified
    createPlan 方法对应的参数解释:
序号 参数名 Scala 类型 JNI 类型 说明 用途 约束/注意事项
1 id Long jlong 执行上下文的唯一标识符 写入 ExecutionContext.id,用于日志、追踪和后续方法调用 releasePlan 使用的句柄一致
2 iterators Array[CometBatchIterator] JObjectArray 输入数据源迭代器数组 对应 Spark 计划中的所有 Scan 节点,Native 侧通过 JNI 回调拉取数据 必须与计划中 Scan 数量一致,顺序对应
3 plan Array[Byte] JByteArray 序列化后的 Spark 执行计划 Protobuf 格式的 Operator,反序列化为 spark_plan,用于后续创建物理计划 Protobuf 格式必须正确,版本需匹配
4 configMapProto Array[Byte] JByteArray 序列化后的配置映射 Protobuf ConfigMap,包含所有 spark.comet.* 配置 缺失的配置使用默认值(如 tracing.enabled 默认为 false
5 partitionCount Int jint 分区总数 写入 ExecutionContext.partition_count,供物理计划和 ShuffleWriter 使用 必须 ≥ 1
6 metrics CometMetricNode JObject Native 指标树的 JVM 表示 创建 GlobalRef 保存,用于在 executePlan 时更新 Spark SQLMetric 必须非 null
7 metricsUpdateInterval Long jlong 指标更新间隔(毫秒) 控制指标更新频率,避免频繁 JNI 调用 > 0 表示定期更新,≤ 0 表示仅在任务结束时更新
8 taskMemoryManager CometTaskMemoryManager JObject 任务级内存管理器 协调 JVM 和 Native 内存使用,创建内存池时传入 必须非 null
9 localDirs Array[String] JObjectArray 本地磁盘目录列表 用于 DataFusion 的临时文件和溢出文件存储 通常来自 SparkEnv.get.blockManager.getLocalDiskDirs
10 batchSize Int jint 批处理大小(行数) 设置 DataFusion SessionConfig 的批大小,影响内存使用和执行效率 默认与 spark.comet.batchSize 保持一致(8196)
11 offHeapMode Boolean jboolean 是否使用 off-heap 内存池 决定内存池类型,与 Spark off-heap 内存统一管理 true 使用 off-heap,false 使用 on-heap
12 memoryPoolType String JString 内存池类型字符串 指定具体的内存池实现类型(如 GreedyUnbounded 等) 必须与 offHeapMode 匹配
13 memoryLimit Long jlong 总内存上限(字节) 设置内存池的总容量限制 必须 > 0
14 memoryLimitPerTask Long jlong 单任务内存上限(字节) 限制单个任务的内存使用,按 CPU 比例分配 通常 = memoryLimit × (taskCPUs / 总核数)
15 taskAttemptId Long jlong Spark Task attempt ID 写入 ExecutionContext.task_attempt_id,用于日志和资源标识 来自 TaskContext.get().taskAttemptId
16 taskCPUs Long jlong Task 分配的 CPU 数 设置 DataFusion 的 target_partitions,影响并行度 来自 TaskContext.get().cpus()
17 keyUnwrapper CometFileKeyUnwrapper JObject 加密 Parquet 文件的 Key 解包器 注册到 Session RuntimeEnv,用于读取加密 Parquet 文件 无加密时传 null,Native 侧会跳过注册

返回值 是 对应ExecutionContext的地址

Rust 侧:

对应的为 jni_api.rs中的 pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan方法:

  • 反序列化ConfigMap

    复制代码
     let bytes = env.convert_byte_array(serialized_spark_configs)?;
     let spark_configs: datafusion_comet_proto::spark_config::ConfigMap = serde::deserialize_config(bytes.as_slice())?;
     let spark_config: HashMap<String, String> = spark_configs.entries.into_iter().collect();

    into_iter().collect() 使用该方法 将Map转换为迭代器并消耗所有权,将元素移动到新的HashMap集合中

  • debug_native/explain_native/tracing_enabled/max_temp_directory_size 获取值

    通过对 HashMap扩展方法,实现对应的取值:

    复制代码
     impl SparkConfig for HashMap<String, String> {
      fn get_bool(&self, name: &str) -> bool {
          self.get(name)
              .and_then(|str_val| str_val.parse::<bool>().ok())
              .unwrap_or(false)
      }
    
      fn get_u64(&self, name: &str, default_value: u64) -> u64 {
          self.get(name)
              .and_then(|str_val| str_val.parse::<u64>().ok())
              .unwrap_or(default_value)
      }
    } 

    对于boolean值调用 get_bool方法,对于整数值,调用get_u64方法,

  • JVMClasses初始化JVMClasses::init(&mut env)
    let env = unsafe { std::mem::transmute::<&mut JNIEnv, &'static mut JNIEnv>(env) }; 使用std::mem::transmute将env改为静态数据类型

    依次调用 JNIEnv的find_class和get_method_id获取对应的JMethodID

  • 反序列化spark_plan

    复制代码
     let bytes = env.convert_byte_array(serialized_query)?;
     let spark_plan = serde::deserialize_op(bytes.as_slice())?;

    这里会转换为 Protobuf格式的Operator

  • 持久化持有CometMetricNode对象,防止Java垃圾回收器(GC)在Rust使用期间回收该对象(GlobalRef 在明确调用 .drop() 或显式释放前一直有效)
    let metrics = Arc::new(jni_new_global_ref!(env, metrics_node)?);
    Array[CometBatchIterator]CometTaskMemoryManager也是一样

  • prepare_datafusion_session_context

    创建DataFusion SessionContext主要通过Spark Context来给对应的Datafusion SessionContext 赋值,具体可以见Configuration
    注意这里:把Dafasusion任务的并行度设置为了Spark 单个task的core数
    datafusion::functions_nested::register_all register_datafusion_spark_function datafusion_comet_spark_expr::register_all_comet_functions 注册对应的内部函数

  • 构造ExecutionContext,并返回指向该对象的指针

    复制代码
     let exec_context: Box<ExecutionContext> = Box::new(ExecutionContext {
                  id,
                  task_attempt_id,
                  spark_plan,
                  partition_count: partition_count as usize,
                  root_op: None,
                  scans: vec![],
                  input_sources,
                  stream: None,
                  metrics,
                  metrics_update_interval,
                  metrics_last_update_time: Instant::now(),
                  poll_count_since_metrics_check: 0,
                  plan_creation_time,
                  session_ctx: Arc::new(session),
                  debug_native,
                  explain_native,
                  memory_pool_config,
                  tracing_enabled,
              });
    
    Ok(Box::into_raw(exec_context) as i64)

    这里还会记录下该Plan的创建耗时plan_creation_time
    Box::into_raw(exec_context)是Rust 中用于消耗 Box 并将其转换为指向堆内存的裸指针 *mut T 的方法。它将内存管理的所有权从 Rust 的自动内存管理系统转移给开发者,使原 Box 实例不再负责释放内存

    此时计划就已经创建好,并且返回了指向了ExecutionContext的指针

相关推荐
咸甜适中2 小时前
rust的docx-rs库,自定义docx模版批量分页生成一个docx文档(方便打印)(逐行注释)
rust·办公自动化·docx-rs
想你依然心痛2 小时前
Spark大数据分析与实战笔记(第六章 Kafka分布式发布订阅消息系统-03)
笔记·分布式·spark·kafka
Vallelonga2 小时前
Rust Option.as_ref() 方法
开发语言·rust
王锋(oxwangfeng)2 小时前
Spark 向量化执行引擎技术选型与实践指南
大数据·分布式·spark
小邓睡不饱耶2 小时前
使用Spark进行学生成绩数据深度分析与处理
大数据·分布式·spark
亚林瓜子3 小时前
pyspark添加一列时间戳数据并改名
python·spark
编程彩机15 小时前
互联网大厂Java面试:从分布式架构到大数据场景解析
java·大数据·微服务·spark·kafka·分布式事务·分布式架构
大卫小东(Sheldon)17 小时前
GIM 2.0 发布:真正让 AI 提交消息可定制、可控、可项目级优化
git·rust·gim
Moshow郑锴1 天前
Spark在银行系统ETL中的实战应用:TXT文件到PostgreSQL的余额处理全流程
postgresql·spark·etl