diff --git a/.github/workflows/pr_benchmark_check.yml b/.github/workflows/pr_benchmark_check.yml
index b07cc03c34..a879493a7f 100644
--- a/.github/workflows/pr_benchmark_check.yml
+++ b/.github/workflows/pr_benchmark_check.yml
@@ -84,9 +84,7 @@ jobs:
${{ runner.os }}-benchmark-maven-
- name: Check Scala compilation and linting
- # Pin to spark-4.0 (Scala 2.13.16) because the default profile is now
- # spark-4.1 / Scala 2.13.17, and semanticdb-scalac_2.13.17 is not yet
- # published, which breaks `-Psemanticdb`. See pr_build_linux.yml for
- # the same exclusion in the main lint matrix.
+ # Pinned to spark-4.0 because semanticdb-scalac_2.13.17 (spark-4.1 default)
+ # is not yet published, which breaks the -Psemanticdb scalafix lint.
run: |
- ./mvnw -B compile test-compile scalafix:scalafix -Dscalafix.mode=CHECK -Psemanticdb -Pspark-4.0 -DskipTests
+ ./mvnw -B compile test-compile scalafix:scalafix -Dscalafix.mode=CHECK -Pspark-4.0 -Psemanticdb -DskipTests
diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml
index dd5377ef54..313af77deb 100644
--- a/.github/workflows/pr_build_linux.yml
+++ b/.github/workflows/pr_build_linux.yml
@@ -309,6 +309,7 @@ jobs:
org.apache.comet.CometFuzzAggregateSuite
org.apache.comet.CometFuzzIcebergSuite
org.apache.comet.CometFuzzMathSuite
+ org.apache.comet.CometCodegenDispatchFuzzSuite
org.apache.comet.DataGeneratorSuite
- name: "shuffle"
value: |
@@ -387,6 +388,8 @@ jobs:
org.apache.comet.expressions.conditional.CometIfSuite
org.apache.comet.expressions.conditional.CometCoalesceSuite
org.apache.comet.expressions.conditional.CometCaseWhenSuite
+ org.apache.comet.CometCodegenDispatchSmokeSuite
+ org.apache.comet.CometCodegenSourceSuite
- name: "sql"
value: |
org.apache.spark.sql.CometToPrettyStringSuite
diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml
index 8abaa1c776..b2af6e43ab 100644
--- a/.github/workflows/pr_build_macos.yml
+++ b/.github/workflows/pr_build_macos.yml
@@ -157,6 +157,7 @@ jobs:
org.apache.comet.CometFuzzAggregateSuite
org.apache.comet.CometFuzzIcebergSuite
org.apache.comet.CometFuzzMathSuite
+ org.apache.comet.CometCodegenDispatchFuzzSuite
org.apache.comet.DataGeneratorSuite
- name: "shuffle"
value: |
@@ -234,6 +235,8 @@ jobs:
org.apache.comet.expressions.conditional.CometIfSuite
org.apache.comet.expressions.conditional.CometCoalesceSuite
org.apache.comet.expressions.conditional.CometCaseWhenSuite
+ org.apache.comet.CometCodegenDispatchSmokeSuite
+ org.apache.comet.CometCodegenSourceSuite
- name: "sql"
value: |
org.apache.spark.sql.CometToPrettyStringSuite
diff --git a/common/src/main/java/org/apache/comet/codegen/CometBatchKernel.java b/common/src/main/java/org/apache/comet/codegen/CometBatchKernel.java
new file mode 100644
index 0000000000..f9fbb775a0
--- /dev/null
+++ b/common/src/main/java/org/apache/comet/codegen/CometBatchKernel.java
@@ -0,0 +1,68 @@
+/*
+ * 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.codegen;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.ValueVector;
+
+/**
+ * Abstract base extended by the Janino-compiled batch kernel emitted by {@code
+ * CometBatchKernelCodegen}. The generated subclass extends {@code CometInternalRow} (so Spark's
+ * {@code BoundReference.genCode} can call {@code this.getUTF8String(ord)} directly) and carries
+ * typed input fields baked at codegen time, one per input column. Expression evaluation plus Arrow
+ * read/write fuse into one method per expression tree.
+ *
+ *
Input scope: any {@code ValueVector[]}; the generated subclass casts each slot to the concrete
+ * Arrow type the compile-time schema specified. Output is a generic {@code FieldVector}; the
+ * generated subclass casts to the concrete type matching the bound expression's {@code dataType}.
+ * Widen input support by adding vector classes to the getter switch in {@code
+ * CometBatchKernelCodegen.emitTypedGetters}; widen output support by adding cases in {@code
+ * CometBatchKernelCodegen.allocateOutput} and {@code emitOutputWriter}.
+ */
+public abstract class CometBatchKernel extends CometInternalRow {
+
+ protected final Object[] references;
+
+ protected CometBatchKernel(Object[] references) {
+ this.references = references;
+ }
+
+ /**
+ * Process one batch.
+ *
+ * @param inputs Arrow input vectors; length and concrete classes must match the schema the kernel
+ * was compiled against
+ * @param output Arrow output vector; caller allocates to the expression's {@code dataType}
+ * @param numRows number of rows in this batch
+ */
+ public abstract void process(ValueVector[] inputs, FieldVector output, int numRows);
+
+ /**
+ * Run partition-dependent initialization. The generated subclass overrides this to execute
+ * statements collected via {@code CodegenContext.addPartitionInitializationStatement}, for
+ * example reseeding {@code Rand}'s {@code XORShiftRandom} from {@code seed + partitionIndex}.
+ * Deterministic expressions leave this as a no-op.
+ *
+ *
The caller must invoke this before the first {@code process} call of each partition. The
+ * generated subclass is not thread-safe across concurrent {@code process} calls, so kernels are
+ * allocated per dispatcher invocation and init is run once on the fresh instance.
+ */
+ public void init(int partitionIndex) {}
+}
diff --git a/common/src/main/java/org/apache/comet/udf/CometUdfBridge.java b/common/src/main/java/org/apache/comet/udf/CometUdfBridge.java
index 5e76819810..9e97ef2226 100644
--- a/common/src/main/java/org/apache/comet/udf/CometUdfBridge.java
+++ b/common/src/main/java/org/apache/comet/udf/CometUdfBridge.java
@@ -29,18 +29,48 @@
import org.apache.arrow.vector.ValueVector;
import org.apache.spark.TaskContext;
import org.apache.spark.comet.CometTaskContextShim;
+import org.apache.spark.util.TaskCompletionListener;
/**
* 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.
+ *
+ *
Cache invariants:
+ *
+ *
+ * - For each live Spark task attempt there is at most one {@link CometUDF} instance per class
+ * name.
+ *
- A {@link CometUDF} instance is visible only within the Spark task attempt that instantiated
+ * it. Two task attempts observing the same class name receive distinct instances.
+ *
- At any instant at most one thread is inside {@code evaluate()} for a given {@code
+ * taskAttemptId}. This follows from Spark executing one native future per partition and Tokio
+ * polling one future per worker at a time.
+ *
- All instances for a task are dropped by the {@link TaskCompletionListener} registered on
+ * the first cache miss for that task. No cache entry outlives its task.
+ *
- When {@code taskContext} is {@code null} (unit tests, direct native driver) the fallback
+ * key {@code -1L} is used; that bucket is never evicted because no task-completion event will
+ * fire.
+ *
+ *
+ * Keying by {@code taskAttemptId} rather than by thread keeps the cache correct under Tokio
+ * work-stealing: on the scan-free execution path the same Spark task can be polled by different
+ * Tokio workers across batches, so a thread-local cache would lose per-task state on migration. The
+ * task attempt ID is stable for the life of the task regardless of which worker is polling.
*/
public class CometUdfBridge {
- // Process-wide cache of UDF instances keyed by class name. CometUDF
- // implementations are required to be stateless (see CometUDF), so a
- // single shared instance per class is safe across native worker threads.
- private static final ConcurrentHashMap INSTANCES = new ConcurrentHashMap<>();
+ /**
+ * Task-scoped cache of {@link CometUDF} instances. Outer map keys are Spark task attempt IDs (or
+ * {@code -1L} when no {@link TaskContext} is available). Inner maps hold one instance per UDF
+ * class name for the task's lifetime. Entries are removed by the {@link TaskCompletionListener}
+ * registered on the first cache miss per task.
+ */
+ private static final ConcurrentHashMap> INSTANCES =
+ new ConcurrentHashMap<>();
+
+ /** Sentinel key for calls that carry no {@link TaskContext} (unit tests, direct driver). */
+ private static final long NO_TASK_ID = -1L;
/**
* Called from native via JNI.
@@ -58,7 +88,9 @@ public class CometUdfBridge {
* thread-local on entry, with the prior value (if any) saved and restored in {@code finally}.
* Lets partition-sensitive built-ins ({@code Rand}, {@code Uuid}, {@code
* MonotonicallyIncreasingID}) work from Tokio workers and avoids reusing a stale TaskContext
- * left on a worker by a previous task.
+ * left on a worker by a previous task. Its task attempt ID also keys the UDF-instance cache,
+ * so a UDF holding per-task state in fields sees a consistent instance for every call within
+ * the task regardless of which Tokio worker is polling.
*/
public static void evaluate(
String udfClassName,
@@ -68,16 +100,33 @@ public static void evaluate(
long outSchemaPtr,
int numRows,
TaskContext taskContext) {
- // Save-and-restore rather than only-install-if-null: the propagated context is the ground
- // truth for this call. Any value already on the thread is either (a) the same object on a
- // Spark task thread, or (b) stale from a prior task on a reused Tokio worker.
+ assert udfClassName != null && !udfClassName.isEmpty() : "udfClassName must be non-empty";
+ assert inputArrayPtrs != null && inputSchemaPtrs != null
+ : "input pointer arrays must be non-null";
+ assert inputArrayPtrs.length == inputSchemaPtrs.length
+ : "input array pointer count must equal schema pointer count";
+ assert numRows >= 0 : "numRows must be non-negative";
+ assert outArrayPtr != 0L : "outArrayPtr must be a valid FFI pointer";
+ assert outSchemaPtr != 0L : "outSchemaPtr must be a valid FFI pointer";
+
+ // Save-and-restore rather than only-install-if-null: the propagated `taskContext` is the
+ // ground truth for this call. Any value already on the thread is either (a) the same object
+ // on a Spark task thread, or (b) stale from a prior task on a reused Tokio worker.
TaskContext prior = TaskContext.get();
if (taskContext != null) {
CometTaskContextShim.set(taskContext);
+ assert TaskContext.get() == taskContext
+ : "TaskContext install did not take effect on this thread";
}
try {
evaluateInternal(
- udfClassName, inputArrayPtrs, inputSchemaPtrs, outArrayPtr, outSchemaPtr, numRows);
+ udfClassName,
+ inputArrayPtrs,
+ inputSchemaPtrs,
+ outArrayPtr,
+ outSchemaPtr,
+ numRows,
+ taskContext);
} finally {
if (taskContext != null) {
if (prior != null) {
@@ -95,9 +144,34 @@ private static void evaluateInternal(
long[] inputSchemaPtrs,
long outArrayPtr,
long outSchemaPtr,
- int numRows) {
- CometUDF udf =
+ int numRows,
+ TaskContext taskContext) {
+ long taskAttemptId = (taskContext != null) ? taskContext.taskAttemptId() : NO_TASK_ID;
+
+ ConcurrentHashMap perTask =
INSTANCES.computeIfAbsent(
+ taskAttemptId,
+ id -> {
+ ConcurrentHashMap fresh = new ConcurrentHashMap<>();
+ if (taskContext != null) {
+ // computeIfAbsent runs this lambda at most once per key, so the listener is
+ // registered exactly once per task attempt.
+ taskContext.addTaskCompletionListener(
+ (TaskCompletionListener)
+ ctx -> {
+ ConcurrentHashMap removed = INSTANCES.remove(id);
+ assert removed != null
+ : "task-completion listener fired but cache already removed "
+ + "entry for task "
+ + id;
+ });
+ }
+ return fresh;
+ });
+ assert perTask != null : "per-task cache must be non-null after computeIfAbsent";
+
+ CometUDF udf =
+ perTask.computeIfAbsent(
udfClassName,
name -> {
try {
@@ -113,6 +187,7 @@ private static void evaluateInternal(
throw new RuntimeException("Failed to instantiate CometUDF: " + name, e);
}
});
+ assert udf != null : "reflective instantiation returned null for " + udfClassName;
BufferAllocator allocator = org.apache.comet.package$.MODULE$.CometArrowAllocator();
diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala
index 9b376837f7..feb4129ac5 100644
--- a/common/src/main/scala/org/apache/comet/CometConf.scala
+++ b/common/src/main/scala/org/apache/comet/CometConf.scala
@@ -380,6 +380,17 @@ object CometConf extends ShimCometConf {
.booleanConf
.createWithDefault(false)
+ val COMET_SCALA_UDF_CODEGEN_ENABLED: ConfigEntry[Boolean] =
+ conf("spark.comet.exec.scalaUDF.codegen.enabled")
+ .category(CATEGORY_EXEC)
+ .doc(
+ "Whether to route Spark `ScalaUDF` expressions through Comet's Arrow-direct codegen " +
+ "dispatcher. When enabled, a supported ScalaUDF is compiled into a per-batch kernel " +
+ "that reads and writes Arrow vectors directly from native execution. When disabled, " +
+ "plans containing a ScalaUDF fall back to Spark for the enclosing operator.")
+ .booleanConf
+ .createWithDefault(true)
+
val COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED: ConfigEntry[Boolean] =
conf("spark.comet.native.shuffle.partitioning.hash.enabled")
.category(CATEGORY_SHUFFLE)
diff --git a/common/src/main/scala/org/apache/comet/codegen/CometArrayData.scala b/common/src/main/scala/org/apache/comet/codegen/CometArrayData.scala
new file mode 100644
index 0000000000..1696c466a3
--- /dev/null
+++ b/common/src/main/scala/org/apache/comet/codegen/CometArrayData.scala
@@ -0,0 +1,106 @@
+/*
+ * 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.codegen
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
+
+import org.apache.comet.shims.CometInternalRowShim
+
+/**
+ * Throwing-default base for [[ArrayData]] in the Arrow-direct codegen kernel. Subclasses override
+ * only the getters their element type needs (e.g. `numElements`, `isNullAt`, `getUTF8String` for
+ * an `ArrayType(StringType)` input).
+ *
+ * Consumer: `InputArray_${path}` nested classes the input emitter generates per `ArrayType` input
+ * column. They back `getArray(ord)` plus the recursion for `Array>` and array-typed
+ * map keys / struct fields.
+ *
+ * `ArrayData` and [[CometInternalRow]]'s [[InternalRow]] are sibling abstract classes in Spark
+ * (both extend `SpecializedGetters`, neither inherits the other), so a base aimed at one cannot
+ * serve the other. The dispatch body that '''is''' shared between them lives in
+ * [[CometSpecializedGettersDispatch]]. The third sibling, [[CometMapData]], backs `InputMap_*`
+ * and routes `keyArray()` / `valueArray()` through `CometArrayData` instances.
+ *
+ * Mixes in [[CometInternalRowShim]] for the same reason `CometInternalRow` does: Spark 4.x adds
+ * abstract `SpecializedGetters` methods (`getVariant`, `getGeography`, `getGeometry`) that both
+ * `InternalRow` and `ArrayData` inherit; the per-profile shim provides throwing defaults.
+ */
+abstract class CometArrayData extends ArrayData with CometInternalRowShim {
+
+ override def getInterval(ordinal: Int): CalendarInterval = unsupported("getInterval")
+
+ override def get(ordinal: Int, dataType: DataType): AnyRef =
+ CometSpecializedGettersDispatch.get(this, ordinal, dataType)
+
+ override def isNullAt(ordinal: Int): Boolean = unsupported("isNullAt")
+
+ override def getBoolean(ordinal: Int): Boolean = unsupported("getBoolean")
+
+ override def getByte(ordinal: Int): Byte = unsupported("getByte")
+
+ override def getShort(ordinal: Int): Short = unsupported("getShort")
+
+ override def getInt(ordinal: Int): Int = unsupported("getInt")
+
+ override def getLong(ordinal: Int): Long = unsupported("getLong")
+
+ override def getFloat(ordinal: Int): Float = unsupported("getFloat")
+
+ override def getDouble(ordinal: Int): Double = unsupported("getDouble")
+
+ override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal =
+ unsupported("getDecimal")
+
+ override def getUTF8String(ordinal: Int): UTF8String = unsupported("getUTF8String")
+
+ override def getBinary(ordinal: Int): Array[Byte] = unsupported("getBinary")
+
+ override def getStruct(ordinal: Int, numFields: Int): InternalRow = unsupported("getStruct")
+
+ override def getArray(ordinal: Int): ArrayData = unsupported("getArray")
+
+ override def getMap(ordinal: Int): MapData = unsupported("getMap")
+
+ override def setNullAt(i: Int): Unit = unsupported("setNullAt")
+
+ protected def unsupported(method: String): Nothing =
+ throw new UnsupportedOperationException(
+ s"${getClass.getSimpleName}: $method not implemented for this array shape")
+
+ override def update(i: Int, value: Any): Unit = unsupported("update")
+
+ override def copy(): ArrayData = unsupported("copy")
+
+ override def array: Array[Any] = unsupported("array")
+
+ override def toString(): String = {
+ val n =
+ try numElements().toString
+ catch {
+ case _: Throwable => "?"
+ }
+ s"${getClass.getSimpleName}(numElements=$n)"
+ }
+
+ override def numElements(): Int = unsupported("numElements")
+}
diff --git a/common/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegen.scala b/common/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegen.scala
new file mode 100644
index 0000000000..bf5a9eaa4b
--- /dev/null
+++ b/common/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegen.scala
@@ -0,0 +1,535 @@
+/*
+ * 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.codegen
+
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.complex.{ListVector, MapVector, StructVector}
+import org.apache.arrow.vector.types.pojo.Field
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, Literal, Unevaluable}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+
+import org.apache.comet.shims.CometExprTraitShim
+
+/**
+ * Compiles a bound [[Expression]] plus an input schema into a [[CometBatchKernel]] that fuses
+ * Arrow input reads, expression evaluation, and Arrow output writes into one Janino-compiled
+ * method per (expression, schema) pair.
+ *
+ * The kernel is generic over Catalyst expressions; it does not know or assume that the bound tree
+ * came from a `ScalaUDF`. Today's only consumer is
+ * [[org.apache.comet.udf.codegen.CometScalaUDFCodegen]], but a future consumer (Spark
+ * `WholeStageCodegenExec` integration, a non-UDF batch evaluator) can drive this class directly.
+ *
+ * Constraints: single output vector per kernel (whole projections need a multi-output extension);
+ * per-row scalar evaluation only (aggregation, window, generator rejected by [[canHandle]]).
+ *
+ * Input- and output-side emission live in [[CometBatchKernelCodegenInput]] and
+ * [[CometBatchKernelCodegenOutput]]. This file owns the [[ArrowColumnSpec]] vocabulary, the
+ * [[canHandle]] / [[allocateOutput]] / [[compile]] / [[generateSource]] entry points, and
+ * cross-cutting kernel-shape decisions (null-intolerant short-circuit, CSE variant).
+ *
+ * The generated kernel '''is''' the `InternalRow` that Spark's `BoundReference.genCode` reads
+ * from: `ctx.INPUT_ROW = "row"` plus `InternalRow row = this;` inside `process` routes
+ * `row.getUTF8String(ord)` to the kernel's own typed getter (final method, constant ordinal; JIT
+ * devirtualizes and folds). `row` rather than `this` because Spark's `splitExpressions` passes
+ * INPUT_ROW as a helper-method parameter name and `this` is a reserved Java keyword.
+ */
+object CometBatchKernelCodegen extends Logging with CometExprTraitShim {
+
+ /**
+ * Resolve an Arrow vector class by its simple name, using the same classloader the codegen uses
+ * internally. Intended for tests: the `common` module shades `org.apache.arrow` to
+ * `org.apache.comet.shaded.arrow`, so `classOf[VarCharVector]` at a call site in an unshaded
+ * module refers to a different [[Class]] object than the one the codegen compares against.
+ * Callers pass a simple name and get back the class the production code actually uses.
+ */
+ def vectorClassBySimpleName(name: String): Class[_ <: ValueVector] = name match {
+ case "BitVector" => classOf[BitVector]
+ case "TinyIntVector" => classOf[TinyIntVector]
+ case "SmallIntVector" => classOf[SmallIntVector]
+ case "IntVector" => classOf[IntVector]
+ case "BigIntVector" => classOf[BigIntVector]
+ case "Float4Vector" => classOf[Float4Vector]
+ case "Float8Vector" => classOf[Float8Vector]
+ case "DecimalVector" => classOf[DecimalVector]
+ case "DateDayVector" => classOf[DateDayVector]
+ case "TimeStampMicroVector" => classOf[TimeStampMicroVector]
+ case "TimeStampMicroTZVector" => classOf[TimeStampMicroTZVector]
+ case "VarCharVector" => classOf[VarCharVector]
+ case "VarBinaryVector" => classOf[VarBinaryVector]
+ case other => throw new IllegalArgumentException(s"unknown Arrow vector class: $other")
+ }
+
+ /**
+ * Type surface the kernel covers, on both the input getter side and the output writer side.
+ * Recursive: `ArrayType` / `StructType` / `MapType` are supported when their children are.
+ * Input and output use a single predicate today; if they ever need to diverge, split this back
+ * into per-direction methods.
+ */
+ def isSupportedDataType(dt: DataType): Boolean = dt match {
+ case BooleanType | ByteType | ShortType | IntegerType | LongType => true
+ case FloatType | DoubleType => true
+ case _: DecimalType => true
+ case _: StringType | _: BinaryType => true
+ case DateType | TimestampType | TimestampNTZType => true
+ case ArrayType(inner, _) => isSupportedDataType(inner)
+ case st: StructType => st.fields.forall(f => isSupportedDataType(f.dataType))
+ case mt: MapType => isSupportedDataType(mt.keyType) && isSupportedDataType(mt.valueType)
+ case _ => false
+ }
+
+ /**
+ * Plan-time predicate: can the codegen dispatcher handle this bound expression end to end?
+ * `None` greenlights the serde to emit the codegen proto; `Some(reason)` forces a Spark
+ * fallback (typically `withInfo(...) + None`) rather than crashing the Janino compile at
+ * execute time.
+ *
+ * Checks every `BoundReference`'s data type and the root `expr.dataType` against
+ * [[isSupportedDataType]], and rejects aggregates / generators. Intermediate nodes are not
+ * checked: only leaves (row reads) and the root (output write) touch Arrow.
+ */
+ def canHandle(boundExpr: Expression): Option[String] = {
+ if (!isSupportedDataType(boundExpr.dataType)) {
+ return Some(s"codegen dispatch: unsupported output type ${boundExpr.dataType}")
+ }
+ // Reject expressions that can't be safely compiled or cached:
+ // - AggregateFunction / Generator: non-scalar bridge shape.
+ // - CodegenFallback: opts out of `doGenCode`, which our compile path assumes works.
+ // Passing one in would emit interpreted-eval glue that our kernel can't splice cleanly.
+ // - Unevaluable: unresolved plan markers. Shouldn't reach a serde, but cheap to guard.
+ // `isCodegenInertUnevaluable` lets the shim exclude version-specific leaves that are
+ // `Unevaluable` but never touched by codegen (e.g. Spark 4.0's `ResolvedCollation`, which
+ // lives in `Collate.collation` as a type marker; `Collate.genCode` delegates to its child).
+ //
+ // TODO(hof-lambdas): the `CodegenFallback` rule rejects `NamedLambdaVariable`, which flags
+ // every higher-order function (`ArrayTransform`, `ArrayAggregate`, `ArrayExists`,
+ // `ArrayFilter`, `ZipWith`, `MapFilter`, etc.) as unsupported. The variable is
+ // `CodegenFallback` only in isolation; the surrounding HOF binds its `value` field inline
+ // as part of its own
+ // `doGenCode`, and the resulting Java compiles fine. Loosening this would unlock
+ // element-iteration over `Array` / `Array