Skip to content

Commit

Permalink
Add version to FunctionMetadata
Browse files Browse the repository at this point in the history
Add version to FunctionMetadata and display in RowExpressionFormatter.
  • Loading branch information
rongrong authored and Rongrong Zhong committed Apr 27, 2021
1 parent b496843 commit 99dcd76
Show file tree
Hide file tree
Showing 20 changed files with 154 additions and 64 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -278,7 +278,8 @@ protected FunctionMetadata sqlInvokedFunctionToMetadata(SqlInvokedFunction funct
function.getRoutineCharacteristics().getLanguage(),
getFunctionImplementationType(function),
function.isDeterministic(),
function.isCalledOnNullInput());
function.isCalledOnNullInput(),
function.getVersion());
}

protected FunctionImplementationType getFunctionImplementationType(SqlInvokedFunction function)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import static com.facebook.presto.spi.StandardErrorCode.AMBIGUOUS_FUNCTION_CALL;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR;
import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.ImmutableList.toImmutableList;
Expand Down Expand Up @@ -165,7 +166,7 @@ public void createFunction(SqlInvokedFunction function, boolean replace)
{
checkCatalog(function);
checkFunctionLanguageSupported(function);
checkArgument(!function.getVersion().isPresent(), "function '%s' is already versioned", function);
checkArgument(!function.hasVersion(), "function '%s' is already versioned", function);

QualifiedObjectName functionName = function.getFunctionId().getFunctionName();
checkFieldLength("Catalog name", functionName.getCatalogName(), MAX_CATALOG_NAME_LENGTH);
Expand Down Expand Up @@ -203,7 +204,7 @@ public void createFunction(SqlInvokedFunction function, boolean replace)
}
else if (latestVersion.get().isDeleted()) {
SqlInvokedFunction latest = latestVersion.get().getFunction();
checkState(latest.getVersion().isPresent(), "Function version missing: %s", latest.getFunctionId());
checkState(latest.hasVersion(), "Function version missing: %s", latest.getFunctionId());
transactionDao.setDeletionStatus(hash(latest.getFunctionId()), latest.getFunctionId(), getLongVersion(latest), false);
}
});
Expand Down Expand Up @@ -231,9 +232,9 @@ public void alterFunction(QualifiedObjectName functionName, Optional<List<TypeSi
latest.getDescription(),
routineCharacteristics.build(),
latest.getBody(),
Optional.empty());
notVersioned());
if (!altered.hasSameDefinitionAs(latest)) {
checkState(latest.getVersion().isPresent(), "Function version missing: %s", latest.getFunctionId());
checkState(latest.hasVersion(), "Function version missing: %s", latest.getFunctionId());
insertSqlInvokedFunction(transactionDao, altered, getLongVersion(latest) + 1);
}
});
Expand All @@ -255,7 +256,7 @@ public void dropFunction(QualifiedObjectName functionName, Optional<List<TypeSig
}
else {
SqlInvokedFunction latest = getOnlyElement(functions);
checkState(latest.getVersion().isPresent(), "Function version missing: %s", latest.getFunctionId());
checkState(latest.hasVersion(), "Function version missing: %s", latest.getFunctionId());
transactionDao.setDeletionStatus(hash(latest.getFunctionId()), latest.getFunctionId(), getLongVersion(latest), true);
}
});
Expand Down Expand Up @@ -303,7 +304,7 @@ private static long getLongVersion(SqlFunctionHandle functionHandle)

private static long getLongVersion(SqlInvokedFunction function)
{
return parseLong(function.getVersion().get());
return parseLong(function.getRequiredVersion());
}

private static void checkFieldLength(String fieldName, String field, int maxLength)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,11 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.List;
import java.util.Optional;

import static com.facebook.airlift.json.JsonCodec.jsonCodec;
import static com.facebook.airlift.json.JsonCodec.listJsonCodec;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.spi.function.FunctionVersion.withVersion;

