Skip to content

Commit

Permalink
Addressed comments
Browse files Browse the repository at this point in the history
  • Loading branch information
pdabre12 authored and Pratik Joseph Dabre committed Jan 13, 2025
1 parent 755a0e3 commit 96bd59e
Show file tree
Hide file tree
Showing 83 changed files with 341 additions and 324 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,8 @@ public class JsonBasedUdfFunctionMetadata
*/
private final Optional<AggregationFunctionMetadata> aggregateMetadata;
/**
* Marked to indicate the arity of the function.
* Marked to indicate whether it is a variable arity function.
* A variable arity function can have a variable number of arguments of the specified type.
*/
private final boolean variableArity;
/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,7 @@ public static TestingPrestoServer createTestingPrestoServer()
"hive-functions",
"hive",
getNamespaceManagerCreationProperties(),
server.getPluginNodeManager(),
false);
server.getPluginNodeManager());
server.refreshNodes();
return server;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -150,8 +150,7 @@ private static TestingPrestoServer createServer()
"hive-functions",
"hive",
Collections.emptyMap(),
server.getPluginNodeManager(),
false);
server.getPluginNodeManager());
server.refreshNodes();
return server;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,16 +81,16 @@
public class IcebergMetadataOptimizer
implements ConnectorPlanOptimizer
{
public static final CatalogSchemaName DEFAULT_NAMESPACE = new CatalogSchemaName("presto", "default");
public static final CatalogSchemaName JAVA_BUILTIN_NAMESPACE = new CatalogSchemaName("presto", "default");
private static final Set<QualifiedObjectName> ALLOWED_FUNCTIONS = ImmutableSet.of(
QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "max"),
QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "min"),
QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "approx_distinct"));
QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, "max"),
QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, "min"),
QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, "approx_distinct"));

// Min/Max could be folded into LEAST/GREATEST
private static final Map<QualifiedObjectName, QualifiedObjectName> AGGREGATION_SCALAR_MAPPING = ImmutableMap.of(
QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "max"), QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "greatest"),
QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "min"), QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, "least"));
QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, "max"), QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, "greatest"),
QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, "min"), QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, "least"));

