Skip to content

Commit

Permalink
Propagate function lifecycle events to SystemSecurityMetadata
Browse files Browse the repository at this point in the history
  • Loading branch information
homar authored and kokosing committed Jan 30, 2025
1 parent cb6a940 commit 5e3e900
Show file tree
Hide file tree
Showing 6 changed files with 248 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,12 @@ public Optional<Identity> getFunctionRunAsIdentity(Session session, CatalogSchem
return Optional.empty();
}

@Override
public void functionCreated(Session session, CatalogSchemaFunctionName function) {}

@Override
public void functionDropped(Session session, CatalogSchemaFunctionName function) {}

@Override
public void schemaCreated(Session session, CatalogSchemaName schema) {}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2713,7 +2713,13 @@ public void createLanguageFunction(Session session, QualifiedObjectName name, La
CatalogHandle catalogHandle = catalogMetadata.getCatalogHandle();
ConnectorMetadata metadata = catalogMetadata.getMetadata(session);

metadata.createLanguageFunction(session.toConnectorSession(catalogHandle), name.asSchemaFunctionName(), function, replace);
SchemaFunctionName schemaFunctionName = name.asSchemaFunctionName();
metadata.createLanguageFunction(session.toConnectorSession(catalogHandle), schemaFunctionName, function, replace);
if (catalogMetadata.getSecurityManagement() == SYSTEM) {
systemSecurityMetadata.functionCreated(
session,
new CatalogSchemaFunctionName(catalogHandle.getCatalogName().toString(), schemaFunctionName));
}
}

