diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f635b7a02734..d6154b1e2666 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2102,7 +2102,7 @@ class Analyzer( throw QueryCompilationErrors.expectPersistentFuncError( nameParts.head, cmd, mismatchHint, u) } else { - ResolvedNonPersistentFunc(nameParts.head, V1Function(info)) + ResolvedNonPersistentFunc(nameParts.head, V1Function.metadataOnly(info)) } }.getOrElse { val CatalogAndIdentifier(catalog, ident) = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 0d5f30bd2d78..cc8ac07126cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -90,11 +90,16 @@ trait FunctionRegistryBase[T] { /* List all of the registered function names. */ def listFunction(): Seq[FunctionIdentifier] - /* Get the class of the registered function by specified name. */ - def lookupFunction(name: FunctionIdentifier): Option[ExpressionInfo] + /* Get both ExpressionInfo and FunctionBuilder in a single lookup. */ + def lookupFunctionEntry(name: FunctionIdentifier): Option[(ExpressionInfo, FunctionBuilder)] + + /* Get the ExpressionInfo of the registered function by specified name. */ + def lookupFunction(name: FunctionIdentifier): Option[ExpressionInfo] = + lookupFunctionEntry(name).map(_._1) /* Get the builder of the registered function by specified name. */ - def lookupFunctionBuilder(name: FunctionIdentifier): Option[FunctionBuilder] + def lookupFunctionBuilder(name: FunctionIdentifier): Option[FunctionBuilder] = + lookupFunctionEntry(name).map(_._2) /** Drop a function and return whether the function existed. */ def dropFunction(name: FunctionIdentifier): Boolean @@ -245,13 +250,9 @@ trait SimpleFunctionRegistryBase[T] extends FunctionRegistryBase[T] with Logging functionBuilders.iterator.map(_._1).toList } - override def lookupFunction(name: FunctionIdentifier): Option[ExpressionInfo] = synchronized { - functionBuilders.get(normalizeFuncName(name)).map(_._1) - } - - override def lookupFunctionBuilder( - name: FunctionIdentifier): Option[FunctionBuilder] = synchronized { - functionBuilders.get(normalizeFuncName(name)).map(_._2) + override def lookupFunctionEntry( + name: FunctionIdentifier): Option[(ExpressionInfo, FunctionBuilder)] = synchronized { + functionBuilders.get(normalizeFuncName(name)) } override def dropFunction(name: FunctionIdentifier): Boolean = synchronized { @@ -281,11 +282,8 @@ trait EmptyFunctionRegistryBase[T] extends FunctionRegistryBase[T] { throw SparkUnsupportedOperationException() } - override def lookupFunction(name: FunctionIdentifier): Option[ExpressionInfo] = { - throw SparkUnsupportedOperationException() - } - - override def lookupFunctionBuilder(name: FunctionIdentifier): Option[FunctionBuilder] = { + override def lookupFunctionEntry( + name: FunctionIdentifier): Option[(ExpressionInfo, FunctionBuilder)] = { throw SparkUnsupportedOperationException() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala index 8d6e2931a73b..29f4db65def0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionResolution.scala @@ -52,11 +52,11 @@ class FunctionResolution( val CatalogAndIdentifier(catalog, ident) = relationResolution.expandIdentifier(u.nameParts) catalog.asFunctionCatalog.loadFunction(ident) match { - case V1Function(_) => - // this triggers the second time v1 function resolution but should be cheap - // (no RPC to external catalog), since the metadata has been already cached - // in FunctionRegistry during the above `catalog.loadFunction` call. - resolveV1Function(ident.asFunctionIdentifier, u.arguments, u) + case v1Func: V1Function => + // V1Function has a lazy builder - invoke() triggers resource loading + // and builder creation only on first invocation + val func = v1Func.invoke(u.arguments) + validateFunction(func, u.arguments.length, u) case unboundV2Func => resolveV2Function(unboundV2Func, u.arguments, u) } @@ -118,14 +118,6 @@ class FunctionResolution( } } - private def resolveV1Function( - ident: FunctionIdentifier, - arguments: Seq[Expression], - u: UnresolvedFunction): Expression = { - val func = v1SessionCatalog.resolvePersistentFunction(ident, arguments) - validateFunction(func, arguments.length, u) - } - private def validateFunction( func: Expression, numArgs: Int, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 191e2091c40d..dc80e7acba0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkThrowable} import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -49,6 +48,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE +import org.apache.spark.sql.internal.connector.V1Function import org.apache.spark.sql.metricview.util.MetricViewPlanner import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType} import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils} @@ -2153,55 +2153,75 @@ class SessionCatalog( } /** - * Look up the `ExpressionInfo` of the given function by name if it's a persistent function. - * This supports both scalar and table functions. + * Look up a persistent function's ExpressionInfo by name (for DESCRIBE FUNCTION). + * This only fetches metadata without loading resources or creating builders. */ def lookupPersistentFunction(name: FunctionIdentifier): ExpressionInfo = { val qualifiedIdent = qualifyIdentifier(name) - val db = qualifiedIdent.database.get - val funcName = qualifiedIdent.funcName + // Check if already cached in either registry functionRegistry.lookupFunction(qualifiedIdent) .orElse(tableFunctionRegistry.lookupFunction(qualifiedIdent)) .getOrElse { - requireDbExists(db) - if (externalCatalog.functionExists(db, funcName)) { - val metadata = externalCatalog.getFunction(db, funcName) - if (metadata.isUserDefinedFunction) { - UserDefinedFunction.fromCatalogFunction(metadata, parser).toExpressionInfo - } else { - makeExprInfoForHiveFunction(metadata.copy(identifier = qualifiedIdent)) - } + val funcMetadata = fetchCatalogFunction(qualifiedIdent) + if (funcMetadata.isUserDefinedFunction) { + UserDefinedFunction.fromCatalogFunction(funcMetadata, parser).toExpressionInfo } else { - failFunctionLookup(name) + makeExprInfoForHiveFunction(funcMetadata) } } } /** - * Look up a persistent scalar function by name and resolves it to an Expression. + * Load a persistent scalar function by name. + * Returns V1Function with: + * - Eager info (from cache or catalog fetch, no resource loading) + * - Lazy builder (resource loading only on first invoke) + * + * This matches V1 behavior where DESCRIBE doesn't load resources. */ - def resolvePersistentFunction( - name: FunctionIdentifier, arguments: Seq[Expression]): Expression = { - resolvePersistentFunctionInternal[Expression]( - name, - arguments, - functionRegistry, - registerHiveFunc = func => - registerFunction( - func, - overrideIfExists = false, - registry = functionRegistry, - functionBuilder = makeFunctionBuilder(func) - ), - registerUserDefinedFunc = function => { - val builder = makeUserDefinedScalarFuncBuilder(function) - registerUserDefinedFunction[Expression]( - function = function, - overrideIfExists = false, - registry = functionRegistry, - functionBuilder = builder) - } - ) + def loadPersistentScalarFunction(name: FunctionIdentifier): V1Function = { + val qualifiedIdent = qualifyIdentifier(name) + + // Check cache first (no synchronization needed for reads) + functionRegistry.lookupFunctionEntry(qualifiedIdent) match { + case Some((cachedInfo, cachedBuilder)) => + // Already cached - return with eager builder + V1Function(cachedInfo, cachedBuilder) + + case None => + // Fetch metadata eagerly (no resource loading yet) + val funcMetadata = fetchCatalogFunction(qualifiedIdent) + val info = if (funcMetadata.isUserDefinedFunction) { + UserDefinedFunction.fromCatalogFunction(funcMetadata, parser).toExpressionInfo + } else { + makeExprInfoForHiveFunction(funcMetadata) + } + + // Builder factory - loads resources only on first invoke() + val builderFactory: () => FunctionBuilder = () => synchronized { + // Re-check cache (another thread may have loaded it) + functionRegistry.lookupFunctionBuilder(qualifiedIdent).getOrElse { + if (funcMetadata.isUserDefinedFunction) { + val udf = UserDefinedFunction.fromCatalogFunction(funcMetadata, parser) + registerUserDefinedFunction[Expression]( + udf, + overrideIfExists = false, + functionRegistry, + makeUserDefinedScalarFuncBuilder(udf)) + } else { + loadFunctionResources(funcMetadata.resources) + registerFunction( + funcMetadata, + overrideIfExists = false, + functionRegistry, + makeFunctionBuilder(funcMetadata)) + } + functionRegistry.lookupFunctionBuilder(qualifiedIdent).get + } + } + + V1Function(info, builderFactory) + } } /** @@ -2209,66 +2229,45 @@ class SessionCatalog( */ def resolvePersistentTableFunction( name: FunctionIdentifier, - arguments: Seq[Expression]): LogicalPlan = { - resolvePersistentFunctionInternal[LogicalPlan]( - name, - arguments, - tableFunctionRegistry, - // We don't support persistent Hive table functions yet. - registerHiveFunc = (func: CatalogFunction) => failFunctionLookup(name), - registerUserDefinedFunc = function => { - val builder = makeUserDefinedTableFuncBuilder(function) - registerUserDefinedFunction[LogicalPlan]( - function = function, - overrideIfExists = false, - registry = tableFunctionRegistry, - functionBuilder = builder) + arguments: Seq[Expression]): LogicalPlan = synchronized { + val qualifiedIdent = qualifyIdentifier(name) + if (tableFunctionRegistry.functionExists(qualifiedIdent)) { + // Already cached + tableFunctionRegistry.lookupFunction(qualifiedIdent, arguments) + } else { + // Load from catalog + val funcMetadata = fetchCatalogFunction(qualifiedIdent) + if (!funcMetadata.isUserDefinedFunction) { + // Hive table functions are not supported + failFunctionLookup(qualifiedIdent) } - ) + val udf = UserDefinedFunction.fromCatalogFunction(funcMetadata, parser) + registerUserDefinedFunction[LogicalPlan]( + udf, + overrideIfExists = false, + tableFunctionRegistry, + makeUserDefinedTableFuncBuilder(udf)) + tableFunctionRegistry.lookupFunction(qualifiedIdent, arguments) + } } - private def resolvePersistentFunctionInternal[T]( - name: FunctionIdentifier, - arguments: Seq[Expression], - registry: FunctionRegistryBase[T], - registerHiveFunc: CatalogFunction => Unit, - registerUserDefinedFunc: UserDefinedFunction => Unit): T = { - // `synchronized` is used to prevent multiple threads from concurrently resolving the - // same function that has not yet been loaded into the function registry. This is needed - // because calling `registerFunction` twice with `overrideIfExists = false` can lead to - // a FunctionAlreadyExistsException. - synchronized { - val qualifiedIdent = qualifyIdentifier(name) - val db = qualifiedIdent.database.get - val funcName = qualifiedIdent.funcName - if (registry.functionExists(qualifiedIdent)) { - // This function has been already loaded into the function registry. - registry.lookupFunction(qualifiedIdent, arguments) - } else { - // The function has not been loaded to the function registry, which means - // that the function is a persistent function (if it actually has been registered - // in the metastore). We need to first put the function in the function registry. - val catalogFunction = try { - externalCatalog.getFunction(db, funcName) - } catch { - case _: AnalysisException => failFunctionLookup(qualifiedIdent) - } - // Please note that qualifiedName is provided by the user. However, - // catalogFunction.identifier.unquotedString is returned by the underlying - // catalog. So, it is possible that qualifiedName is not exactly the same as - // catalogFunction.identifier.unquotedString (difference is on case-sensitivity). - // At here, we preserve the input from the user. - val funcMetadata = catalogFunction.copy(identifier = qualifiedIdent) - if (!catalogFunction.isUserDefinedFunction) { - loadFunctionResources(catalogFunction.resources) - registerHiveFunc(funcMetadata) - } else { - val function = UserDefinedFunction.fromCatalogFunction(funcMetadata, parser) - registerUserDefinedFunc(function) - } - // Now, we need to create the Expression. - registry.lookupFunction(qualifiedIdent, arguments) - } + /** + * Fetch a catalog function from the external catalog. + */ + private def fetchCatalogFunction(qualifiedIdent: FunctionIdentifier): CatalogFunction = { + val db = qualifiedIdent.database.get + val funcName = qualifiedIdent.funcName + requireDbExists(db) + try { + // Please note that qualifiedIdent is provided by the user. However, + // CatalogFunction.identifier is returned by the underlying catalog. + // So, it is possible that qualifiedIdent is not exactly the same as + // catalogFunction.identifier (difference is on case-sensitivity). + // At here, we preserve the input from the user. + externalCatalog.getFunction(db, funcName).copy(identifier = qualifiedIdent) + } catch { + case _: NoSuchPermanentFunctionException | _: NoSuchFunctionException => + failFunctionLookup(qualifiedIdent) } } @@ -2290,9 +2289,9 @@ class SessionCatalog( def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { if (name.database.isEmpty) { resolveBuiltinOrTempFunction(name.funcName, children) - .getOrElse(resolvePersistentFunction(name, children)) + .getOrElse(loadPersistentScalarFunction(name).invoke(children)) } else { - resolvePersistentFunction(name, children) + loadPersistentScalarFunction(name).invoke(children) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala index 488d1acf43ac..f741690e908b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ResolveDefaultColumnsUtil.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.connector.catalog.{CatalogManager, Column, DefaultVa import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.connector.V1Function import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ @@ -671,7 +670,7 @@ object ResolveDefaultColumns extends QueryErrorsBase throw SparkUnsupportedOperationException() } override def loadFunction(ident: Identifier): UnboundFunction = { - V1Function(v1Catalog.lookupPersistentFunction(ident.asFunctionIdentifier)) + v1Catalog.loadPersistentScalarFunction(ident.asFunctionIdentifier) } override def functionExists(ident: Identifier): Boolean = { v1Catalog.isPersistentFunction(ident.asFunctionIdentifier) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/V1Function.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/V1Function.scala index 4224a292a1e1..c4c22d64e46c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/V1Function.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/V1Function.scala @@ -17,15 +17,79 @@ package org.apache.spark.sql.internal.connector -import org.apache.spark.SparkUnsupportedOperationException -import org.apache.spark.sql.catalyst.expressions.ExpressionInfo +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, UnboundFunction} import org.apache.spark.sql.types.StructType -case class V1Function(info: ExpressionInfo) extends UnboundFunction { +/** + * A wrapper for V1 scalar functions that mirrors the V2 UnboundFunction pattern. + * + * V1Function has two responsibilities: + * 1. Provide ExpressionInfo for DESCRIBE FUNCTION (accessed via `info`) + * 2. Provide a function builder for invocation (accessed via `invoke()`) + * + * The key design is two-phase lazy loading: + * - `info` is computed eagerly or from cache (no resource loading) + * - `functionBuilder` is computed lazily on first `invoke()` (triggers resource loading) + * + * This matches V1 behavior where DESCRIBE doesn't load resources but invocation does. + */ +class V1Function private ( + val info: ExpressionInfo, + builderFactory: () => FunctionBuilder) extends UnboundFunction { + + /** + * Lazy function builder - only computed on first invoke(). + * For persistent functions, this triggers resource loading. + */ + private lazy val functionBuilder: FunctionBuilder = builderFactory() + + /** + * Invoke the function with the given arguments. + * This is the V1 equivalent of V2's bind() pattern. + * For persistent functions, first invocation triggers resource loading. + */ + def invoke(arguments: Seq[Expression]): Expression = functionBuilder(arguments) + + // UnboundFunction interface override def bind(inputType: StructType): BoundFunction = { - throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3110") + // V1 functions don't use the V2 bind() pattern - they use invoke() instead + throw SparkException.internalError("V1Function.bind() should not be called") } + override def name(): String = info.getName + override def description(): String = info.getUsage } + +object V1Function { + /** A placeholder builder for metadata-only V1Functions that throws on invocation. */ + private val metadataOnlyBuilder: FunctionBuilder = _ => + throw SparkException.internalError( + "Metadata-only V1Function should not be invoked") + + /** + * Create a V1Function with eager info and lazy builder. + * The builderFactory is called on first invoke(). + */ + def apply(info: ExpressionInfo, builderFactory: () => FunctionBuilder): V1Function = { + new V1Function(info, builderFactory) + } + + /** + * Create a V1Function with eager info and builder (for built-in/temp/cached functions). + */ + def apply(info: ExpressionInfo, builder: FunctionBuilder): V1Function = { + new V1Function(info, () => builder) + } + + /** + * Create a metadata-only V1Function (for DESCRIBE FUNCTION). + * If invoke() is called, it will throw an error. + */ + def metadataOnly(info: ExpressionInfo): V1Function = { + new V1Function(info, () => metadataOnlyBuilder) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala index 1ae3e3fa6860..183d37f972a2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/LookupFunctionsSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.{CatalogManager, FunctionCatalog, Identifier} import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.connector.V1Function import org.apache.spark.sql.util.CaseInsensitiveStringMap class LookupFunctionsSuite extends PlanTest { @@ -152,7 +151,7 @@ class CustomV2SessionCatalog(v1Catalog: SessionCatalog) extends FunctionCatalog } override def loadFunction(ident: Identifier): UnboundFunction = { - V1Function(v1Catalog.lookupPersistentFunction(ident.asFunctionIdentifier)) + v1Catalog.loadPersistentScalarFunction(ident.asFunctionIdentifier) } override def functionExists(ident: Identifier): Boolean = { v1Catalog.isPersistentFunction(ident.asFunctionIdentifier) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index a15ba2796202..7caadb0631f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -483,8 +483,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) if conf.useV1Command => ShowTablePropertiesCommand(ident, propertyKey, output) - case DescribeFunction(ResolvedNonPersistentFunc(_, V1Function(info)), extended) => - DescribeFunctionCommand(info, extended) + case DescribeFunction(ResolvedNonPersistentFunc(_, v1Func: V1Function), extended) => + DescribeFunctionCommand(v1Func.info, extended) case DescribeFunction(ResolvedPersistentFunc(catalog, _, func), extended) => if (isSessionCatalog(catalog)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index f11ddbc51d33..f596780979de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.connector.V1Function import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ @@ -475,7 +474,7 @@ class V2SessionCatalog(catalog: SessionCatalog) } override def loadFunction(ident: Identifier): UnboundFunction = { - V1Function(catalog.lookupPersistentFunction(ident.asFunctionIdentifier)) + catalog.loadPersistentScalarFunction(ident.asFunctionIdentifier) } override def listFunctions(namespace: Array[String]): Array[Identifier] = {