Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
39 commits
Select commit Hold shift + click to select a range
9727d00
builtin/session fucntion qualifier.
srielau Dec 22, 2025
9a232f9
Table function support
srielau Dec 22, 2025
ad66ec0
Fixes
srielau Dec 22, 2025
f0d5b4e
Fix
srielau Dec 23, 2025
dcc309e
Fix TF
srielau Dec 23, 2025
16c2cc2
More fixes
srielau Dec 23, 2025
c59bed0
Rework design
srielau Dec 23, 2025
1e267c1
Redesign to use origunal clone
srielau Dec 23, 2025
7fc7f59
fix
srielau Dec 23, 2025
8b17973
Fixes
srielau Dec 24, 2025
1a213b8
More fixes
srielau Dec 24, 2025
65119e8
Fix extension fucntions show fucntions
srielau Dec 24, 2025
4fff9c6
Fix ML fucntions
srielau Dec 24, 2025
c8173e2
update golden files
srielau Dec 24, 2025
a0cb898
more fixes
srielau Dec 24, 2025
ba18966
Fix persistent function caching to use unqualified keys
srielau Dec 25, 2025
f20377d
Fix persistent function key conflicts - use qualified keys
srielau Dec 25, 2025
897f218
WIP: Add debug logging to trace view function resolution
srielau Dec 25, 2025
aa553aa
Fix view function resolution bug - use correct isBuiltin check
srielau Dec 25, 2025
fa70575
Clean up temporary test files
srielau Dec 25, 2025
1bfce30
Add comprehensive function registry API analysis
srielau Dec 25, 2025
f8b8fca
Document difference between builtin and internal functions
srielau Dec 25, 2025
6921fcb
Fix CreateSQLFunctionParserSuite to expect new error class
srielau Dec 25, 2025
8816772
Remove temporary analysis/documentation files from git
srielau Dec 25, 2025
3f41dd8
Remove temporary test SQL files from git
srielau Dec 25, 2025
d8fc5be
Refactor: eliminate scalar/table function duplication and improve enc…
srielau Dec 25, 2025
ae33dde
Refactor: improve code idiomaticity and remove unused variables
srielau Dec 25, 2025
a19ba60
Remove duplicate SQL test for function qualification
srielau Dec 25, 2025
8f38c3f
Refine design
srielau Dec 25, 2025
62c6518
Fixes
srielau Dec 25, 2025
fc826e6
More cleanup
srielau Dec 25, 2025
49a3468
"unified" registry
srielau Dec 26, 2025
4a35263
Simple path based resolution
srielau Dec 26, 2025
4da0a7f
refactor
srielau Dec 26, 2025
8c0f655
Fix temp functiomn reference in view
srielau Dec 26, 2025
c55ac38
Ready for review
srielau Dec 26, 2025
22f88ec
Enforce no overloading of session scalar vs table functions
srielau Dec 26, 2025
c1842d8
Fix perf problem with lookups.
srielau Dec 26, 2025
dc28d0b
support auxiliarry statements
srielau Dec 29, 2025
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
6 changes: 6 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -4034,6 +4034,12 @@
},
"sqlState" : "42823"
},
"INVALID_TEMP_OBJ_QUALIFIER" : {
"message" : [
"Temporary <objectType> <objectName> cannot be qualified with <qualifier>. Temporary objects can only be qualified with SESSION or SYSTEM.SESSION."
],
"sqlState" : "42602"
},
"INVALID_TEMP_OBJ_REFERENCE" : {
"message" : [
"Cannot create the persistent object <objName> of the type <obj> because it references to the temporary object <tempObjName> of the type <tempObj>. Please make the temporary object <tempObjName> persistent, or make the persistent object <objName> temporary."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2077,25 +2077,25 @@ class Analyzer(

plan.resolveExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) {
case f @ UnresolvedFunction(nameParts, _, _, _, _, _, _) =>
if (functionResolution.lookupBuiltinOrTempFunction(nameParts, Some(f)).isDefined) {
// Check cache first for persistent functions to avoid repeated external catalog lookups.
val CatalogAndIdentifier(catalog, ident) =
relationResolution.expandIdentifier(nameParts)
val fullName = normalizeFuncName(
(catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq)

if (externalFunctionNameSet.contains(fullName)) {
// Already validated this persistent function - skip validation.
f
} else {
val CatalogAndIdentifier(catalog, ident) =
relationResolution.expandIdentifier(nameParts)
val fullName =
normalizeFuncName((catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq)
if (externalFunctionNameSet.contains(fullName)) {
f
} else if (catalog.asFunctionCatalog.functionExists(ident)) {
// Not in cache - validate the function.
// Returns true if builtin/temp, false if persistent.
val isBuiltinOrTemp = functionResolution.validateFunctionExistence(nameParts, f)

// If it's a persistent function, add to cache.
if (!isBuiltinOrTemp) {
externalFunctionNameSet.add(fullName)
f
} else {
val catalogPath = (catalog.name() +: catalogManager.currentNamespace).mkString(".")
throw QueryCompilationErrors.unresolvedRoutineError(
nameParts,
Seq("system.builtin", "system.session", catalogPath),
f.origin)
}
f
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.variant._
import org.apache.spark.sql.catalyst.expressions.xml._
import org.apache.spark.sql.catalyst.plans.logical.{FunctionBuilderBase, Generate, LogicalPlan, OneRowRelation, PythonWorkerLogs, Range}
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -78,10 +79,17 @@ trait FunctionRegistryBase[T] {
/* Create or replace a temporary function. */
final def createOrReplaceTempFunction(
name: String, builder: FunctionBuilder, source: String): Unit = {
registerFunction(
FunctionIdentifier(name),
builder,
source)
// Internal functions (source="internal") are NOT qualified with
// CatalogManager.SESSION_NAMESPACE database because they use a separate
// internal registry and are resolved differently
val identifier = if (source == "internal") {
FunctionIdentifier(name)
} else {
// Regular temporary functions are qualified with CatalogManager.SESSION_NAMESPACE
// to enable coexistence with builtin functions of the same name
FunctionIdentifier(name, Some(CatalogManager.SESSION_NAMESPACE))
}
registerFunction(identifier, builder, source)
}

@throws[AnalysisException]("If function does not exist")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,40 @@ class FunctionResolution(

private val trimWarningEnabled = new AtomicBoolean(true)

/**
* Checks if a multi-part name is qualified with a specific namespace.
* Supports both 2-part (namespace.name) and 3-part (system.namespace.name) qualifications.
*
* @param nameParts The multi-part name to check
* @param namespace The namespace to check for (e.g., "builtin", "session")
* @return true if qualified with the given namespace
*/
private def isQualifiedWithNamespace(nameParts: Seq[String], namespace: String): Boolean = {
nameParts.length match {
case 2 => nameParts.head.equalsIgnoreCase(namespace)
case 3 =>
nameParts(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) &&
nameParts(1).equalsIgnoreCase(namespace)
case _ => false
}
}

/**
* Check if a function name is qualified as a builtin function.
* Valid forms: builtin.func or system.builtin.func
*/
private def maybeBuiltinFunctionName(nameParts: Seq[String]): Boolean = {
isQualifiedWithNamespace(nameParts, CatalogManager.BUILTIN_NAMESPACE)
}

/**
* Check if a function name is qualified as a session temporary function.
* Valid forms: session.func or system.session.func
*/
private def maybeTempFunctionName(nameParts: Seq[String]): Boolean = {
isQualifiedWithNamespace(nameParts, CatalogManager.SESSION_NAMESPACE)
}

def resolveFunction(u: UnresolvedFunction): Expression = {
withPosition(u) {
resolveBuiltinOrTempFunction(u.nameParts, u.arguments, u).getOrElse {
Expand Down Expand Up @@ -74,7 +108,14 @@ class FunctionResolution(
u: Option[UnresolvedFunction]): Option[ExpressionInfo] = {
if (name.size == 1 && u.exists(_.isInternal)) {
FunctionRegistry.internal.lookupFunction(FunctionIdentifier(name.head))
} else if (maybeBuiltinFunctionName(name)) {
// Explicitly qualified as builtin - lookup only builtin
v1SessionCatalog.lookupBuiltinFunction(name.last)
} else if (maybeTempFunctionName(name)) {
// Explicitly qualified as temp - lookup only temp
v1SessionCatalog.lookupTempFunction(name.last)
} else if (name.size == 1) {
// Unqualified - check temp first (shadowing), then builtin
v1SessionCatalog.lookupBuiltinOrTempFunction(name.head)
} else {
None
Expand All @@ -89,17 +130,87 @@ class FunctionResolution(
}
}

/**
* Validates that a function exists and can be used in the given context.
* This is used by the LookupFunctions analyzer rule for early validation.
*
* @param nameParts The function name parts.
* @param node The UnresolvedFunction node for error reporting.
* @return true if the function is a builtin or temporary function, false if it's persistent.
*/
def validateFunctionExistence(
nameParts: Seq[String],
node: UnresolvedFunction): Boolean = {

// Check if function exists as scalar function.
val existsAsScalar = lookupBuiltinOrTempFunction(nameParts, Some(node)).isDefined

if (existsAsScalar) {
// Function exists in scalar registry, can be used in scalar context.
return true // It's a builtin or temp function
}

// Check if function exists as table function.
val existsAsTable = lookupBuiltinOrTempTableFunction(nameParts).isDefined

if (existsAsTable) {
// Function exists ONLY in table registry - cannot be used in scalar context.
throw QueryCompilationErrors.notAScalarFunctionError(nameParts.mkString("."), node)
}

// Not found in builtin/temp registries - check external catalog.
val CatalogAndIdentifier(catalog, ident) = relationResolution.expandIdentifier(nameParts)

if (!catalog.asFunctionCatalog.functionExists(ident)) {
// Function doesn't exist anywhere - throw UNRESOLVED_ROUTINE error.
val catalogPath = (catalog.name() +: catalogManager.currentNamespace).mkString(".")
throw QueryCompilationErrors.unresolvedRoutineError(
nameParts,
Seq("system.builtin", "system.session", catalogPath),
node.origin)
}

// Function exists in external catalog - it's persistent.
false // Not a builtin or temp function
}

def resolveBuiltinOrTempFunction(
name: Seq[String],
arguments: Seq[Expression],
u: UnresolvedFunction): Option[Expression] = {

// Step 1: Try to resolve as scalar function
val expression = if (name.size == 1 && u.isInternal) {
Option(FunctionRegistry.internal.lookupFunction(FunctionIdentifier(name.head), arguments))
} else if (maybeBuiltinFunctionName(name)) {
// Explicitly qualified as builtin - resolve only builtin
v1SessionCatalog.resolveBuiltinFunction(name.last, arguments)
} else if (maybeTempFunctionName(name)) {
// Explicitly qualified as temp - resolve only temp
v1SessionCatalog.resolveTempFunction(name.last, arguments)
} else if (name.size == 1) {
v1SessionCatalog.resolveBuiltinOrTempFunction(name.head, arguments)
// For unqualified names, check cross-type shadowing before resolving
// If a temp table function exists with this name, it shadows any builtin scalar function
val funcName = name.head
if (v1SessionCatalog.lookupTempTableFunction(funcName).isDefined) {
// Temp table function exists - throw error
throw QueryCompilationErrors.notAScalarFunctionError(name.mkString("."), u)
} else {
// No temp table function - safe to resolve as scalar
v1SessionCatalog.resolveBuiltinOrTempFunction(funcName, arguments)
}
} else {
None
}

// Step 2: Check for table-only functions (cross-type error detection)
// If not found as scalar, check if it exists as a table-only function
if (expression.isEmpty && name.size == 1) {
if (v1SessionCatalog.lookupBuiltinOrTempTableFunction(name.head).isDefined) {
throw QueryCompilationErrors.notAScalarFunctionError(name.mkString("."), u)
}
}

expression.map { func =>
validateFunction(func, arguments.length, u)
}
Expand All @@ -108,11 +219,37 @@ class FunctionResolution(
def resolveBuiltinOrTempTableFunction(
name: Seq[String],
arguments: Seq[Expression]): Option[LogicalPlan] = {
if (name.length == 1) {
v1SessionCatalog.resolveBuiltinOrTempTableFunction(name.head, arguments)

// Step 1: Try to resolve as table function
val tableFunctionResult = if (name.length == 1) {
// For unqualified names, check cross-type shadowing before resolving
// If a temp scalar function exists with this name, it shadows any builtin table function
val funcName = name.head
if (v1SessionCatalog.lookupTempFunction(funcName).isDefined) {
// Temp scalar function exists - will throw error below
None
} else {
// No temp scalar function - safe to resolve as table function
v1SessionCatalog.resolveBuiltinOrTempTableFunction(funcName, arguments)
}
} else {
None
}

// Step 2: Fallback to scalar registry for type mismatch detection
// Architecture: Generators are one-way (table-to-scalar extraction). If a function exists
// ONLY as a scalar function and is used in table context, throw specific error.
//
// Note: This also handles cross-type shadowing. If a temp scalar function shadows a builtin
// table function, the check above returns None, and we fall through here to detect
// it's a scalar-only function and throw NOT_A_TABLE_FUNCTION.
if (tableFunctionResult.isEmpty && name.length == 1) {
if (v1SessionCatalog.lookupBuiltinOrTempFunction(name.head).isDefined) {
throw QueryCompilationErrors.notATableFunctionError(name.mkString("."))
}
}

tableFunctionResult
}

private def resolveV1Function(
Expand Down
Loading