Skip to content
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
141 changes: 141 additions & 0 deletions common/src/main/java/org/apache/comet/udf/CometUdfBridge.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,141 @@
/*
* 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.comet.udf;

import java.util.LinkedHashMap;
import java.util.Map;

import org.apache.arrow.c.ArrowArray;
import org.apache.arrow.c.ArrowSchema;
import org.apache.arrow.c.Data;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.vector.FieldVector;
import org.apache.arrow.vector.ValueVector;

/**
* JNI entry point for native execution to invoke a {@link CometUDF}. Matches the static-method
* pattern used by CometScalarSubquery so the native side can dispatch via
* call_static_method_unchecked.
*/
public class CometUdfBridge {

// Per-thread, bounded LRU of UDF instances keyed by class name. Comet
// native execution threads (Tokio/DataFusion worker pool) are reused
// across tasks within an executor, so the effective lifetime of cached
// entries is the worker thread (i.e. the executor JVM). This is fine for
// stateless UDFs like ArrayExistsUDF; future stateful UDFs would need
// explicit per-task isolation.
private static final int CACHE_CAPACITY = 64;

private static final ThreadLocal<LinkedHashMap<String, CometUDF>> INSTANCES =
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Should we ensure one instance per thread? Spark/Hive UDFs don't seem to do this.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

yeah, that is a good point, will take another look

ThreadLocal.withInitial(
() ->
new LinkedHashMap<String, CometUDF>(CACHE_CAPACITY, 0.75f, true) {
@Override
protected boolean removeEldestEntry(Map.Entry<String, CometUDF> eldest) {
return size() > CACHE_CAPACITY;
}
});

/**
* Called from native via JNI.
*
* @param udfClassName fully-qualified class name implementing CometUDF
* @param inputArrayPtrs addresses of pre-allocated FFI_ArrowArray structs (one per input)
* @param inputSchemaPtrs addresses of pre-allocated FFI_ArrowSchema structs (one per input)
* @param outArrayPtr address of pre-allocated FFI_ArrowArray for the result
* @param outSchemaPtr address of pre-allocated FFI_ArrowSchema for the result
*/
public static void evaluate(
String udfClassName,
long[] inputArrayPtrs,
long[] inputSchemaPtrs,
long outArrayPtr,
long outSchemaPtr) {
LinkedHashMap<String, CometUDF> cache = INSTANCES.get();
CometUDF udf = cache.get(udfClassName);
if (udf == null) {
try {
// Resolve via the executor's context classloader so user-supplied UDF jars
// (added via spark.jars / --jars) are visible.
ClassLoader cl = Thread.currentThread().getContextClassLoader();
if (cl == null) {
cl = CometUdfBridge.class.getClassLoader();
}
udf =
(CometUDF) Class.forName(udfClassName, true, cl).getDeclaredConstructor().newInstance();
} catch (ReflectiveOperationException e) {
throw new RuntimeException("Failed to instantiate CometUDF: " + udfClassName, e);
}
cache.put(udfClassName, udf);
}

BufferAllocator allocator = org.apache.comet.package$.MODULE$.CometArrowAllocator();

ValueVector[] inputs = new ValueVector[inputArrayPtrs.length];
ValueVector result = null;
try {
for (int i = 0; i < inputArrayPtrs.length; i++) {
ArrowArray inArr = ArrowArray.wrap(inputArrayPtrs[i]);
ArrowSchema inSch = ArrowSchema.wrap(inputSchemaPtrs[i]);
inputs[i] = Data.importVector(allocator, inArr, inSch, null);
}

result = udf.evaluate(inputs);
if (!(result instanceof FieldVector)) {
throw new RuntimeException(
"CometUDF.evaluate() must return a FieldVector, got: " + result.getClass().getName());
}
// Result length must match the longest input. Scalar (length-1) inputs
// are allowed to be shorter, but a vector input bounds the output.
int expectedLen = 0;
for (ValueVector v : inputs) {
expectedLen = Math.max(expectedLen, v.getValueCount());
}
if (result.getValueCount() != expectedLen) {
throw new RuntimeException(
"CometUDF.evaluate() returned "
+ result.getValueCount()
+ " rows, expected "
+ expectedLen);
}
ArrowArray outArr = ArrowArray.wrap(outArrayPtr);
ArrowSchema outSch = ArrowSchema.wrap(outSchemaPtr);
Data.exportVector(allocator, (FieldVector) result, null, outArr, outSch);
} finally {
for (ValueVector v : inputs) {
if (v != null) {
try {
v.close();
} catch (RuntimeException ignored) {
// do not mask the original throwable
}
}
}
if (result != null) {
try {
result.close();
} catch (RuntimeException ignored) {
// do not mask the original throwable
}
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.comet.udf

import java.util.UUID
import java.util.concurrent.ConcurrentHashMap

import org.apache.spark.sql.catalyst.expressions.Expression

/**
* Thread-safe registry bridging plan-time Spark expressions to execution-time UDF lookup. At plan
* time the serde layer registers a lambda expression under a unique key; at execution time the
* UDF retrieves it by that key (passed as a scalar argument).
*/
object CometLambdaRegistry {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Where is CometLambdaRegistry used?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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


private val registry = new ConcurrentHashMap[String, Expression]()

def register(expression: Expression): String = {
val key = UUID.randomUUID().toString
registry.put(key, expression)
key
}

def get(key: String): Expression = {
val expr = registry.get(key)
if (expr == null) {
throw new IllegalStateException(
s"Lambda expression not found in registry for key: $key. " +
"This indicates a lifecycle issue between plan creation and execution.")
}
expr
}

def remove(key: String): Unit = {
registry.remove(key)
}

// Visible for testing
def size(): Int = registry.size()
}
37 changes: 37 additions & 0 deletions common/src/main/scala/org/apache/comet/udf/CometUDF.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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.comet.udf

import org.apache.arrow.vector.ValueVector

/**
* Scalar UDF invoked from native execution via JNI. Receives Arrow vectors as input and returns
* an Arrow vector.
*
* - Vector arguments arrive at the row count of the current batch.
* - Scalar (literal-folded) arguments arrive as length-1 vectors and must be read at index 0.
* - The returned vector's length must match the longest input.
*
* Implementations must have a public no-arg constructor and should be stateless: instances are
* cached per executor thread for the lifetime of the JVM.
*/
trait CometUDF {
def evaluate(inputs: Array[ValueVector]): ValueVector
}
2 changes: 2 additions & 0 deletions native/Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

25 changes: 21 additions & 4 deletions native/core/src/execution/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -122,10 +122,10 @@ use datafusion_comet_proto::{
spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning},
};
use datafusion_comet_spark_expr::{
ArrayInsert, Avg, AvgDecimal, Cast, CheckOverflow, Correlation, Covariance, CreateNamedStruct,
DecimalRescaleCheckOverflow, GetArrayStructFields, GetStructField, IfExpr, ListExtract,
NormalizeNaNAndZero, SparkCastOptions, Stddev, SumDecimal, ToJson, UnboundColumn, Variance,
WideDecimalBinaryExpr, WideDecimalOp,
jvm_udf::JvmScalarUdfExpr, ArrayInsert, Avg, AvgDecimal, Cast, CheckOverflow, Correlation,
Covariance, CreateNamedStruct, DecimalRescaleCheckOverflow, GetArrayStructFields,
GetStructField, IfExpr, ListExtract, NormalizeNaNAndZero, SparkCastOptions, Stddev, SumDecimal,
ToJson, UnboundColumn, Variance, WideDecimalBinaryExpr, WideDecimalOp,
};
use itertools::Itertools;
use jni::objects::{Global, JObject};
Expand Down Expand Up @@ -701,6 +701,23 @@ impl PhysicalPlanner {
expr.names.clone(),
)))
}
ExprStruct::JvmScalarUdf(udf) => {
let args = udf
.args
.iter()
.map(|e| self.create_expr(e, Arc::clone(&input_schema)))
.collect::<Result<Vec<_>, _>>()?;
let return_type =
to_arrow_datatype(udf.return_type.as_ref().ok_or_else(|| {
GeneralError("JvmScalarUdf missing return_type".to_string())
})?);
Ok(Arc::new(JvmScalarUdfExpr::new(
udf.class_name.clone(),
args,
return_type,
udf.return_nullable,
)))
}
expr => Err(GeneralError(format!("Not implemented: {expr:?}"))),
}
}
Expand Down
50 changes: 50 additions & 0 deletions native/jni-bridge/src/comet_udf_bridge.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// 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.