public class SqlInvokedFunctionRowMapper
implements RowMapper<SqlInvokedFunction>
Expand Down Expand Up @@ -57,6 +57,6 @@ public SqlInvokedFunction map(ResultSet rs, StatementContext ctx)
description,
routineCharacteristics,
body,
Optional.of(version));
withVersion(version));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,10 @@
import com.facebook.presto.spi.function.SqlInvokedFunction;
import com.google.common.collect.ImmutableList;

import java.util.Optional;

import static com.facebook.presto.common.type.StandardTypes.DOUBLE;
import static com.facebook.presto.common.type.StandardTypes.INTEGER;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC;
import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.RETURNS_NULL_ON_NULL_INPUT;

Expand All @@ -47,7 +46,7 @@ private SqlInvokedFunctionTestUtils()
"power tower",
RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).build(),
"RETURN pow(x, x)",
Optional.empty());
notVersioned());

public static final SqlInvokedFunction FUNCTION_POWER_TOWER_DOUBLE_UPDATED = new SqlInvokedFunction(
POWER_TOWER,
Expand All @@ -56,7 +55,7 @@ private SqlInvokedFunctionTestUtils()
"power tower",
RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT).build(),
"RETURN pow(x, x)",
Optional.empty());
notVersioned());

public static final SqlInvokedFunction FUNCTION_POWER_TOWER_INT = new SqlInvokedFunction(
POWER_TOWER,
Expand All @@ -65,7 +64,7 @@ private SqlInvokedFunctionTestUtils()
"power tower",
RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT).build(),
"RETURN pow(x, x)",
Optional.empty());
notVersioned());

public static final SqlInvokedFunction FUNCTION_TANGENT = new SqlInvokedFunction(
TANGENT,
Expand All @@ -77,5 +76,5 @@ private SqlInvokedFunctionTestUtils()
.setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"RETURN sin(x) / cos(x)",
Optional.empty());
notVersioned());
}
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,8 @@
import static com.facebook.presto.functionNamespace.testing.SqlInvokedFunctionTestUtils.TEST_SCHEMA;
import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS;
import static com.facebook.presto.spi.function.FunctionKind.SCALAR;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.spi.function.FunctionVersion.withVersion;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC;
import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.CALLED_ON_NULL_INPUT;
import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.RETURNS_NULL_ON_NULL_INPUT;
Expand Down Expand Up @@ -288,7 +290,7 @@ public void testAlterFunction()
"power tower",
RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT).build(),
"RETURN pow(x, x)",
Optional.of("2")));
withVersion("2")));

// Drop function and alter function by name
dropFunction(POWER_TOWER, Optional.of(ImmutableList.of(parseTypeSignature(DOUBLE))), false);
Expand All @@ -303,7 +305,7 @@ public void testAlterFunction()
FUNCTION_TANGENT.getDescription(),
RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).build(),
FUNCTION_TANGENT.getBody(),
Optional.of("2"));
withVersion("2"));
assertGetFunctions(TANGENT, tangentV2);

// Alter function with no change
Expand Down Expand Up @@ -492,7 +494,7 @@ private static SqlInvokedFunction constructTestFunction(QualifiedObjectName func
"power tower",
RoutineCharacteristics.builder().setDeterminism(DETERMINISTIC).build(),
"pow(x, x)",
Optional.empty());
notVersioned());
}

private static SqlInvokedFunction createFunctionTangent(QualifiedObjectName functionName)
Expand All @@ -504,7 +506,7 @@ private static SqlInvokedFunction createFunctionTangent(QualifiedObjectName func
FUNCTION_TANGENT.getDescription(),
FUNCTION_TANGENT.getRoutineCharacteristics(),
FUNCTION_TANGENT.getBody(),
Optional.empty());
notVersioned());
}

private static SqlInvokedFunction createFunctionTangent(List<Parameter> parameters)
Expand All @@ -516,7 +518,7 @@ private static SqlInvokedFunction createFunctionTangent(List<Parameter> paramete
FUNCTION_TANGENT.getDescription(),
FUNCTION_TANGENT.getRoutineCharacteristics(),
FUNCTION_TANGENT.getBody(),
Optional.empty());
notVersioned());
}

