From 3767a711395fd6b3437e970fe662562df431bda5 Mon Sep 17 00:00:00 2001 From: Joe Abraham Date: Mon, 29 Sep 2025 10:15:53 +0530 Subject: [PATCH] feat(native): support custom schemas in native sidecar function registry --- .../presto_cpp/main/PrestoServer.cpp | 12 ++++ .../presto_cpp/main/connectors/CMakeLists.txt | 4 +- .../connectors/PrestoToVeloxConnector.cpp | 13 ++++ .../main/connectors/hive/CMakeLists.txt | 13 ++++ .../connectors/hive/functions/CMakeLists.txt | 13 ++++ .../functions/HiveFunctionRegistration.cpp | 38 ++++++++++++ .../hive/functions/HiveFunctionRegistration.h | 23 +++++++ .../hive/functions/InitcapFunction.h | 53 ++++++++++++++++ .../main/functions/FunctionMetadata.cpp | 62 +++++++++++++++++++ .../main/functions/FunctionMetadata.h | 3 + .../functions/tests/FunctionMetadataTest.cpp | 44 ++++++++++++- .../NativeFunctionDefinitionProvider.java | 24 ++++--- .../NativeSidecarPluginQueryRunnerUtils.java | 11 ++++ .../main/resources/rest_function_server.yaml | 42 +++++++++++++ 14 files changed, 346 insertions(+), 9 deletions(-) create mode 100644 presto-native-execution/presto_cpp/main/connectors/hive/CMakeLists.txt create mode 100644 presto-native-execution/presto_cpp/main/connectors/hive/functions/CMakeLists.txt create mode 100644 presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.cpp create mode 100644 presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.h create mode 100644 presto-native-execution/presto_cpp/main/connectors/hive/functions/InitcapFunction.h diff --git a/presto-native-execution/presto_cpp/main/PrestoServer.cpp b/presto-native-execution/presto_cpp/main/PrestoServer.cpp index 80b3531ceb05b..c41694b75035e 100644 --- a/presto-native-execution/presto_cpp/main/PrestoServer.cpp +++ b/presto-native-execution/presto_cpp/main/PrestoServer.cpp @@ -1669,6 +1669,18 @@ void PrestoServer::registerSidecarEndpoints() { proxygen::ResponseHandler* downstream) { http::sendOkResponse(downstream, getFunctionsMetadata()); }); + httpServer_->registerGet( + R"(/v1/functions/([^/]+))", + [](proxygen::HTTPMessage* /*message*/, + const std::vector& pathMatch) { + return new http::CallbackRequestHandler( + [catalog = pathMatch[1]]( + proxygen::HTTPMessage* /*message*/, + std::vector>& /*body*/, + proxygen::ResponseHandler* downstream) { + http::sendOkResponse(downstream, getFunctionsMetadata(catalog)); + }); + }); httpServer_->registerPost( "/v1/velox/plan", [server = this]( diff --git a/presto-native-execution/presto_cpp/main/connectors/CMakeLists.txt b/presto-native-execution/presto_cpp/main/connectors/CMakeLists.txt index 5ba165c0ad67d..483499d71bab6 100644 --- a/presto-native-execution/presto_cpp/main/connectors/CMakeLists.txt +++ b/presto-native-execution/presto_cpp/main/connectors/CMakeLists.txt @@ -20,4 +20,6 @@ if(PRESTO_ENABLE_ARROW_FLIGHT_CONNECTOR) endif() target_link_libraries(presto_connectors presto_velox_expr_conversion - velox_type_fbhive) + velox_type_fbhive presto_hive_functions) + +add_subdirectory(hive) diff --git a/presto-native-execution/presto_cpp/main/connectors/PrestoToVeloxConnector.cpp b/presto-native-execution/presto_cpp/main/connectors/PrestoToVeloxConnector.cpp index 9f5f4acf0824f..8eff6f2d55d12 100644 --- a/presto-native-execution/presto_cpp/main/connectors/PrestoToVeloxConnector.cpp +++ b/presto-native-execution/presto_cpp/main/connectors/PrestoToVeloxConnector.cpp @@ -13,11 +13,16 @@ */ #include "presto_cpp/main/connectors/PrestoToVeloxConnector.h" +#include +#include "presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.h" +#include "presto_cpp/main/connectors/hive/functions/InitcapFunction.h" +#include "presto_cpp/main/functions/dynamic_registry/DynamicFunctionRegistrar.h" #include "presto_cpp/main/types/PrestoToVeloxExpr.h" #include "presto_cpp/main/types/TypeParser.h" #include "presto_cpp/presto_protocol/connector/hive/HiveConnectorProtocol.h" #include "presto_cpp/presto_protocol/connector/iceberg/IcebergConnectorProtocol.h" #include "presto_cpp/presto_protocol/connector/tpch/TpchConnectorProtocol.h" +#include "velox/functions/FunctionRegistry.h" #include #include "velox/connectors/hive/HiveConnector.h" @@ -52,6 +57,14 @@ void registerPrestoToVeloxConnector( connectorName); protocol::registerConnectorProtocol( connectorName, std::move(connectorProtocol)); + + // Register hive-specific functions when hive catalog is detected. + // Delegate to generic Hive native function registrar which is idempotent. + if (connectorName == + velox::connector::hive::HiveConnectorFactory::kHiveConnectorName || + connectorName == std::string("hive-hadoop2")) { + hive::functions::registerHiveNativeFunctions(); + } } void unregisterPrestoToVeloxConnector(const std::string& connectorName) { diff --git a/presto-native-execution/presto_cpp/main/connectors/hive/CMakeLists.txt b/presto-native-execution/presto_cpp/main/connectors/hive/CMakeLists.txt new file mode 100644 index 0000000000000..fb98894c302db --- /dev/null +++ b/presto-native-execution/presto_cpp/main/connectors/hive/CMakeLists.txt @@ -0,0 +1,13 @@ +# Licensed 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. + +add_subdirectory(functions) diff --git a/presto-native-execution/presto_cpp/main/connectors/hive/functions/CMakeLists.txt b/presto-native-execution/presto_cpp/main/connectors/hive/functions/CMakeLists.txt new file mode 100644 index 0000000000000..5cbb814637f8e --- /dev/null +++ b/presto-native-execution/presto_cpp/main/connectors/hive/functions/CMakeLists.txt @@ -0,0 +1,13 @@ +# Licensed 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. +add_library(presto_hive_functions HiveFunctionRegistration.cpp) +target_link_libraries(presto_hive_functions velox_functions_prestosql) diff --git a/presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.cpp b/presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.cpp new file mode 100644 index 0000000000000..0aba8a5df84db --- /dev/null +++ b/presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.cpp @@ -0,0 +1,38 @@ +/* + * Licensed 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. + */ + +#include "presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.h" + +#include + +#include "presto_cpp/main/connectors/hive/functions/InitcapFunction.h" +#include "presto_cpp/main/functions/dynamic_registry/DynamicFunctionRegistrar.h" +#include "velox/functions/FunctionRegistry.h" + +namespace facebook::presto::hive::functions { + +namespace { +void registerHiveFunctions() { + // Register functions under the 'hive.default' namespace. + facebook::presto::registerPrestoFunction( + "initcap", "hive.default"); +} +} // namespace + +void registerHiveNativeFunctions() { + static std::once_flag once; + std::call_once(once, []() { registerHiveFunctions(); }); +} + +} // namespace facebook::presto::hive::functions diff --git a/presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.h b/presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.h new file mode 100644 index 0000000000000..338938f2bbb67 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/connectors/hive/functions/HiveFunctionRegistration.h @@ -0,0 +1,23 @@ +/* + * Licensed 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. + */ +#pragma once + +namespace facebook::presto::hive::functions { + +// Registers Hive-specific native functions into the 'hive.default' namespace. +// This method is safe to call multiple times; it performs one-time registration +// guarded by an internal call_once. +void registerHiveNativeFunctions(); + +} // namespace facebook::presto::hive::functions diff --git a/presto-native-execution/presto_cpp/main/connectors/hive/functions/InitcapFunction.h b/presto-native-execution/presto_cpp/main/connectors/hive/functions/InitcapFunction.h new file mode 100644 index 0000000000000..7d575a93fc563 --- /dev/null +++ b/presto-native-execution/presto_cpp/main/connectors/hive/functions/InitcapFunction.h @@ -0,0 +1,53 @@ +/* + * Licensed 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. + */ + +#pragma once + +#include "velox/functions/Macros.h" +#include "velox/functions/lib/string/StringImpl.h" + +using namespace facebook::velox; +using namespace facebook::velox::functions; +namespace facebook::presto::hive::functions { + +/// The InitCapFunction capitalizes the first character of each word in a +/// string, and lowercases the rest. +template +struct InitCapFunction { + VELOX_DEFINE_FUNCTION_TYPES(T); + + static constexpr bool is_default_ascii_behavior = true; + + FOLLY_ALWAYS_INLINE void call( + out_type& result, + const arg_type& input) { + stringImpl::initcap< + /*strictSpace=*/false, + /*isAscii=*/false, + /*turkishCasing=*/true, + /*greekFinalSigma=*/true>(result, input); + } + + FOLLY_ALWAYS_INLINE void callAscii( + out_type& result, + const arg_type& input) { + stringImpl::initcap< + /*strictSpace=*/false, + /*isAscii=*/true, + /*turkishCasing=*/true, + /*greekFinalSigma=*/true>(result, input); + } +}; + +} // namespace facebook::presto::hive::functions diff --git a/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.cpp b/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.cpp index ddcaa6f86cba3..2125e7484b1af 100644 --- a/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.cpp +++ b/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.cpp @@ -319,4 +319,66 @@ json getFunctionsMetadata() { return j; } +json getFunctionsMetadata(const std::string& catalog) { + json j; + + // Get metadata for all registered scalar functions in velox. + const auto signatures = getFunctionSignatures(); + static const std::unordered_set kBlockList = { + "row_constructor", "in", "is_null"}; + // Exclude aggregate companion functions (extract aggregate companion + // functions are registered as vector functions). + const auto aggregateFunctions = exec::aggregateFunctions().copy(); + for (const auto& entry : signatures) { + const auto name = entry.first; + // Skip internal functions. They don't have any prefix. + if (kBlockList.count(name) != 0 || + name.find("$internal$") != std::string::npos || + getScalarMetadata(name).companionFunction) { + continue; + } + + const auto parts = getFunctionNameParts(name); + if (parts[0] != catalog) { + continue; + } + const auto schema = parts[1]; + const auto function = parts[2]; + j[function] = buildScalarMetadata(name, schema, entry.second); + } + + // Get metadata for all registered aggregate functions in velox. + for (const auto& entry : aggregateFunctions) { + if (!aggregateFunctions.at(entry.first).metadata.companionFunction) { + const auto name = entry.first; + const auto parts = getFunctionNameParts(name); + if (parts[0] != catalog) { + continue; + } + const auto schema = parts[1]; + const auto function = parts[2]; + j[function] = + buildAggregateMetadata(name, schema, entry.second.signatures); + } + } + + // Get metadata for all registered window functions in velox. Skip aggregates + // as they have been processed. + const auto& functions = exec::windowFunctions(); + for (const auto& entry : functions) { + if (aggregateFunctions.count(entry.first) == 0) { + const auto name = entry.first; + const auto parts = getFunctionNameParts(entry.first); + if (parts[0] != catalog) { + continue; + } + const auto schema = parts[1]; + const auto function = parts[2]; + j[function] = buildWindowMetadata(name, schema, entry.second.signatures); + } + } + + return j; +} + } // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.h b/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.h index 7a39aebb95037..3f5d73afeb491 100644 --- a/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.h +++ b/presto-native-execution/presto_cpp/main/functions/FunctionMetadata.h @@ -21,4 +21,7 @@ namespace facebook::presto { // Returns metadata for all registered functions as json. nlohmann::json getFunctionsMetadata(); +// Returns metadata for all registered functions filtered by catalog as json. +nlohmann::json getFunctionsMetadata(const std::string& catalog); + } // namespace facebook::presto diff --git a/presto-native-execution/presto_cpp/main/functions/tests/FunctionMetadataTest.cpp b/presto-native-execution/presto_cpp/main/functions/tests/FunctionMetadataTest.cpp index 10b78e2ccef51..65bfc384403a3 100644 --- a/presto-native-execution/presto_cpp/main/functions/tests/FunctionMetadataTest.cpp +++ b/presto-native-execution/presto_cpp/main/functions/tests/FunctionMetadataTest.cpp @@ -26,7 +26,6 @@ using namespace facebook::presto; using json = nlohmann::json; static const std::string kPrestoDefaultPrefix = "presto.default."; -static const std::string kDefaultSchema = "default"; class FunctionMetadataTest : public ::testing::Test { protected: @@ -114,3 +113,46 @@ TEST_F(FunctionMetadataTest, transformKeys) { TEST_F(FunctionMetadataTest, variance) { testFunction("variance", "Variance.json", 5); } + +TEST_F(FunctionMetadataTest, GetFunctionsMetadataWithCatalog) { + // Test with the "presto" catalog that is registered in SetUpTestSuite + std::string catalog = "presto"; + auto metadata = getFunctionsMetadata(catalog); + + // The result should be a JSON object with function names as keys + ASSERT_TRUE(metadata.is_object()); + ASSERT_FALSE(metadata.empty()); + + // Verify that common functions are present + ASSERT_TRUE(metadata.contains("abs")); + ASSERT_TRUE(metadata.contains("mod")); + + // Each function should have an array of signatures + for (auto it = metadata.begin(); it != metadata.end(); ++it) { + ASSERT_TRUE(it.value().is_array()) << "Function: " << it.key(); + ASSERT_FALSE(it.value().empty()) << "Function: " << it.key(); + + // Each signature should have the required fields + for (const auto& signature : it.value()) { + ASSERT_TRUE(signature.contains("outputType")) << "Function: " << it.key(); + ASSERT_TRUE(signature.contains("paramTypes")) << "Function: " << it.key(); + ASSERT_TRUE(signature.contains("schema")) << "Function: " << it.key(); + ASSERT_TRUE(signature.contains("functionKind")) + << "Function: " << it.key(); + + // Schema should be "default" since we registered with "presto.default." + // prefix + EXPECT_EQ(signature["schema"], "default") << "Function: " << it.key(); + } + } +} + +TEST_F(FunctionMetadataTest, GetFunctionsMetadataWithNonExistentCatalog) { + // Test with a catalog that doesn't exist + std::string catalog = "nonexistent"; + auto metadata = getFunctionsMetadata(catalog); + + // When no functions match, it returns a null JSON value or empty object + // The default json() constructor creates a null value + ASSERT_TRUE(metadata.is_null() || (metadata.is_object() && metadata.empty())); +} diff --git a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionDefinitionProvider.java b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionDefinitionProvider.java index 677752dd4d2c7..191ebcffd027a 100644 --- a/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionDefinitionProvider.java +++ b/presto-native-sidecar-plugin/src/main/java/com/facebook/presto/sidecar/functionNamespace/NativeFunctionDefinitionProvider.java @@ -14,10 +14,12 @@ package com.facebook.presto.sidecar.functionNamespace; import com.facebook.airlift.http.client.HttpClient; +import com.facebook.airlift.http.client.HttpUriBuilder; import com.facebook.airlift.http.client.Request; import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; import com.facebook.presto.functionNamespace.JsonBasedUdfFunctionMetadata; +import com.facebook.presto.functionNamespace.ServingCatalog; import com.facebook.presto.functionNamespace.UdfFunctionSignatureMap; import com.facebook.presto.sidecar.ForSidecarInfo; import com.facebook.presto.spi.NodeManager; @@ -26,6 +28,7 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; +import java.net.URI; import java.util.List; import java.util.Map; @@ -42,32 +45,39 @@ public class NativeFunctionDefinitionProvider private final JsonCodec>> nativeFunctionSignatureMapJsonCodec; private final HttpClient httpClient; private final NativeFunctionNamespaceManagerConfig config; + private final String catalogName; @Inject public NativeFunctionDefinitionProvider( @ForSidecarInfo HttpClient httpClient, JsonCodec>> nativeFunctionSignatureMapJsonCodec, - NativeFunctionNamespaceManagerConfig config) + NativeFunctionNamespaceManagerConfig config, + @ServingCatalog String catalogName) { this.nativeFunctionSignatureMapJsonCodec = requireNonNull(nativeFunctionSignatureMapJsonCodec, "nativeFunctionSignatureMapJsonCodec is null"); this.httpClient = requireNonNull(httpClient, "httpClient is null"); this.config = requireNonNull(config, "config is null"); + this.catalogName = requireNonNull(catalogName, "catalogName is null"); } @Override public UdfFunctionSignatureMap getUdfDefinition(NodeManager nodeManager) { try { - Request request = - prepareGet().setUri( - getSidecarLocationOnStartup( - nodeManager, config.getSidecarNumRetries(), config.getSidecarRetryDelay().toMillis())).build(); - Map> nativeFunctionSignatureMap = httpClient.execute(request, createJsonResponseHandler(nativeFunctionSignatureMapJsonCodec)); + // Base endpoint: /v1/functions + URI baseUri = getSidecarLocationOnStartup( + nodeManager, config.getSidecarNumRetries(), config.getSidecarRetryDelay().toMillis()); + // Catalog-filtered endpoint: /v1/functions/{catalog} + URI catalogUri = HttpUriBuilder.uriBuilderFrom(baseUri).appendPath(catalogName).build(); + Request catalogRequest = prepareGet().setUri(catalogUri).build(); + Map> nativeFunctionSignatureMap = + httpClient.execute(catalogRequest, createJsonResponseHandler(nativeFunctionSignatureMapJsonCodec)); return new UdfFunctionSignatureMap(ImmutableMap.copyOf(nativeFunctionSignatureMap)); } catch (Exception e) { - throw new PrestoException(INVALID_ARGUMENTS, "Failed to get functions from sidecar.", e); + // Do not fall back to unfiltered endpoint to avoid cross-catalog leakage. + throw new PrestoException(INVALID_ARGUMENTS, String.format("Failed to get catalog-scoped functions from sidecar for catalog '%s'", catalogName), e); } } diff --git a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java index c8c7e1123f974..b871ac9d0d725 100644 --- a/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java +++ b/presto-native-sidecar-plugin/src/test/java/com/facebook/presto/sidecar/NativeSidecarPluginQueryRunnerUtils.java @@ -30,12 +30,23 @@ public static void setupNativeSidecarPlugin(QueryRunner queryRunner) queryRunner.loadSessionPropertyProvider( NativeSystemSessionPropertyProviderFactory.NAME, ImmutableMap.of()); + + // Register native catalog for built-in functions queryRunner.loadFunctionNamespaceManager( NativeFunctionNamespaceManagerFactory.NAME, "native", ImmutableMap.of( "supported-function-languages", "CPP", "function-implementation-type", "CPP")); + + // Register hive catalog for hive-specific functions + queryRunner.loadFunctionNamespaceManager( + NativeFunctionNamespaceManagerFactory.NAME, + "hive", + ImmutableMap.of( + "supported-function-languages", "CPP", + "function-implementation-type", "CPP")); + queryRunner.loadTypeManager(NativeTypeManagerFactory.NAME); queryRunner.loadPlanCheckerProviderManager("native", ImmutableMap.of()); queryRunner.installPlugin(new NativeSqlInvokedFunctionsPlugin()); diff --git a/presto-openapi/src/main/resources/rest_function_server.yaml b/presto-openapi/src/main/resources/rest_function_server.yaml index e9bab77abc10b..7a1cff5e3363c 100644 --- a/presto-openapi/src/main/resources/rest_function_server.yaml +++ b/presto-openapi/src/main/resources/rest_function_server.yaml @@ -41,6 +41,48 @@ paths: application/json: schema: $ref: '#/components/schemas/UdfSignatureMap' + /v1/functions/{catalog}: + parameters: + - name: catalog + in: path + required: true + schema: + type: string + description: The catalog name to filter functions by (e.g., 'native', 'hive'). + example: "hive" + get: + summary: Retrieve list of functions in the specified catalog. + description: | + This endpoint returns functions that are registered under a specific catalog namespace. + Functions in Presto can be namespaced by catalog (e.g., hive.default.initcap, native.default.abs). + This endpoint filters and returns only those functions that belong to the specified catalog. + + This is particularly useful for: + - Separating built-in functions from connector-specific functions + - Organizing custom C++ functions by their associated connector or catalog + - Supporting multiple function namespace managers with different catalogs + + Example: GET /v1/functions/hive returns only functions registered under the 'hive' catalog. + responses: + '200': + description: A map of function names to lists of function metadata for the specified catalog. + content: + application/json: + schema: + $ref: '#/components/schemas/UdfSignatureMap' + example: + initcap: + - docString: "initcap" + functionKind: "SCALAR" + outputType: "varchar" + paramTypes: ["varchar"] + schema: "default" + routineCharacteristics: + language: "CPP" + determinism: "DETERMINISTIC" + nullCallClause: "RETURNS_NULL_ON_NULL_INPUT" + '404': + description: The specified catalog was not found or contains no functions. /v1/functions/{schema}: parameters: - name: schema