use jni::{
errors::Result as JniResult,
objects::{JClass, JStaticMethodID},
signature::{Primitive, ReturnType},
strings::JNIString,
Env,
};

/// JNI handle for the JVM `org.apache.comet.udf.CometUdfBridge` class.
/// Mirrors the static-method pattern in `comet_exec.rs` (`CometScalarSubquery`).
#[allow(dead_code)] // class field is held to keep JStaticMethodID alive
pub struct CometUdfBridge<'a> {
pub class: JClass<'a>,
pub method_evaluate: JStaticMethodID,
pub method_evaluate_ret: ReturnType,
}

impl<'a> CometUdfBridge<'a> {
pub const JVM_CLASS: &'static str = "org/apache/comet/udf/CometUdfBridge";

pub fn new(env: &mut Env<'a>) -> JniResult<CometUdfBridge<'a>> {
let class = env.find_class(JNIString::new(Self::JVM_CLASS))?;
Ok(CometUdfBridge {
method_evaluate: env.get_static_method_id(
JNIString::new(Self::JVM_CLASS),
jni::jni_str!("evaluate"),
jni::jni_sig!("(Ljava/lang/String;[J[JJJ)V"),
)?,
method_evaluate_ret: ReturnType::Primitive(Primitive::Void),
class,
})
}
}
12 changes: 12 additions & 0 deletions native/jni-bridge/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -192,11 +192,13 @@ pub use comet_exec::*;
mod batch_iterator;
mod comet_metric_node;
mod comet_task_memory_manager;
mod comet_udf_bridge;
mod shuffle_block_iterator;

use batch_iterator::CometBatchIterator;
pub use comet_metric_node::*;
pub use comet_task_memory_manager::*;
use comet_udf_bridge::CometUdfBridge;
use shuffle_block_iterator::CometShuffleBlockIterator;

/// The JVM classes that are used in the JNI calls.
Expand Down Expand Up @@ -228,6 +230,9 @@ pub struct JVMClasses<'a> {
/// The CometTaskMemoryManager used for interacting with JVM side to
/// acquire & release native memory.
pub comet_task_memory_manager: CometTaskMemoryManager<'a>,
/// The CometUdfBridge class used to dispatch JVM scalar UDFs.
/// `None` if the class is not on the classpath.
pub comet_udf_bridge: Option<CometUdfBridge<'a>>,
}

unsafe impl Send for JVMClasses<'_> {}
Expand Down Expand Up @@ -298,6 +303,13 @@ impl JVMClasses<'_> {
comet_batch_iterator: CometBatchIterator::new(env).unwrap(),
comet_shuffle_block_iterator: CometShuffleBlockIterator::new(env).unwrap(),
comet_task_memory_manager: CometTaskMemoryManager::new(env).unwrap(),
comet_udf_bridge: {
let bridge = CometUdfBridge::new(env).ok();
if env.exception_check() {
env.exception_clear();
}
bridge
},
}
});
}
Expand Down
Loading
Loading