Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add a native function namespace manager #23358

Open
wants to merge 8 commits into
base: master
Choose a base branch
from

Conversation

pdabre12
Copy link
Contributor

@pdabre12 pdabre12 commented Aug 1, 2024

Description

Adds a native function namespace manager

Motivation and Context

To help resolve : #23000

Impact

Test Plan

Unit and end-to-end tests. More comprehensive end-to-end tests will be written in the future.

Contributor checklist

  • Please make sure your submission complies with our development, formatting, commit message, and attribution guidelines.
  • PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced.
  • Documented new properties (with its default value), SQL syntax, functions, or other functionality.
  • If release notes are required, they follow the release notes guidelines.
  • Adequate tests were added if applicable.
  • CI passed.
== RELEASE NOTES ==

Coordinator Plugin Changes
* Add a native function namespace manager :pr:`23358`

Depends on : #23671

Copy link

linux-foundation-easycla bot commented Aug 1, 2024

CLA Signed

The committers listed above are authorized under a signed CLA.

@pdabre12 pdabre12 force-pushed the native-function-namespace-manager branch from 49d3b9d to 5ed5a18 Compare August 13, 2024 23:37
@pdabre12 pdabre12 force-pushed the native-function-namespace-manager branch 2 times, most recently from bcb09bc to ce7ca4b Compare August 17, 2024 00:06
steveburnett
steveburnett previously approved these changes Sep 10, 2024
Copy link
Contributor

@steveburnett steveburnett left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM! (docs)

Pull branch, new local doc build, the doc looks good. Thanks!

@pdabre12 pdabre12 force-pushed the native-function-namespace-manager branch from 4d00991 to c3ed29f Compare September 13, 2024 23:03
@pdabre12 pdabre12 changed the title [WIP] Native function namespace manager [WIP] Add a native function namespace manager Sep 13, 2024
steveburnett
steveburnett previously approved these changes Sep 25, 2024
Copy link
Contributor

@steveburnett steveburnett left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM! (docs)

Pull updated branch, new local doc build, doc looks good. Thanks!

@pdabre12 pdabre12 force-pushed the native-function-namespace-manager branch 2 times, most recently from 3fde055 to ebfa361 Compare September 27, 2024 05:55
@pdabre12 pdabre12 changed the title [WIP] Add a native function namespace manager Add a native function namespace manager Sep 27, 2024
@pdabre12 pdabre12 force-pushed the native-function-namespace-manager branch from f203d25 to 88e1152 Compare November 1, 2024 01:09
@pdabre12 pdabre12 marked this pull request as ready for review November 1, 2024 16:40
@tdcmeehan tdcmeehan self-assigned this Nov 11, 2024
Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just some initial comments

