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 7 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
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
7 changes: 7 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -172,6 +172,7 @@
<module>presto-kudu</module>
<module>presto-elasticsearch</module>
<module>presto-function-namespace-managers</module>
<module>presto-function-namespace-managers-common</module>
<module>presto-expressions</module>
<module>presto-benchmark-runner</module>
<module>presto-spark-classloader-interface</module>
Expand Down Expand Up @@ -954,6 +955,12 @@
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>com.facebook.presto</groupId>
<artifactId>presto-native-sidecar-plugin</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>com.facebook.hive</groupId>
<artifactId>hive-dwrf</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ private static Object fixValue(TypeSignature signature, Object value)
if (List.class.isAssignableFrom(value.getClass())) {
List<Object> fixedValue = new ArrayList<>();
for (Object object : List.class.cast(value)) {
fixedValue.add(fixValue(signature.getTypeParametersAsTypeSignatures().get(0), object));
fixedValue.add(fixValue(signature.getTypeOrNamedTypeParametersAsTypeSignatures().get(0), object));
}
return fixedValue;
}
Expand All @@ -122,8 +122,8 @@ else if (value.getClass() == String.class) {
}
if (signature.getBase().equals(MAP)) {
if (Map.class.isAssignableFrom(value.getClass())) {
TypeSignature keySignature = signature.getTypeParametersAsTypeSignatures().get(0);
TypeSignature valueSignature = signature.getTypeParametersAsTypeSignatures().get(1);
TypeSignature keySignature = signature.getTypeOrNamedTypeParametersAsTypeSignatures().get(0);
TypeSignature valueSignature = signature.getTypeOrNamedTypeParametersAsTypeSignatures().get(1);
Map<Object, Object> fixedValue = new HashMap<>();
for (Map.Entry<?, ?> entry : (Set<Map.Entry<?, ?>>) Map.class.cast(value).entrySet()) {
fixedValue.put(fixValue(keySignature, entry.getKey()), fixValue(valueSignature, entry.getValue()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,38 @@

import java.util.Optional;

/**
* The {@code ParameterKind} enum represents various kinds of parameters used in Presto's type system.
* The available parameter kinds are:
*
* <ul>
* <li><b>TYPE</b>:
* Used when the parameter itself is of type {@code TYPE_SIGNATURE}, representing a type definition.</li>
*
* <li><b>NAMED_TYPE</b>:
* Represents parameters that are explicitly named and can be referenced using their name.
* This is primarily used when the base type is a row type.</li>
*
* <li><b>LONG</b>:
* Used for types that take a long literal as a parameter. Examples include
* types like {@code decimal} and {@code varchar}.</li>
*
* <li><b>VARIABLE</b>:
* Used when variables are passed as parameters. This allows dynamic and flexible parameter handling.</li>
*
* <li><b>LONG_ENUM</b>:
* Represents a mapping of string values to long values. It is efficient for cases where
* symbolic names correspond to numeric values.</li>
*
* <li><b>VARCHAR_ENUM</b>:
* Represents a mapping of string values to string values. This is useful for symbolic names
* that do not require numeric representation.</li>
*
* <li><b>DISTINCT_TYPE</b>:
* Represents distinct user-defined types, enabling the creation of custom types in Presto's type system.</li>
* </ul>
*/

@ThriftEnum
public enum ParameterKind
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,17 +133,24 @@ public List<TypeSignatureParameter> getParameters()
return parameters;
}

public List<TypeSignature> getTypeParametersAsTypeSignatures()
public List<TypeSignature> getTypeOrNamedTypeParametersAsTypeSignatures()
pdabre12 marked this conversation as resolved.
Show resolved Hide resolved
{
List<TypeSignature> result = new ArrayList<>();
for (TypeSignatureParameter parameter : parameters) {
if (parameter.getKind() != ParameterKind.TYPE) {
throw new IllegalStateException(
format("Expected all parameters to be TypeSignatures but [%s] was found", parameter.toString()));
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));
}
result.add(parameter.getTypeSignature());
}
return result;
return unmodifiableList(result);
}

public boolean isCalculated()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
/*
* 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.
*/
package com.facebook.presto.common.type;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;

import static java.util.Collections.unmodifiableList;

public final class TypeSignatureUtils
{
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))

{
Map<TypeSignature, TypeSignature> typeSignatureMap = getTypeSignatureMap(actualTypeSignatureParameters, expectedTypeSignatureParameters);
return resolveTypeSignatures(actualTypeSignature, typeSignatureMap).getTypeSignature();
}

private static Map<TypeSignature, TypeSignature> getTypeSignatureMap(List<TypeSignature> parameters, List<TypeSignature> argumentTypes)
{
Map<TypeSignature, TypeSignature> typeSignatureMap = new HashMap<>();
if (argumentTypes.size() != parameters.size()) {
throw new IllegalStateException(
"Parameters size: " + parameters.size() + " and argumentTypes size: " + argumentTypes.size() + " do not match !");
}

for (int i = 0; i < argumentTypes.size(); i++) {
TypeSignature parameter = parameters.get(i);
TypeSignature argumentType = argumentTypes.get(i);

// Realistically, there are only two cases,
// 1. When parameter.getParameters() is empty :
// - Eg: parameter type = generic type(T) and argumentType = array(double), we can directly put
// map.put(parameter, argumentType) there is no need to loop over the argumentType.
// - Eg: param type = non-generic type(bigint) , in this case argumentType will also be of the
// same non-generic type hence map.put(parameter, argumentType) is valid here too.
// 2. When parameter.getParameters() is not empty:
// - Eg: parameter type = generic type (array(T)) and argumentType = array(double), we recursively run
// this function until we reach condition 1.
// Example calls for parameter type = generic type (array(T)) and argumentType = array(double)
// Iteration 1:
// Parameter:
// Type signature base = array
// parameters = T
// ArgumentType :
// Type signature base = array
// parameters = double
// Iteration 2:
// Parameter:
// Type signature base = T
// parameters = empty
// ArgumentType :
// Type signature base = double
// parameters = empty
// return typeSignatureMap = {"T": "double"}

// 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() || !(areParametersTypeSignatureOrNamedTypedSignature(parameter.getParameters()))) {
typeSignatureMap.put(parameter, argumentType);
}
else {
typeSignatureMap.putAll(getTypeSignatureMap(
parameter.getTypeOrNamedTypeParametersAsTypeSignatures(),
argumentType.getTypeOrNamedTypeParametersAsTypeSignatures()));
}
}
return typeSignatureMap;
}

