Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
16 commits
Select commit Hold shift + click to select a range
9e3f31f
feat: route additional scalar expressions through codegen dispatcher …
andygrove May 30, 2026
048a485
feat: route Incompatible expressions through codegen dispatcher by de…
andygrove May 30, 2026
087c62c
Merge remote-tracking branch 'apache/main' into codegen-dispatch-scal…
andygrove Jun 2, 2026
18dbbda
feat: route map_from_entries Incompatible case through codegen dispat…
andygrove Jun 2, 2026
ca8a09c
feat: route map_concat through codegen dispatcher
andygrove Jun 2, 2026
3758dd0
Merge remote-tracking branch 'apache/main' into codegen-dispatch-scal…
andygrove Jun 2, 2026
2211e41
docs: mark codegen-dispatch expressions as supported in expressions.md
andygrove Jun 2, 2026
955a346
test: update fallback expectations for codegen-dispatched Incompatibl…
andygrove Jun 2, 2026
de00c79
Merge remote-tracking branch 'apache/main' into codegen-dispatch-scal…
andygrove Jun 4, 2026
ce594c8
docs: drop stale map_concat note
andygrove Jun 4, 2026
585980f
docs: drop redundant codegen-dispatch comments
andygrove Jun 4, 2026
3a9ec2d
refactor: make codegen-dispatch enrollment opt-in via CodegenDispatch…
andygrove Jun 4, 2026
74d6124
test: align collation reverse with codegen dispatch and cover native …
andygrove Jun 5, 2026
cc529f1
Merge remote-tracking branch 'apache/main' into codegen-dispatch-scal…
andygrove Jun 10, 2026
31e0fcd
test: assert codegen dispatch keeps NTZ and non-UTC temporal exprs na…
andygrove Jun 10, 2026
8498450
test: cover fallback and allowIncompatible paths for NTZ/non-UTC temp…
andygrove Jun 10, 2026
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
44 changes: 22 additions & 22 deletions docs/source/user-guide/latest/expressions.md
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ The tables below list every Spark built-in expression with its current status.
| `element_at` | ✅ | MapType input falls back |
| `flatten` | ✅ | Binary/struct/map elements fall back |
| `get` | ✅ | |
| `sequence` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `sequence` | | |
| `shuffle` | 🔜 | Random array shuffle |
| `slice` | ✅ | Native ([#4149](https://github.com/apache/datafusion-comet/issues/4149)) |
| `sort_array` | ✅ | Nested struct/null arrays fall back |
Expand Down Expand Up @@ -200,7 +200,7 @@ The tables below list every Spark built-in expression with its current status.
| `coalesce` | ✅ | |
| `if` | ✅ | |
| `ifnull` | ✅ | |
| `nanvl` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `nanvl` | | |
| `nullif` | ✅ | |
| `nullifzero` | ✅ | Lowers to `if`/`=` (Spark 4.0+) |
| `nvl` | ✅ | |
Expand Down Expand Up @@ -373,7 +373,7 @@ All higher-order functions are planned via [#4224](https://github.com/apache/dat
| --- | --- | --- |
| `element_at` | ✅ | MapType input falls back |
| `map` | 🔜 | Constructs a map |
| `map_concat` | 🔜 | Concatenates maps |
| `map_concat` | | |
| `map_contains_key` | ✅ | |
| `map_entries` | ✅ | |
| `map_from_arrays` | ✅ | |
Expand Down Expand Up @@ -403,11 +403,11 @@ All higher-order functions are planned via [#4224](https://github.com/apache/dat
| `atan2` | ✅ | |
| `atanh` | ✅ | |
| `bin` | ✅ | |
| `bround` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `bround` | | |
| `cbrt` | ✅ | |
| `ceil` | ✅ | Two-arg form falls back |
| `ceiling` | ✅ | |
| `conv` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `conv` | | |
| `cos` | ✅ | |
| `cosh` | ✅ | |
| `cot` | ✅ | |
Expand All @@ -421,17 +421,17 @@ All higher-order functions are planned via [#4224](https://github.com/apache/dat
| `floor` | ✅ | Two-arg form falls back |
| `greatest` | ✅ | |
| `hex` | ✅ | |
| `hypot` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `hypot` | | |
| `least` | ✅ | |
| `ln` | ✅ | |
| `log` | ✅ | |
| `log10` | ✅ | |
| `log1p` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `log1p` | | |
| `log2` | ✅ | |
| `mod` | ✅ | |
| `negative` | ✅ | |
| `pi` | ✅ | |
| `pmod` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `pmod` | | |
| `positive` | ✅ | |
| `pow` | ✅ | |
| `power` | ✅ | |
Expand Down Expand Up @@ -540,29 +540,29 @@ All higher-order functions are planned via [#4224](https://github.com/apache/dat
| `concat_ws` | ✅ | |
| `contains` | ✅ | |
| `decode` | ✅ | |
| `elt` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `elt` | | |
| `encode` | 🔜 | Lowers to `StaticInvoke(encode)` (not allowlisted); falls back |
| `endswith` | ✅ | |
| `find_in_set` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `format_number` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `format_string` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `find_in_set` | | |
| `format_number` | | |
| `format_string` | | |
| `initcap` | ✅ | |
| `instr` | ✅ | |
| `lcase` | ✅ | |
| `left` | ✅ | |
| `len` | ✅ | |
| `length` | ✅ | |
| `levenshtein` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `locate` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `levenshtein` | | |
| `locate` | | |
| `lower` | ✅ | |
| `lpad` | ✅ | |
| `ltrim` | ✅ | |
| `luhn_check` | ✅ | Native via `StaticInvoke` (tests: luhn_check.sql) |
| `mask` | 🔜 | Data masking |
| `octet_length` | ✅ | |
| `overlay` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `position` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `printf` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `overlay` | | |
| `position` | | |
| `printf` | | |
| `regexp_count` | 🔜 | tracking [#4098](https://github.com/apache/datafusion-comet/issues/4098) |
| `regexp_extract` | 🔜 | tracking [#4098](https://github.com/apache/datafusion-comet/issues/4098) |
| `regexp_extract_all` | 🔜 | tracking [#4098](https://github.com/apache/datafusion-comet/issues/4098) |
Expand All @@ -574,7 +574,7 @@ All higher-order functions are planned via [#4224](https://github.com/apache/dat
| `right` | ✅ | |
| `rpad` | ✅ | |
| `rtrim` | ✅ | |
| `soundex` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `soundex` | | |
| `space` | ✅ | |
| `split` | ✅ | |
| `split_part` | 🔜 | Lowers to `element_at(StringSplitSQL(...))`; `StringSplitSQL` falls back ([#4561](https://github.com/apache/datafusion-comet/issues/4561)) |
Expand All @@ -583,15 +583,15 @@ All higher-order functions are planned via [#4224](https://github.com/apache/dat
| `substring` | ✅ | |
| `substring_index` | ✅ | |
| `to_binary` | ✅ | Hex form accelerated; other formats fall back |
| `to_char` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `to_number` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `to_varchar` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `to_char` | | |
| `to_number` | | |
| `to_varchar` | | |
| `translate` | ✅ | |
| `trim` | ✅ | |
| `try_to_binary` | 🔜 | Lowers to `TryEval(...)`, which falls back |
| `try_to_number` | 🔜 | TRY variant of `to_number` |
| `ucase` | ✅ | |
| `unbase64` | 🔜 | [#4538](https://github.com/apache/datafusion-comet/issues/4538) |
| `unbase64` | | |
| `upper` | ✅ | |

---
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,3 +96,15 @@ trait CometExpressionSerde[T <: Expression] {
*/
def convert(expr: T, inputs: Seq[Attribute], binding: Boolean): Option[ExprOuterClass.Expr]
}

/**
* Opt-in marker for expression serdes that have a native implementation which is `Incompatible`
* with Spark for some inputs. When such an expression reports `Incompatible` and the user has not
* enabled `allowIncompatible` for it, mixing in this trait routes it through the JVM codegen
* dispatcher (running Spark's own `doGenCode` inside the Comet pipeline) instead of falling the
* projection back to Spark, so it stays native while still matching Spark exactly.
*
* Enrollment is opt-in: only serdes that explicitly mix this in are routed through the
* dispatcher. Every other `Incompatible` expression falls back to Spark.
*/
trait CodegenDispatchFallback { self: CometExpressionSerde[_] => }
53 changes: 43 additions & 10 deletions spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,8 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim {
classOf[Flatten] -> CometFlatten,
classOf[GetArrayItem] -> CometGetArrayItem,
classOf[Size] -> CometSize,
classOf[ArraysZip] -> CometArraysZip)
classOf[ArraysZip] -> CometArraysZip,
classOf[Sequence] -> CometSequence)

private val conditionalExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] =
Map(classOf[CaseWhen] -> CometCaseWhen, classOf[If] -> CometIf)
Expand Down Expand Up @@ -143,7 +144,15 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim {
classOf[UnaryMinus] -> CometUnaryMinus,
classOf[Unhex] -> CometUnhex,
classOf[Abs] -> CometAbs,
classOf[Bin] -> CometScalarFunction("bin"))
classOf[Bin] -> CometScalarFunction("bin"),
classOf[Hypot] -> CometHypot,
classOf[NaNvl] -> CometNaNvl,
classOf[BRound] -> CometBRound,
classOf[Conv] -> CometConv,
classOf[Log1p] -> CometLog1p,
classOf[Pmod] -> CometPmod,
classOf[WidthBucket] -> CometWidthBucket,
classOf[UnaryPositive] -> CometUnaryPositive)

private[comet] val mapExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map(
classOf[GetMapValue] -> CometMapExtract,
Expand All @@ -153,6 +162,7 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim {
classOf[MapFromArrays] -> CometMapFromArrays,
classOf[MapContainsKey] -> CometMapContainsKey,
classOf[MapFromEntries] -> CometMapFromEntries,
classOf[MapConcat] -> CometMapConcat,
classOf[StringToMap] -> CometStrToMap)

private[comet] val structExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] =
Expand Down Expand Up @@ -184,6 +194,7 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim {
classOf[GetJsonObject] -> CometGetJsonObject,
classOf[InitCap] -> CometInitCap,
classOf[Length] -> CometLength,
classOf[Levenshtein] -> CometLevenshtein,
classOf[Like] -> CometLike,
classOf[Lower] -> CometLower,
classOf[OctetLength] -> CometScalarFunction("octet_length"),
Expand All @@ -210,7 +221,17 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim {
classOf[Right] -> CometRight,
classOf[Substring] -> CometSubstring,
classOf[SubstringIndex] -> CometSubstringIndex,
classOf[Upper] -> CometUpper)
classOf[Upper] -> CometUpper,
classOf[Elt] -> CometElt,
classOf[FindInSet] -> CometFindInSet,
classOf[FormatNumber] -> CometFormatNumber,
classOf[FormatString] -> CometFormatString,
classOf[Overlay] -> CometOverlay,
classOf[SoundEx] -> CometSoundEx,
classOf[StringLocate] -> CometStringLocate,
classOf[UnBase64] -> CometUnBase64,
classOf[ToCharacter] -> CometToCharacter,
classOf[ToNumber] -> CometToNumber)

private val bitwiseExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map(
classOf[BitwiseAnd] -> CometBitwiseAnd,
Expand Down Expand Up @@ -725,13 +746,25 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim {
}
handler.convert(expr, inputs, binding)
} else {
val optionalNotes = notes.map(str => s" ($str)").getOrElse("")
withFallbackReason(
expr,
s"$expr is not fully compatible with Spark$optionalNotes. To enable it anyway, " +
s"set ${CometConf.getExprAllowIncompatConfigKey(exprConfName)}=true. " +
s"${CometConf.COMPAT_GUIDE}.")
None
// Expressions that opt in via `CodegenDispatchFallback` route their `Incompatible`
// result through the JVM codegen dispatcher (Spark's own `doGenCode` inside the Comet
// pipeline) so the projection stays native while still matching Spark. Everything else
// falls back to Spark. Falling back is also the result when the dispatcher cannot
// handle the expression.
val dispatched = handler match {
case _: CodegenDispatchFallback =>
CometScalaUDF.emitJvmCodegenDispatch(expr, inputs, binding)
case _ => None
}
dispatched.orElse {
val optionalNotes = notes.map(str => s" ($str)").getOrElse("")
withFallbackReason(
expr,
s"$expr is not fully compatible with Spark$optionalNotes. To enable it anyway, " +
s"set ${CometConf.getExprAllowIncompatConfigKey(exprConfName)}=true. " +
s"${CometConf.COMPAT_GUIDE}.")
None
}
}
case Compatible(notes) =>
if (notes.isDefined) {
Expand Down
4 changes: 3 additions & 1 deletion spark/src/main/scala/org/apache/comet/serde/arrays.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ package org.apache.comet.serde
import scala.annotation.tailrec
import scala.jdk.CollectionConverters._

import org.apache.spark.sql.catalyst.expressions.{And, ArrayAppend, ArrayContains, ArrayExcept, ArrayFilter, ArrayInsert, ArrayIntersect, ArrayJoin, ArrayMax, ArrayMin, ArrayPosition, ArrayRemove, ArrayRepeat, ArraysOverlap, ArraysZip, ArrayUnion, Attribute, Cast, CreateArray, ElementAt, EmptyRow, Expression, Flatten, GetArrayItem, IsNotNull, Literal, Reverse, Size, Slice, SortArray}
import org.apache.spark.sql.catalyst.expressions.{And, ArrayAppend, ArrayContains, ArrayExcept, ArrayFilter, ArrayInsert, ArrayIntersect, ArrayJoin, ArrayMax, ArrayMin, ArrayPosition, ArrayRemove, ArrayRepeat, ArraysOverlap, ArraysZip, ArrayUnion, Attribute, Cast, CreateArray, ElementAt, EmptyRow, Expression, Flatten, GetArrayItem, IsNotNull, Literal, Reverse, Sequence, Size, Slice, SortArray}
import org.apache.spark.sql.catalyst.util.GenericArrayData
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -843,3 +843,5 @@ trait ArraysBase {
}
}
}

object CometSequence extends CometCodegenDispatch[Sequence]
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,10 @@ import org.apache.spark.sql.types.ArrayType
import org.apache.comet.serde.ExprOuterClass.Expr
import org.apache.comet.shims.CometTypeShim

object CometReverse extends CometScalarFunction[Reverse]("reverse") with CometTypeShim {
object CometReverse
extends CometScalarFunction[Reverse]("reverse")
with CometTypeShim
with CodegenDispatchFallback {

// Spark 4.0 widens the string branch of Reverse to accept collated strings and propagates the
// collation through dataType. The native reverse UDF reverses code units and produces UTF8
Expand Down
12 changes: 7 additions & 5 deletions spark/src/main/scala/org/apache/comet/serde/datetime.scala
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ object CometQuarter extends CometExpressionSerde[Quarter] with CometExprGetDateF
}
}

object CometHour extends CometExpressionSerde[Hour] {
object CometHour extends CometExpressionSerde[Hour] with CodegenDispatchFallback {

val incompatReason: String = "Incorrectly applies timezone conversion to TimestampNTZ inputs" +
" (https://github.com/apache/datafusion-comet/issues/3180)"
Expand Down Expand Up @@ -222,7 +222,7 @@ object CometHour extends CometExpressionSerde[Hour] {
}
}

object CometMinute extends CometExpressionSerde[Minute] {
object CometMinute extends CometExpressionSerde[Minute] with CodegenDispatchFallback {

override def getIncompatibleReasons(): Seq[String] = Seq(
"Incorrectly applies timezone conversion to TimestampNTZ inputs" +
Expand Down Expand Up @@ -264,7 +264,7 @@ object CometMinute extends CometExpressionSerde[Minute] {
}
}

object CometSecond extends CometExpressionSerde[Second] {
object CometSecond extends CometExpressionSerde[Second] with CodegenDispatchFallback {

override def getIncompatibleReasons(): Seq[String] = Seq(
"Incorrectly applies timezone conversion to TimestampNTZ inputs" +
Expand Down Expand Up @@ -509,7 +509,7 @@ object CometUnixDate extends CometExpressionSerde[UnixDate] {
}
}

object CometTruncDate extends CometExpressionSerde[TruncDate] {
object CometTruncDate extends CometExpressionSerde[TruncDate] with CodegenDispatchFallback {

val supportedFormats: Seq[String] =
Seq("year", "yyyy", "yy", "quarter", "mon", "month", "mm", "week")
Expand Down Expand Up @@ -551,7 +551,9 @@ object CometTruncDate extends CometExpressionSerde[TruncDate] {
}
}

object CometTruncTimestamp extends CometExpressionSerde[TruncTimestamp] {
object CometTruncTimestamp
extends CometExpressionSerde[TruncTimestamp]
with CodegenDispatchFallback {

override def getIncompatibleReasons(): Seq[String] = Seq(
"Produces incorrect results when used with non-UTC timezones. Compatible when timezone is" +
Expand Down
6 changes: 5 additions & 1 deletion spark/src/main/scala/org/apache/comet/serde/maps.scala
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,9 @@ object CometMapContainsKey extends CometExpressionSerde[MapContainsKey] {
}
}

object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from_entries") {
object CometMapFromEntries
extends CometScalarFunction[MapFromEntries]("map_from_entries")
with CodegenDispatchFallback {
val keyUnsupportedReason =
"`BinaryType` is not supported as a map key in `map_from_entries`"
val valueUnsupportedReason =
Expand Down Expand Up @@ -163,3 +165,5 @@ object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from
}

object CometStrToMap extends CometScalarFunction[StringToMap]("str_to_map")

object CometMapConcat extends CometCodegenDispatch[MapConcat]
18 changes: 17 additions & 1 deletion spark/src/main/scala/org/apache/comet/serde/math.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

package org.apache.comet.serde

import org.apache.spark.sql.catalyst.expressions.{Abs, Add, Atan2, Attribute, Ceil, CheckOverflow, Expression, Floor, Hex, If, LessThanOrEqual, Literal, Log, Log10, Log2, Logarithm, Unhex}
import org.apache.spark.sql.catalyst.expressions.{Abs, Add, Atan2, Attribute, BRound, Ceil, CheckOverflow, Conv, Expression, Floor, Hex, Hypot, If, LessThanOrEqual, Literal, Log, Log10, Log1p, Log2, Logarithm, NaNvl, Pmod, UnaryPositive, Unhex, WidthBucket}
import org.apache.spark.sql.types.{DecimalType, DoubleType, NumericType}

import org.apache.comet.CometSparkSessionExtensions.withFallbackReason
Expand Down Expand Up @@ -242,3 +242,19 @@ object CometCheckOverflow extends CometExpressionSerde[CheckOverflow] {
}
}
}

object CometHypot extends CometCodegenDispatch[Hypot]

object CometNaNvl extends CometCodegenDispatch[NaNvl]

object CometBRound extends CometCodegenDispatch[BRound]

object CometConv extends CometCodegenDispatch[Conv]

object CometLog1p extends CometCodegenDispatch[Log1p]

object CometPmod extends CometCodegenDispatch[Pmod]

object CometWidthBucket extends CometCodegenDispatch[WidthBucket]

object CometUnaryPositive extends CometCodegenDispatch[UnaryPositive]
26 changes: 25 additions & 1 deletion spark/src/main/scala/org/apache/comet/serde/strings.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ package org.apache.comet.serde

import java.util.Locale

import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, ConcatWs, Expression, GetJsonObject, If, InitCap, IsNull, Left, Length, Like, Literal, Lower, RegExpExtract, RegExpExtractAll, RegExpInStr, RegExpReplace, Right, RLike, StringLPad, StringRepeat, StringReplace, StringRPad, StringSplit, Substring, SubstringIndex, Upper}
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, ConcatWs, Elt, Expression, FindInSet, FormatNumber, FormatString, GetJsonObject, If, InitCap, IsNull, Left, Length, Levenshtein, Like, Literal, Lower, Overlay, RegExpExtract, RegExpExtractAll, RegExpInStr, RegExpReplace, Right, RLike, SoundEx, StringLocate, StringLPad, StringRepeat, StringReplace, StringRPad, StringSplit, Substring, SubstringIndex, ToCharacter, ToNumber, UnBase64, Upper}
import org.apache.spark.sql.types.{BinaryType, DataTypes, LongType, StringType}
import org.apache.spark.unsafe.types.UTF8String

Expand Down Expand Up @@ -546,3 +546,27 @@ trait CommonStringExprs {
}
}
}

// Expressions routed through the JVM codegen dispatcher: no native implementation, so Spark's own
// doGenCode runs inside the Comet pipeline, matching Spark exactly.
object CometLevenshtein extends CometCodegenDispatch[Levenshtein]

object CometElt extends CometCodegenDispatch[Elt]

object CometFindInSet extends CometCodegenDispatch[FindInSet]

object CometFormatNumber extends CometCodegenDispatch[FormatNumber]

object CometFormatString extends CometCodegenDispatch[FormatString]

object CometOverlay extends CometCodegenDispatch[Overlay]

object CometSoundEx extends CometCodegenDispatch[SoundEx]

object CometStringLocate extends CometCodegenDispatch[StringLocate]

object CometUnBase64 extends CometCodegenDispatch[UnBase64]

object CometToCharacter extends CometCodegenDispatch[ToCharacter]

object CometToNumber extends CometCodegenDispatch[ToNumber]
Loading
Loading