Comment on lines 153 to 154
if ((parameter.getKind() != ParameterKind.TYPE) &&
(parameter.getKind() != ParameterKind.NAMED_TYPE)) {
Copy link
Contributor

@tdcmeehan tdcmeehan Nov 11, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
if ((parameter.getKind() != ParameterKind.TYPE) &&
(parameter.getKind() != ParameterKind.NAMED_TYPE)) {
if (!(parameter.getKind() == ParameterKind.TYPE ||
parameter.getKind() == ParameterKind.NAMED_TYPE)) {

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@tdcmeehan I thought the original is easier to read. but your suggestion should have || instead of &&.

Copy link
Contributor

@tdcmeehan tdcmeehan Nov 15, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Edited. I read this as "if the parameter kind is not a TYPE OR a NAMED TYPE", which to me is just a more natural way of describing the outcome than "if the parameter is not a TYPE AND not a NAMED TYPE".

But I just realized we could just do this, which seems better:

    public List<TypeSignature> getTypeOrNamedTypeParametersAsTypeSignatures()
    {
        List<TypeSignature> result = new ArrayList<>();
        for (TypeSignatureParameter parameter : parameters) {
            switch (parameter.getKind()) {
                case TYPE:
                    result.add(parameter.getTypeSignature());
                    break;
                case NAMED_TYPE:
                    result.add(parameter.getNamedTypeSignature().getTypeSignature());
                    break;
                default:
                    throw new IllegalStateException(
                            format("Expected all parameters to be of kind TYPE or NAMED_TYPE but [%s] kind was found for parameter: [%s]",
                                    parameter.getKind(), parameter));
            }
        }
        return unmodifiableList(result);
    }

@@ -245,7 +246,7 @@ public CompletableFuture<SqlFunctionResult> executeFunction(String source, Funct
typeManager.getType(functionMetadata.getReturnType()));
}

private static PrestoException convertToPrestoException(UncheckedExecutionException exception, String failureMessage)
protected static PrestoException convertToPrestoException(UncheckedExecutionException exception, String failureMessage)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just reimplement where needed, don't change the visibility.

@@ -362,10 +363,13 @@ protected AggregationFunctionImplementation sqlInvokedFunctionToAggregationImple
"Need aggregationMetadata to get aggregation function implementation");

AggregationFunctionMetadata aggregationMetadata = function.getAggregationMetadata().get();
List<Type> parameters = function.getSignature().getArgumentTypes().stream().map(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remind me why we need this please?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

presto-grpc-api/pom.xml Outdated Show resolved Hide resolved
presto-jdbc/pom.xml Outdated Show resolved Hide resolved
presto-main/pom.xml Outdated Show resolved Hide resolved
@@ -256,14 +276,14 @@ public CacheStatsMBean getFunctionResolutionCacheStats()
public void loadFunctionNamespaceManager(
String functionNamespaceManagerName,
String catalogName,
Map<String, String> properties)
Map<String, String> properties,
Optional<NodeManager> nodeManager)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is it optional?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because nodeManager is only required while loading the NativeFunctionNamespaceManager and not the other function namespace managers, hence I thought making it optional is the right way.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we supply it always and just have other namespace managers not use it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also in some of the places where loadFunctionNamespaceManager is called I do not have access to an instance of the NodeManager.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Refactored qualifyObjectName from static to instance method

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add utility functions to resolve intermediate type in aggregate funct…

Thank you for the really extensive tests for this utility. I think the tests should be broken down to the various different types, because the tests are rather large.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[native] Fix bug when handling multiple params and no params aggregat…

Can you help me understand when this bug manifests? Is there any way to add a unit test for it?

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[native] Add additional function metadata to function signatures endp…

Looks good, but there is a workspace.xml file that needs to be removed.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a new module presto-function-namespace-managers-common

Previous feedback remains, let's keep common as a very slim module.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[native] Introduce native function namespace SPI and integrate with s…

if (name.getOriginalParts().size() != 3) {
throw new PrestoException(FUNCTION_NOT_FOUND, format("Functions that are not temporary or builtin must be referenced by 'catalog.schema.function_name', found: %s", name));
}
return QualifiedObjectName.valueOf(name.getParts().get(0), name.getParts().get(1), name.getParts().get(2));
}