private final FunctionMetadataManager functionMetadataManager;
private final TypeManager typeManager;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -236,7 +236,6 @@
import com.facebook.presto.spi.function.SqlInvokedFunction;
import com.facebook.presto.spi.function.SqlInvokedScalarFunctionImplementation;
import com.facebook.presto.sql.analyzer.FunctionsConfig;
import com.facebook.presto.sql.analyzer.TypeSignatureProvider;
import com.facebook.presto.type.BigintOperators;
import com.facebook.presto.type.BooleanOperators;
import com.facebook.presto.type.CharOperators;
Expand Down Expand Up @@ -345,7 +344,6 @@
import static com.facebook.presto.geospatial.SphericalGeographyType.SPHERICAL_GEOGRAPHY;
import static com.facebook.presto.geospatial.type.BingTileType.BING_TILE;
import static com.facebook.presto.geospatial.type.GeometryType.GEOMETRY;
import static com.facebook.presto.metadata.SignatureBinder.applyBoundVariables;
import static com.facebook.presto.operator.aggregation.AlternativeArbitraryAggregationFunction.ALTERNATIVE_ANY_VALUE_AGGREGATION;
import static com.facebook.presto.operator.aggregation.AlternativeArbitraryAggregationFunction.ALTERNATIVE_ARBITRARY_AGGREGATION;
import static com.facebook.presto.operator.aggregation.AlternativeMaxAggregationFunction.ALTERNATIVE_MAX;
Expand Down Expand Up @@ -461,7 +459,6 @@
import static com.facebook.presto.spi.function.FunctionKind.SCALAR;
import static com.facebook.presto.spi.function.FunctionKind.WINDOW;
import static com.facebook.presto.spi.function.SqlFunctionVisibility.HIDDEN;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures;
import static com.facebook.presto.sql.planner.LiteralEncoder.MAGIC_LITERAL_FUNCTION_PREFIX;
import static com.facebook.presto.type.ArrayParametricType.ARRAY;
import static com.facebook.presto.type.CodePointsType.CODE_POINTS;
Expand Down Expand Up @@ -521,7 +518,6 @@
import static com.facebook.presto.type.Re2JRegexpType.RE2J_REGEXP;
import static com.facebook.presto.type.RowParametricType.ROW;
import static com.facebook.presto.type.SfmSketchType.SFM_SKETCH;
import static com.facebook.presto.type.TypeUtils.resolveTypes;
import static com.facebook.presto.type.khyperloglog.KHyperLogLogType.K_HYPER_LOG_LOG;
import static com.facebook.presto.type.setdigest.SetDigestType.SET_DIGEST;
import static com.google.common.base.Preconditions.checkArgument;
Expand All @@ -537,7 +533,7 @@
public class BuiltInTypeAndFunctionNamespaceManager
implements FunctionNamespaceManager<SqlFunction>
{
public static final CatalogSchemaName DEFAULT_NAMESPACE = new CatalogSchemaName("presto", "default");
public static final CatalogSchemaName JAVA_BUILTIN_NAMESPACE = new CatalogSchemaName("presto", "default");
public static final String ID = "builtin";

private final FunctionAndTypeManager functionAndTypeManager;
Expand Down Expand Up @@ -1343,52 +1339,11 @@ private SpecializedFunctionKey getSpecializedFunctionKey(Signature signature)

private SpecializedFunctionKey doGetSpecializedFunctionKey(Signature signature)
{
Collection<SqlFunction> candidates = getFunctions(null, signature.getName());
return doGetSpecializedFunctionKey(signature, candidates);
return functionAndTypeManager.getSpecializedFunctionKey(signature);
}

public SpecializedFunctionKey doGetSpecializedFunctionKey(Signature signature, Collection<SqlFunction> candidates)
public SpecializedFunctionKey doGetSpecializedFunctionKeyForMagicLiteralFunctions(Signature signature, FunctionAndTypeManager functionAndTypeManager)
{
// search for exact match
Type returnType = functionAndTypeManager.getType(signature.getReturnType());
List<TypeSignatureProvider> argumentTypeSignatureProviders = fromTypeSignatures(signature.getArgumentTypes());
for (SqlFunction candidate : candidates) {
Optional<BoundVariables> boundVariables = new SignatureBinder(functionAndTypeManager, candidate.getSignature(), false)
.bindVariables(argumentTypeSignatureProviders, returnType);
if (boundVariables.isPresent()) {
return new SpecializedFunctionKey(candidate, boundVariables.get(), argumentTypeSignatureProviders.size());
}
}

// TODO: hack because there could be "type only" coercions (which aren't necessarily included as implicit casts),
// so do a second pass allowing "type only" coercions
List<Type> argumentTypes = resolveTypes(signature.getArgumentTypes(), functionAndTypeManager);
for (SqlFunction candidate : candidates) {
SignatureBinder binder = new SignatureBinder(functionAndTypeManager, candidate.getSignature(), true);
Optional<BoundVariables> boundVariables = binder.bindVariables(argumentTypeSignatureProviders, returnType);
if (!boundVariables.isPresent()) {
continue;
}
Signature boundSignature = applyBoundVariables(candidate.getSignature(), boundVariables.get(), argumentTypes.size());

if (!functionAndTypeManager.isTypeOnlyCoercion(functionAndTypeManager.getType(boundSignature.getReturnType()), returnType)) {
continue;
}
boolean nonTypeOnlyCoercion = false;
for (int i = 0; i < argumentTypes.size(); i++) {
Type expectedType = functionAndTypeManager.getType(boundSignature.getArgumentTypes().get(i));
if (!functionAndTypeManager.isTypeOnlyCoercion(argumentTypes.get(i), expectedType)) {
nonTypeOnlyCoercion = true;
break;
}
}
if (nonTypeOnlyCoercion) {
continue;
}

return new SpecializedFunctionKey(candidate, boundVariables.get(), argumentTypes.size());
}

// TODO: this is a hack and should be removed
if (signature.getNameSuffix().startsWith(MAGIC_LITERAL_FUNCTION_PREFIX)) {
List<TypeSignature> parameterTypes = signature.getArgumentTypes();
Expand All @@ -1411,7 +1366,6 @@ public SpecializedFunctionKey doGetSpecializedFunctionKey(Signature signature, C
.build(),
1);
}

throw new PrestoException(FUNCTION_IMPLEMENTATION_MISSING, format("%s not found", signature));
}

Expand Down Expand Up @@ -1551,7 +1505,7 @@ private static class MagicLiteralFunction

MagicLiteralFunction(BlockEncodingSerde blockEncodingSerde)
{
super(new Signature(QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, MAGIC_LITERAL_FUNCTION_PREFIX), SCALAR, TypeSignature.parseTypeSignature("R"), TypeSignature.parseTypeSignature("T")));
super(new Signature(QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, MAGIC_LITERAL_FUNCTION_PREFIX), SCALAR, TypeSignature.parseTypeSignature("R"), TypeSignature.parseTypeSignature("T")));
this.blockEncodingSerde = requireNonNull(blockEncodingSerde, "blockEncodingSerde is null");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import com.facebook.presto.common.QualifiedObjectName;
import com.facebook.presto.common.function.OperatorType;

import static com.facebook.presto.metadata.BuiltInTypeAndFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.metadata.BuiltInTypeAndFunctionNamespaceManager.JAVA_BUILTIN_NAMESPACE;
import static com.facebook.presto.operator.scalar.JsonStringToArrayCast.JSON_STRING_TO_ARRAY_NAME;
import static com.facebook.presto.operator.scalar.JsonStringToMapCast.JSON_STRING_TO_MAP_NAME;
import static com.facebook.presto.operator.scalar.JsonStringToRowCast.JSON_STRING_TO_ROW_NAME;
Expand All @@ -27,10 +27,10 @@ public enum CastType
{
CAST(OperatorType.CAST.getFunctionName(), true),
SATURATED_FLOOR_CAST(OperatorType.SATURATED_FLOOR_CAST.getFunctionName(), true),
TRY_CAST(QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, TRY_CAST_NAME), false),
JSON_TO_ARRAY_CAST(QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, JSON_STRING_TO_ARRAY_NAME), false),
JSON_TO_MAP_CAST(QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, JSON_STRING_TO_MAP_NAME), false),
JSON_TO_ROW_CAST(QualifiedObjectName.valueOf(DEFAULT_NAMESPACE, JSON_STRING_TO_ROW_NAME), false);
TRY_CAST(QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, TRY_CAST_NAME), false),
JSON_TO_ARRAY_CAST(QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, JSON_STRING_TO_ARRAY_NAME), false),
JSON_TO_MAP_CAST(QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, JSON_STRING_TO_MAP_NAME), false),
JSON_TO_ROW_CAST(QualifiedObjectName.valueOf(JAVA_BUILTIN_NAMESPACE, JSON_STRING_TO_ROW_NAME), false);

private final QualifiedObjectName castName;
private final boolean isOperatorType;
Expand Down
Loading

0 comments on commit 96bd59e

Please sign in to comment.