private static SqlInvokedFunction createFunctionTangent(TypeSignature returnType)
Expand All @@ -528,7 +530,7 @@ private static SqlInvokedFunction createFunctionTangent(TypeSignature returnType
FUNCTION_TANGENT.getDescription(),
FUNCTION_TANGENT.getRoutineCharacteristics(),
FUNCTION_TANGENT.getBody(),
Optional.empty());
notVersioned());
}

private static TypeSignature createLargeRowType(int fieldCount)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import static com.facebook.presto.metadata.FunctionAndTypeManager.qualifyObjectName;
import static com.facebook.presto.metadata.SessionFunctionHandle.SESSION_NAMESPACE;
import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.sql.SqlFormatter.formatSql;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.util.concurrent.Futures.immediateFuture;
Expand Down Expand Up @@ -152,6 +153,6 @@ public Expression rewriteExpression(Expression expression, Void context, Express
description,
routineCharacteristics,
formatSql(body, Optional.empty()),
Optional.empty());
notVersioned());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -965,7 +965,8 @@ else if (function instanceof SqlInvokedFunction) {
sqlFunction.getRoutineCharacteristics().getLanguage(),
SQL,
function.isDeterministic(),
function.isCalledOnNullInput());
function.isCalledOnNullInput(),
sqlFunction.getVersion());
}
else {
return new FunctionMetadata(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,8 @@ public FunctionMetadata getFunctionMetadata()
sqlFunction.getRoutineCharacteristics().getLanguage(),
FunctionImplementationType.SQL,
sqlFunction.isDeterministic(),
sqlFunction.isCalledOnNullInput());
sqlFunction.isCalledOnNullInput(),
sqlFunction.getVersion());
}

@JsonProperty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import static com.facebook.presto.metadata.BuiltInTypeAndFunctionNamespaceManager.DEFAULT_NAMESPACE;
import static com.facebook.presto.operator.annotations.FunctionsParserHelper.findPublicStaticMethods;
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_IMPLEMENTATION_ERROR;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.NOT_DETERMINISTIC;
import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.CALLED_ON_NULL_INPUT;
Expand Down Expand Up @@ -162,7 +163,7 @@ else if (method.isAnnotationPresent(SqlParameters.class)) {
functionDescription,
routineCharacteristics,
body,
Optional.of("unique")))
notVersioned()))
.collect(toImmutableList());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.facebook.presto.common.type.Type;
import com.facebook.presto.metadata.FunctionAndTypeManager;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.function.FunctionMetadata;
import com.facebook.presto.spi.function.FunctionMetadataManager;
import com.facebook.presto.spi.function.StandardFunctionResolution;
import com.facebook.presto.spi.relation.CallExpression;
Expand Down Expand Up @@ -81,7 +82,8 @@ else if (standardFunctionResolution.isBetweenFunction(node.getFunctionHandle()))
List<String> formattedExpresions = formatRowExpressions(session, node.getArguments());
return String.format("%s BETWEEN (%s) AND (%s)", formattedExpresions.get(0), formattedExpresions.get(1), formattedExpresions.get(2));
}
return node.getDisplayName() + "(" + String.join(", ", formatRowExpressions(session, node.getArguments())) + ")";
FunctionMetadata metadata = functionMetadataManager.getFunctionMetadata(node.getFunctionHandle());
return node.getDisplayName() + (metadata.getVersion().hasVersion() ? ":" + metadata.getVersion() : "") + "(" + String.join(", ", formatRowExpressions(session, node.getArguments())) + ")";
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import static com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.ArgumentProperty.valueTypeArgumentProperty;
import static com.facebook.presto.operator.scalar.BuiltInScalarFunctionImplementation.NullConvention.RETURN_NULL_ON_NULL;
import static com.facebook.presto.spi.function.FunctionKind.SCALAR;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.spi.function.Signature.typeVariable;
import static com.facebook.presto.spi.function.SqlFunctionVisibility.PUBLIC;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures;
Expand Down Expand Up @@ -218,7 +219,7 @@ public void testSessionFunctions()
"",
RoutineCharacteristics.builder().build(),
"",
Optional.empty());
notVersioned());