@Override
Expand All @@ -2723,7 +2729,13 @@ public void dropLanguageFunction(Session session, QualifiedObjectName name, Stri
CatalogHandle catalogHandle = catalogMetadata.getCatalogHandle();
ConnectorMetadata metadata = catalogMetadata.getMetadata(session);

metadata.dropLanguageFunction(session.toConnectorSession(catalogHandle), name.asSchemaFunctionName(), signatureToken);
SchemaFunctionName schemaFunctionName = name.asSchemaFunctionName();
metadata.dropLanguageFunction(session.toConnectorSession(catalogHandle), schemaFunctionName, signatureToken);
if (catalogMetadata.getSecurityManagement() == SYSTEM) {
systemSecurityMetadata.functionDropped(
session,
new CatalogSchemaFunctionName(catalogHandle.getCatalogName().toString(), schemaFunctionName));
}
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,16 @@ default void validateEntityKindAndPrivileges(Session session, String entityKind,
*/
Optional<Identity> getFunctionRunAsIdentity(Session session, CatalogSchemaFunctionName functionName);

/**
* A function is created
*/
void functionCreated(Session session, CatalogSchemaFunctionName function);

/**
* A function is dropped
*/
void functionDropped(Session session, CatalogSchemaFunctionName function);

/**
* A schema was created
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,7 @@ public class TestingAccessControlManager
extends AccessControlManager
{
private static final BiPredicate<Identity, String> IDENTITY_TABLE_TRUE = (identity, table) -> true;
private static final BiPredicate<Identity, String> IDENTITY_FUNCTION_TRUE = (identity, function) -> true;

private final Set<TestingPrivilege> denyPrivileges = new HashSet<>();
private final Map<RowFilterKey, List<ViewExpression>> rowFilters = new HashMap<>();
Expand All @@ -142,6 +143,7 @@ public class TestingAccessControlManager
private Predicate<String> deniedSchemas = s -> true;
private Predicate<SchemaTableName> deniedTables = s -> true;
private BiPredicate<Identity, String> denyIdentityTable = IDENTITY_TABLE_TRUE;
private BiPredicate<Identity, String> denyIdentityFunction = IDENTITY_FUNCTION_TRUE;

@Inject
public TestingAccessControlManager(
Expand Down Expand Up @@ -216,6 +218,11 @@ public void denyIdentityTable(BiPredicate<Identity, String> denyIdentityTable)
this.denyIdentityTable = requireNonNull(denyIdentityTable, "denyIdentityTable is null");
}

public void denyIdentityFunction(BiPredicate<Identity, String> denyIdentityFunction)
{
this.denyIdentityFunction = requireNonNull(denyIdentityFunction, "denyIdentityFunction is null");
}

@Override
public Set<String> filterCatalogs(SecurityContext securityContext, Set<String> catalogs)
{
Expand Down Expand Up @@ -698,6 +705,9 @@ public void checkCanSelectFromColumns(SecurityContext context, QualifiedObjectNa
@Override
public boolean canExecuteFunction(SecurityContext context, QualifiedObjectName functionName)
{
if (!denyIdentityFunction.test(context.getIdentity(), functionName.asSchemaFunctionName().toString())) {
return false;
}
if (shouldDenyPrivilege(context.getIdentity().getUser(), functionName.toString(), EXECUTE_FUNCTION)) {
return false;
}
Expand All @@ -710,6 +720,9 @@ public boolean canExecuteFunction(SecurityContext context, QualifiedObjectName f
@Override
public boolean canCreateViewWithExecuteFunction(SecurityContext context, QualifiedObjectName functionName)
{
if (!denyIdentityFunction.test(context.getIdentity(), functionName.asSchemaFunctionName().toString())) {
return false;
}
if (shouldDenyPrivilege(context.getIdentity().getUser(), functionName.toString(), GRANT_EXECUTE_FUNCTION)) {
return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,12 +40,14 @@
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.connector.SchemaTablePrefix;
import io.trino.spi.function.BoundSignature;
import io.trino.spi.function.CatalogSchemaFunctionName;
import io.trino.spi.function.FunctionDependencies;
import io.trino.spi.function.FunctionId;
import io.trino.spi.function.FunctionMetadata;
import io.trino.spi.function.FunctionProvider;
import io.trino.spi.function.InvocationConvention;
import io.trino.spi.function.ScalarFunctionImplementation;
import io.trino.spi.function.SchemaFunctionName;
import io.trino.spi.function.Signature;
import io.trino.spi.security.AccessDeniedException;
import io.trino.spi.security.Identity;
Expand Down Expand Up @@ -552,6 +554,183 @@ public void testJoinBaseTableWithView()
assertAccessAllowed(viewOwnerSession, "DROP VIEW " + viewName);
}

@Test
public void testAllowCallFunction()
{
reset();

String functionOwner = "function_owner";
CatalogSchemaFunctionName outerFunction = new CatalogSchemaFunctionName("memory", new SchemaFunctionName("default", "function_allow_outer"));
CatalogSchemaFunctionName innerFunction = new CatalogSchemaFunctionName("memory", new SchemaFunctionName("default", "function_allow_inner"));

Session functionOwnerSession = TestingSession.testSessionBuilder()
.setIdentity(Identity.ofUser(functionOwner))
.setCatalog(getSession().getCatalog())
.setSchema(getSession().getSchema())
.build();

// simply create a function
assertAccessAllowed(
functionOwnerSession,
"CREATE FUNCTION memory.default.function_allow_inner (x integer) RETURNS bigint RETURN x + 42");
assertThat(systemSecurityMetadata.getFunctionOwner(innerFunction)).isEqualTo(functionOwner);

// assert that function can be called for both definer session and default session
assertAccessAllowed(
functionOwnerSession,
"SELECT memory.default.function_allow_inner(2)");
assertAccessAllowed("SELECT memory.default.function_allow_inner(2)");

// simply create another function, internally calls the first function
assertAccessAllowed(
functionOwnerSession,
"CREATE FUNCTION memory.default.function_allow_outer (x integer) RETURNS bigint RETURN x + memory.default.function_allow_inner(58)");
assertThat(systemSecurityMetadata.getFunctionOwner(outerFunction)).isEqualTo(functionOwner);

// assert that THE outer function can be called for both definer session and default session
assertAccessAllowed(
functionOwnerSession,
"SELECT memory.default.function_allow_outer(2)");
assertAccessAllowed("SELECT memory.default.function_allow_outer(2)");

// assert that lack of privileges to execute inner function doesn't block calling it through outer one
assertAccessAllowed(
"SELECT memory.default.function_allow_outer(2)",
privilege(getSession().getUser(), "memory.default.function_allow_inner", EXECUTE_FUNCTION));

assertAccessDenied(
"SELECT memory.default.function_allow_inner(2)",
"Cannot execute function memory.default.function_allow_inner",
privilege(getSession().getUser(), "memory.default.function_allow_inner", EXECUTE_FUNCTION));
}

@Test
public void testAllowCallFunctionWithRoleGrant()
{
reset();

String functionOwner = "function_owner";
CatalogSchemaFunctionName outerFunction = new CatalogSchemaFunctionName("memory", new SchemaFunctionName("default", "function_deny_outer"));
CatalogSchemaFunctionName innerFunction = new CatalogSchemaFunctionName("memory", new SchemaFunctionName("default", "function_deny_inner"));
TrinoPrincipal functionOwnerPrincipal = new TrinoPrincipal(USER, functionOwner);
systemSecurityMetadata.grantRoles(getSession(), ImmutableSet.of("function_owner_role"), ImmutableSet.of(functionOwnerPrincipal), false, Optional.empty());

Session functionOwnerSession = TestingSession.testSessionBuilder()
.setIdentity(Identity.forUser(functionOwner)
.withEnabledRoles(Set.of("function_owner_role"))
.build())
.setCatalog(getSession().getCatalog())
.setSchema(getSession().getSchema())
.build();

// simply create a function
assertAccessAllowed(
functionOwnerSession,
"CREATE FUNCTION memory.default.function_deny_inner (x integer) RETURNS bigint RETURN x + 42");
assertThat(systemSecurityMetadata.getFunctionOwner(innerFunction)).isEqualTo(functionOwner);

// simply create another function that internally calls the first function
assertAccessAllowed(
functionOwnerSession,
"CREATE FUNCTION memory.default.function_deny_outer (x integer) RETURNS bigint RETURN x + memory.default.function_deny_inner(58)");
assertThat(systemSecurityMetadata.getFunctionOwner(outerFunction)).isEqualTo(functionOwner);

// assert that outer function can be called for both definer session and default session
assertAccessAllowed(
functionOwnerSession,
"SELECT memory.default.function_deny_outer(2)");
assertAccessAllowed("SELECT memory.default.function_deny_outer(2)");

// block role function_owner_role_without_access from calling inner function
getQueryRunner().getAccessControl()
.denyIdentityFunction((identity, function) -> !(identity.getEnabledRoles().contains("function_owner_role_without_access") && "default.function_deny_inner".equals(function)));
// assign function_owner_role_without_access to function definer
systemSecurityMetadata.grantRoles(getSession(), ImmutableSet.of("function_owner_role_without_access"), ImmutableSet.of(functionOwnerPrincipal), false, Optional.empty());

// assert that because definer has function_owner_role_without_access role assigned it is impossible to call outer function
assertAccessDenied(
functionOwnerSession,
"SELECT memory.default.function_deny_outer(2)",
"Cannot execute function memory.default.function_deny_inner");
assertAccessDenied(
"SELECT memory.default.function_deny_outer(2)",
"Cannot execute function memory.default.function_deny_inner");

systemSecurityMetadata.revokeRoles(getSession(), ImmutableSet.of("function_owner_role_without_access"), ImmutableSet.of(functionOwnerPrincipal), false, Optional.empty());

// assert that after revoking function_owner_role_without_access from definer function can be called once more
assertAccessAllowed(
functionOwnerSession,
"SELECT memory.default.function_deny_outer(2)");
assertAccessAllowed("SELECT memory.default.function_deny_outer(2)");
}

@Test
public void testFunctionOwnerWhenDroppingFunction()
{
reset();

String functionOwner1 = "function_owner1";
String functionOwner2 = "function_owner2";
CatalogSchemaFunctionName functionName = new CatalogSchemaFunctionName("memory", new SchemaFunctionName("default", "my_function"));

Session functionOwnerSession1 = TestingSession.testSessionBuilder()
.setIdentity(Identity.ofUser(functionOwner1))
.setCatalog(getSession().getCatalog())
.setSchema(getSession().getSchema())
.build();
Session functionOwnerSession2 = TestingSession.testSessionBuilder()
.setIdentity(Identity.ofUser(functionOwner2))
.setCatalog(getSession().getCatalog())
.setSchema(getSession().getSchema())
.build();

assertAccessAllowed(
functionOwnerSession1,
"CREATE FUNCTION memory.default.my_function (x integer) RETURNS bigint RETURN x + 42");
assertThat(systemSecurityMetadata.getFunctionOwner(functionName)).isEqualTo(functionOwner1);

assertAccessAllowed(
functionOwnerSession1,
"DROP FUNCTION memory.default.my_function(integer)");

assertAccessAllowed(
functionOwnerSession2,
"CREATE FUNCTION memory.default.my_function (x integer) RETURNS bigint RETURN x + 42");
assertThat(systemSecurityMetadata.getFunctionOwner(functionName)).isEqualTo(functionOwner2);
}

@Test
public void testFunctionOwnerWhenReplacingFunction()
{
reset();

String functionOwner1 = "function_owner1";
String functionOwner2 = "function_owner2";
CatalogSchemaFunctionName functionName = new CatalogSchemaFunctionName("memory", new SchemaFunctionName("default", "my_replace_function"));

Session functionOwnerSession1 = TestingSession.testSessionBuilder()
.setIdentity(Identity.ofUser(functionOwner1))
.setCatalog(getSession().getCatalog())
.setSchema(getSession().getSchema())
.build();
Session functionOwnerSession2 = TestingSession.testSessionBuilder()
.setIdentity(Identity.ofUser(functionOwner2))
.setCatalog(getSession().getCatalog())
.setSchema(getSession().getSchema())
.build();

assertAccessAllowed(
functionOwnerSession1,
"CREATE FUNCTION memory.default.my_replace_function (x integer) RETURNS bigint RETURN x + 42");
assertThat(systemSecurityMetadata.getFunctionOwner(functionName)).isEqualTo(functionOwner1);

assertAccessAllowed(
functionOwnerSession2,
"CREATE OR REPLACE FUNCTION memory.default.my_replace_function (x integer) RETURNS bigint RETURN x + 42");
assertThat(systemSecurityMetadata.getFunctionOwner(functionName)).isEqualTo(functionOwner2);
}

@Test
public void testViewFunctionAccessControl()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,12 +48,19 @@ class TestingSystemSecurityMetadata
private final Set<String> roles = synchronizedSet(new HashSet<>());
private final Set<RoleGrant> roleGrants = synchronizedSet(new HashSet<>());
private final Map<CatalogSchemaTableName, Identity> viewOwners = synchronizedMap(new HashMap<>());
private final Map<CatalogSchemaFunctionName, Identity> functionOwners = synchronizedMap(new HashMap<>());

public void reset()
{
roles.clear();
roleGrants.clear();
viewOwners.clear();
functionOwners.clear();
}

public String getFunctionOwner(CatalogSchemaFunctionName functionName)
{
return functionOwners.get(functionName).getUser();
}

@Override
Expand Down Expand Up @@ -246,7 +253,25 @@ public void setViewOwner(Session session, CatalogSchemaTableName view, TrinoPrin
@Override
public Optional<Identity> getFunctionRunAsIdentity(Session session, CatalogSchemaFunctionName functionName)
{
return Optional.empty();
return Optional.ofNullable(functionOwners.get(functionName))
.map(identity -> Identity.from(identity)
.withEnabledRoles(getRoleGrantsRecursively(new TrinoPrincipal(USER, identity.getUser()))
.stream()
.map(RoleGrant::getRoleName)
.collect(toImmutableSet()))
.build());
}

@Override
public void functionCreated(Session session, CatalogSchemaFunctionName function)
{
functionOwners.put(function, session.getIdentity());
}

@Override
public void functionDropped(Session session, CatalogSchemaFunctionName function)
{
functionOwners.remove(function);
}

@Override
Expand Down

0 comments on commit 5e3e900

Please sign in to comment.