// A utility function to resolve intermediate type signatures.
// Realistically, these are the different cases that we can face:
// 1. If there are no params/argTypes in the first call itself, return the resolvedType from the map directly.
// 2. If params != empty, we loop over the params:
// - Check whether the param is present in the typeSignatureMap, if its present,
// add the resolvedTypeParameterSignature mapping as a param directly and continue to the next param.
// - The idea behind this logic is that if param type is present in the typeSignatureMap, it means that for
// that particular param, we could just resolve the type from the map and no need to recursively call as its completely resolved.
// - Eg: param : T , map : {"T":"array(double)"}.
// - If the mapping isn't present, we recursively call the resolveTypeSignatures() again.
// - Example calls for param: array(T) , map : {"T" : "array(double)"}
// resolvedIntermediateType = null
// Iteration 1:
// Parameter:
// key to lookup in map: array(T), key found : false
// Iteration 2:
// Parameter:
// key to lookup in map: T, key found : true
// resolvedIntermediateType = array(double)
// return resolvedIntermediateType = array(array(double))
//
//
private static NamedTypeSignature resolveTypeSignatures(TypeSignature typeSignature, Map<TypeSignature, TypeSignature> typeSignatureMap)
{
if (typeSignatureMap.containsKey(typeSignature)) {
TypeSignature resolvedTypeSignature = typeSignatureMap.get(typeSignature);
return new NamedTypeSignature(Optional.empty(), resolvedTypeSignature);
}
List<NamedTypeSignature> namedTypeSignatures = new ArrayList<>();
List<TypeSignature> typeSignatures = new ArrayList<>();
for (TypeSignature typeParameterSignature : typeSignature.getTypeOrNamedTypeParametersAsTypeSignatures()) {
// if base is "row" typeSignature, all typeParameterSignatures need to be of type NamedTypeSignature.
boolean isRowTypeSignatureBase = typeSignature.getBase().equals("row");
if (typeSignatureMap.containsKey(typeParameterSignature)) {
TypeSignature resolvedTypeParameterSignature = typeSignatureMap.get(typeParameterSignature);
if (isRowTypeSignatureBase) {
namedTypeSignatures.add(new NamedTypeSignature(Optional.empty(), resolvedTypeParameterSignature));
}
else {
typeSignatures.add(resolvedTypeParameterSignature);
}
}
else {
NamedTypeSignature namedTypeSignature = resolveTypeSignatures(typeParameterSignature, typeSignatureMap);
if (isRowTypeSignatureBase) {
namedTypeSignatures.add(namedTypeSignature);
}
else {
typeSignatures.add(namedTypeSignature.getTypeSignature());
}
}
}

List<TypeSignatureParameter> parameters;
if (!typeSignatures.isEmpty()) {
parameters = typeSignatures.stream().map(TypeSignatureParameter::of).collect(Collectors.toList());
}
else {
parameters = namedTypeSignatures.stream().map(TypeSignatureParameter::of).collect(Collectors.toList());
}

return new NamedTypeSignature(Optional.empty(), new TypeSignature(typeSignature.getBase(), unmodifiableList(parameters)));
}

private static boolean areParametersTypeSignatureOrNamedTypedSignature(List<TypeSignatureParameter> parameters)
{
return !parameters.isEmpty() && parameters.stream()
.map(TypeSignatureParameter::getKind)
.allMatch(parameterKind ->
parameterKind.equals(ParameterKind.NAMED_TYPE) || parameterKind.equals(ParameterKind.TYPE));
}
}
Loading
Loading