public void configureDefaultNamespace(String defaultNamespacePrefixString)
{
String pattern = "[a-z]+\\.[a-z]+";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's use a constant for this.

throws IllegalStateException
{
try {
throw new UnsupportedOperationException();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we just throw UOE?

return signature.toString();
}

private static void checkArgument(boolean condition, String message, Object... args)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can use the Guava utility.

return latestFunctionsTemp;
}

private void populateNameSpaceManager(UdfFunctionSignatureMap udfFunctionSignatureMap)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

populateNamespaceManager

private Map<SqlFunctionId, SqlInvokedFunction> bootstrapNamespace()
{
UdfFunctionSignatureMap nativeFunctionSignatureMap = functionDefinitionProvider.getUdfDefinition(nodeManager);
Map<SqlFunctionId, SqlInvokedFunction> latestFunctionsTemp = new ConcurrentHashMap<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please just use an immutable map, there is no need for concurrency here.

UdfFunctionSignatureMap nativeFunctionSignatureMap = functionDefinitionProvider.getUdfDefinition(nodeManager);
Map<SqlFunctionId, SqlInvokedFunction> latestFunctionsTemp = new ConcurrentHashMap<>();
if (nativeFunctionSignatureMap == null || nativeFunctionSignatureMap.isEmpty()) {
return latestFunctionsTemp;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

return ImmutableMap.of();

return functionMetadataManager.getSpecializedFunctionKey(signature);
}

private Map<SqlFunctionId, SqlInvokedFunction> bootstrapNamespace()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is how I would suggest the concurrency control to go.

Wherever we mutate latestFunctions, synchronize that. If it's lightweight and in-memory, just synchronize the method. This applies to createFunction and bootstrapNamespace`.

parameterTypeSignature = parameter.getTypeSignature();
}

if (parameterTypeSignature.getParameters().isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's very clearly call out the hack, where it is needed and why. Let's prefix the comment with // HACK: so it's obvious.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems like this is several methods long, so let's add an end comment that says when the hack is over.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

[native] Add e2e native function validation tests with sidecar

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add OpenAPI documentation for /v1/functions

throw new PrestoException(TOO_MANY_SIDECARS, "Expected exactly one coordinator sidecar, but found " + coordinatorSidecars.size());
}
return getOnlyElement(coordinatorSidecars);
return coordinatorSidecars.iterator().next();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same comment as on the other PR that we should choose randomly.

Comment on lines 153 to 154
if ((parameter.getKind() != ParameterKind.TYPE) &&
(parameter.getKind() != ParameterKind.NAMED_TYPE)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@tdcmeehan I thought the original is easier to read. but your suggestion should have || instead of &&.

@@ -146,6 +146,25 @@ public List<TypeSignature> getTypeParametersAsTypeSignatures()
return result;
}

public List<TypeSignature> getTypeOrNamedTypeParametersAsTypeSignatures()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you instead modify getTypeParametersAsTypeSignatures. I don't think there's any reason NamedTypeParameters are excluded there.


// If parameter params are of type long e.g decimal(15, 2) or
// of type varchar e.g decimal(i4, i5), we don't need to recursively call the function on its params
if (parameter.getParameters().isEmpty() || areParametersLongOrVariable(parameter.getParameters())) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what about parameters that are none of these kinds (long_enum, varchar_enum, distinct_type)?

also, since you've been looking at these different parameter kinds, would be great to add some javadoc to the enum to explain what the different parameter kinds mean.

{
private TypeSignatureUtils() {}

public static TypeSignature resolveIntermediateType(TypeSignature actualTypeSignature, List<TypeSignature> actualTypeSignatureParameters, List<TypeSignature> expectedTypeSignatureParameters)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we need special logic to handle aggregation function intermediate types?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We only need special logic to handle generic aggregate functions.
To resolve the intermediate type for generic aggregate functions, we rely on utility functions from TypeSignatureUtils. Specifically, we use the parameter types passed down in the function signatures to determine and insert the appropriate intermediateType.

  • Why do we need this?
    This is required because aggregate function metadata for generic aggregate functions contain generic intermediate args i.e K,V and we need a way to resolve these args.

Example:
max function:
For the max function, here is the relevant metadata:

{
  "aggregateMetadata": {
    "intermediateType": "t",
    "isOrderSensitive": false
  },
  "docString": "native.default.max",
  "functionKind": "AGGREGATE",
  "outputType": "t",
  "paramTypes": [
    "t"
  ],
  "routineCharacteristics": {
    "determinism": "DETERMINISTIC",
    "language": "CPP",
    "nullCallClause": "CALLED_ON_NULL_INPUT"
  },
  "schema": "default",
  "typeVariableConstraints": [
    {
      "comparableRequired": true,
      "name": "t",
      "nonDecimalNumericRequired": false,
      "orderableRequired": true,
      "variadicBound": ""
    }
  ],
  "variableArity": false
}

In this case:

  • The functionHandle passed is an instance of NativeFunctionHandle.
  • Let's suppose the signature that gets passed down is native.default.max(bigint):bigint, we use the passed down param type bigint to resolve the intermediate type t as bigint.

max_by function:

{
  "aggregateMetadata": {
    "intermediateType": "row(bigint,array(boolean),array(v))",
    "isOrderSensitive": true
  },
  "docString": "native.default.max_by",
  "functionKind": "AGGREGATE",
  "outputType": "array(v)",
  "paramTypes": [
    "v",
    "boolean",
    "bigint"
  ],
  "routineCharacteristics": {
    "determinism": "DETERMINISTIC",
    "language": "CPP",
    "nullCallClause": "CALLED_ON_NULL_INPUT"
  },
  "schema": "default",
  "typeVariableConstraints": [
    {
      "comparableRequired": false,
      "name": "v",
      "nonDecimalNumericRequired": false,
      "orderableRequired": false,
      "variadicBound": ""
    }
  ],
  "variableArity": false
}

In this case:

  • The functionHandle passed is an instance of NativeFunctionHandle.
  • Let's suppose the signature that gets passed down is native.default.max_by(integer, boolean, bigint):array(integer), we use the passed down param types integer to resolve the intermediate type "intermediateType": "row(bigint,array(boolean),array(v)) as "intermediateType": "row(bigint,array(boolean),array(integer))

@@ -845,15 +845,17 @@ void VeloxQueryPlanConverterBase::toAggregations(
auto pos = functionId.find(";", start + 1);
if (pos == std::string::npos) {
auto argumentType = functionId.substr(start + 1);
aggregate.rawInputTypes.push_back(
stringToType(argumentType, typeParser_));
if (!argumentType.empty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you explain the bug? Can you share an example of how this manifests. Also, add a test. If this is a bug fix, also consider pulling this out into a separate PR to merge earlier.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Opened a PR: #24059

@pdabre12 pdabre12 force-pushed the native-function-namespace-manager branch from 88e1152 to d5ccae3 Compare November 21, 2024 20:43
@pdabre12
Copy link
Contributor Author

@tdcmeehan @rschlussel
Thank you for your suggestions.
Addressed your comments, please have another look.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add utility functions to resolve intermediate type in generic aggrega…

return resolveTypeSignatures(actualTypeSignature, typeSignatureMap).getTypeSignature();
}

public static Map<TypeSignature, TypeSignature> getTypeSignatureMap(List<TypeSignature> parameters, List<TypeSignature> argumentTypes)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe this is only used in the tests and in the above method. Can we make it private? I think there's sufficient unit test coverage through resolveIntermediateType.

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a new module presto-function-namespace-managers-common

Much better, thanks!

Copy link
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking pretty good, some more comments.


import static java.util.Objects.requireNonNull;

public class NativeFunctionNamespaceManagerProvider
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this class? Seems unused?

}

@Config("presto.default-namespace")
public PluginManagerConfig setDefaultNamespacePrefix(String defaultNamespacePrefix)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rather than match a default namespace with a namespace that is configured in the function namespace manager, just make it a boolean flag property on the function namespace manager's config that it is the default namespace. If none have that specified, use presto.default, if more than one have that specified, then throw.

The place to add this property would be in StaticFunctionNamespaceStore.

return aggregationImplementationByHandle.get(sqlFunctionHandle);
}

private static SqlInvokedFunction copyFunction(SqlInvokedFunction function)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SqlInvokedFunction is immutable, why do we need to do this copy?

parameterTypeSignature = parameter.getTypeSignature();
}

if (parameterTypeSignature.getParameters().isEmpty()) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems like this is several methods long, so let's add an end comment that says when the hack is over.

@Override
public AggregationMetadata getAggregationMetadata()
{
return null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe it should be safe or even preferable to throw here, since they should be called.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Add fail-fast function validation support for Presto C++
4 participants