SqlFunctionId varcharSignature = new SqlFunctionId(QualifiedObjectName.valueOf("presto.default.foo"), ImmutableList.of(parseTypeSignature("varchar")));
SqlInvokedFunction varcharFunction = new SqlInvokedFunction(
Expand All @@ -228,7 +229,7 @@ public void testSessionFunctions()
"",
RoutineCharacteristics.builder().build(),
"",
Optional.empty());
notVersioned());

Map<SqlFunctionId, SqlInvokedFunction> sessionFunctions = ImmutableMap.of(bigintSignature, bigintFunction, varcharSignature, varcharFunction);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@
import static com.facebook.presto.client.PrestoHeaders.PRESTO_USER;
import static com.facebook.presto.common.type.StandardTypes.INTEGER;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC;
import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.RETURNS_NULL_ON_NULL_INPUT;
import static java.lang.String.format;
Expand Down Expand Up @@ -233,7 +234,7 @@ public static SqlInvokedFunction createFunctionAdd()
.setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"RETURN x + y",
Optional.empty());
notVersioned());
}

public static SqlFunctionId createSqlFunctionIdAdd1ToIntArray()
Expand All @@ -253,6 +254,6 @@ public static SqlInvokedFunction createFunctionAdd1ToIntArray()
.setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"RETURN transform(x, x -> x + 1)",
Optional.empty());
notVersioned());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@
import static com.facebook.presto.operator.scalar.ApplyFunction.APPLY_FUNCTION;
import static com.facebook.presto.spi.ConnectorId.createInformationSchemaConnectorId;
import static com.facebook.presto.spi.ConnectorId.createSystemTablesConnectorId;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Language.SQL;
import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.RETURNS_NULL_ON_NULL_INPUT;
Expand Down Expand Up @@ -114,7 +115,7 @@ public class AbstractAnalyzerTest
.setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"RETURN x * x",
Optional.empty());
notVersioned());

protected TransactionManager transactionManager;
protected AccessControl accessControl;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,11 +45,11 @@
import org.testng.annotations.Test;

import java.util.Map;
import java.util.Optional;

import static com.facebook.presto.common.type.StandardTypes.INTEGER;
import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.spi.function.FunctionImplementationType.THRIFT;
import static com.facebook.presto.spi.function.FunctionVersion.notVersioned;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Determinism.DETERMINISTIC;
import static com.facebook.presto.spi.function.RoutineCharacteristics.Language.SQL;
import static com.facebook.presto.spi.function.RoutineCharacteristics.NullCallClause.RETURNS_NULL_ON_NULL_INPUT;
Expand All @@ -74,7 +74,7 @@ public class TestInlineSqlFunctions
.setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"RETURN x * x",
Optional.empty());
notVersioned());

private static final SqlInvokedFunction THRIFT_FUNCTION_FOO = new SqlInvokedFunction(
QualifiedObjectName.valueOf(new CatalogSchemaName("unittest", "memory"), "foo"),
Expand All @@ -86,7 +86,7 @@ public class TestInlineSqlFunctions
.setDeterminism(DETERMINISTIC).setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"",
Optional.empty());
notVersioned());

private static final SqlInvokedFunction SQL_FUNCTION_ADD_1_TO_INT_ARRAY = new SqlInvokedFunction(
QualifiedObjectName.valueOf(new CatalogSchemaName("unittest", "memory"), "add_1_int"),
Expand All @@ -98,7 +98,7 @@ public class TestInlineSqlFunctions
.setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"RETURN transform(x, x -> x + 1)",
Optional.empty());
notVersioned());

private static final SqlInvokedFunction SQL_FUNCTION_ADD_1_TO_BIGINT_ARRAY = new SqlInvokedFunction(
QualifiedObjectName.valueOf(new CatalogSchemaName("unittest", "memory"), "add_1_bigint"),
Expand All @@ -110,7 +110,7 @@ public class TestInlineSqlFunctions
.setNullCallClause(RETURNS_NULL_ON_NULL_INPUT)
.build(),
"RETURN transform(x, x -> x + 1)",
Optional.empty());
notVersioned());

private RuleTester tester;

Expand Down
Loading

0 comments on commit 99dcd76

Please sign in to comment.