Skip to content

Commit

Permalink
cql3: allow UDTs in permissions on UDFs
Browse files Browse the repository at this point in the history
Currently, when preparing an authorization statement on a specific
function, we're trying to "prepare" all cql types that appear in
the function signature while parsing the statement. We cannot
do that for UDTs, because we don't know the UDTs that are present
in the databse at parsing time. As a result, such authorization
statements fail.
To work around this problem, we postpone the "preparation" of cql
types until the actual statement validation and execution time.
Until then, we store all type strings in the resource object.
The "preparation" happens in the `maybe_correct_resource` method,
which is called before every `execute` during a `check_access` call.
At that point, we have access to the `query_processor`, and as a
result, to `user_types_metadata` which allows us to prepare the
argument types even for UDTs.
  • Loading branch information
wmitros committed Mar 10, 2023
1 parent 4f0b353 commit 6b8c182
Show file tree
Hide file tree
Showing 8 changed files with 115 additions and 26 deletions.
60 changes: 43 additions & 17 deletions auth/resource.cc
Original file line number Diff line number Diff line change
Expand Up @@ -134,25 +134,17 @@ resource::resource(functions_resource_t, std::string_view keyspace, std::string_
_parts.emplace_back(function_signature);
}

resource::resource(functions_resource_t, std::string_view keyspace, std::string_view function_name, std::vector<sstring> function_signature) : resource(resource_kind::functions) {
resource::resource(functions_resource_t, std::string_view keyspace, std::string_view function_name, std::vector<::shared_ptr<cql3::cql3_type::raw>> function_args) : resource(resource_kind::functions) {
_parts.emplace_back(keyspace);
sstring encoded_signature = format("{}[{}]",
function_name,
::join("^", function_signature));
_parts.emplace_back(encoded_signature);
}

resource make_functions_resource(std::string_view keyspace, std::string_view function_name, std::vector<::shared_ptr<cql3::cql3_type::raw>> function_signature) {
if (keyspace.empty()) {
throw exceptions::invalid_request_exception("In this context function name must be explictly qualified by a keyspace");
_parts.emplace_back(function_name);
if (function_args.empty()) {
_parts.emplace_back("");
return;
}
std::vector<sstring> args_types;
for (auto& raw_type : function_signature) {
// FIXME(sarna): provide information on user-defined types - this is tricky, because this information
// is kept in a database instance, and is thus dynamic
args_types.emplace_back(raw_type->prepare_internal(sstring(keyspace), data_dictionary::user_types_metadata{}).get_type()->name());
for (auto& arg_type : function_args) {
// We can't validate the UDTs here, so we just use the raw cql type names.
_parts.emplace_back(arg_type->to_string());
}
return resource(functions_resource_t{}, keyspace, function_name, std::move(args_types));
}

sstring resource::name() const {
Expand Down Expand Up @@ -228,6 +220,9 @@ std::pair<sstring, std::vector<data_type>> decode_signature(std::string_view enc
std::string_view function_name = encoded_signature.substr(0, name_delim);
encoded_signature.remove_prefix(name_delim + 1);
encoded_signature.remove_suffix(1);
if (encoded_signature.empty()) {
return {sstring(function_name), {}};
}
std::vector<std::string_view> raw_types;
boost::split(raw_types, encoded_signature, boost::is_any_of("^"));
std::vector<data_type> decoded_types = boost::copy_range<std::vector<data_type>>(
Expand All @@ -252,9 +247,17 @@ static sstring decoded_signature_string(std::string_view encoded_signature) {
std::ostream &operator<<(std::ostream &os, const functions_resource_view &v) {
const auto keyspace = v.keyspace();
const auto function_signature = v.function_signature();
const auto name = v.function_name();
const auto args = v.function_args();

if (!keyspace) {
os << "<all functions>";
} else if (name) {
os << "<function " << *keyspace << '.' << cql3::util::maybe_quote(sstring(*name)) << '(';
for (auto arg : *args) {
os << arg << ',';
}
os << ")>";
} else if (!function_signature) {
os << "<all functions in " << *keyspace << '>';
} else {
Expand All @@ -278,13 +281,36 @@ std::optional<std::string_view> functions_resource_view::keyspace() const {
}

std::optional<std::string_view> functions_resource_view::function_signature() const {
if (_resource._parts.size() <= 2) {
if (_resource._parts.size() <= 2 || _resource._parts.size() > 3) {
return {};
}

return _resource._parts[2];
}

std::optional<std::string_view> functions_resource_view::function_name() const {
if (_resource._parts.size() <= 3) {
return {};
}

return _resource._parts[2];
}

std::optional<std::vector<std::string_view>> functions_resource_view::function_args() const {
if (_resource._parts.size() <= 3) {
return {};
}

std::vector<std::string_view> parts;
if (_resource._parts[3] == "") {
return {};
}
for (size_t i = 3; i < _resource._parts.size(); i++) {
parts.push_back(_resource._parts[i]);
}
return parts;
}

data_resource_view::data_resource_view(const resource& r) : _resource(r) {
if (r._kind != resource_kind::data) {
throw resource_kind_mismatch(resource_kind::data, r._kind);
Expand Down
8 changes: 6 additions & 2 deletions auth/resource.hh
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ public:
resource(functions_resource_t, std::string_view keyspace);
resource(functions_resource_t, std::string_view keyspace, std::string_view function_signature);
resource(functions_resource_t, std::string_view keyspace, std::string_view function_name,
std::vector<sstring> function_signature);
std::vector<::shared_ptr<cql3::cql3_type::raw>> function_args);

resource_kind kind() const noexcept {
return _kind;
Expand Down Expand Up @@ -208,6 +208,8 @@ public:

std::optional<std::string_view> keyspace() const;
std::optional<std::string_view> function_signature() const;
std::optional<std::string_view> function_name() const;
std::optional<std::vector<std::string_view>> function_args() const;
};

std::ostream& operator<<(std::ostream&, const functions_resource_view&);
Expand Down Expand Up @@ -254,7 +256,9 @@ inline resource make_functions_resource(std::string_view keyspace, std::string_v
return resource(functions_resource_t{}, keyspace, function_signature);
}

resource make_functions_resource(std::string_view keyspace, std::string_view function_name, std::vector<::shared_ptr<cql3::cql3_type::raw>> function_signature);
inline resource make_functions_resource(std::string_view keyspace, std::string_view function_name, std::vector<::shared_ptr<cql3::cql3_type::raw>> function_signature) {
return resource(functions_resource_t{}, keyspace, function_name, function_signature);
}

sstring encode_signature(std::string_view name, std::vector<data_type> args);

Expand Down
4 changes: 4 additions & 0 deletions cql3/cql3_type.hh
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,9 @@ namespace data_dictionary {
class database;
class user_types_metadata;
}
namespace auth {
class resource;
}

namespace cql3 {

Expand Down Expand Up @@ -63,6 +66,7 @@ public:
static shared_ptr<raw> tuple(std::vector<shared_ptr<raw>> ts);
static shared_ptr<raw> frozen(shared_ptr<raw> t);
friend std::ostream& operator<<(std::ostream& os, const raw& r);
friend class auth::resource;
};

private:
Expand Down
41 changes: 40 additions & 1 deletion cql3/statements/authorization_statement.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,13 @@
#include "transport/messages/result_message.hh"
#include "service/client_state.hh"
#include "auth/resource.hh"
#include "cql3/query_processor.hh"
#include "exceptions/exceptions.hh"
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/classification.hpp>
#include "cql3/util.hh"
#include "db/cql_type_parser.hh"

uint32_t cql3::statements::authorization_statement::get_bound_terms() const {
return 0;
Expand All @@ -30,7 +37,7 @@ future<> cql3::statements::authorization_statement::check_access(query_processor
return make_ready_future<>();
}

void cql3::statements::authorization_statement::maybe_correct_resource(auth::resource& resource, const service::client_state& state){
void cql3::statements::authorization_statement::maybe_correct_resource(auth::resource& resource, const service::client_state& state, query_processor& qp) {
if (resource.kind() == auth::resource_kind::data) {
const auto data_view = auth::data_resource_view(resource);
const auto keyspace = data_view.keyspace();
Expand All @@ -39,6 +46,38 @@ void cql3::statements::authorization_statement::maybe_correct_resource(auth::res
if (table && keyspace->empty()) {
resource = auth::make_data_resource(state.get_keyspace(), *table);
}
} else if (resource.kind() == auth::resource_kind::functions) {
// Maybe correct the resource for a specific function.
const auto functions_view = auth::functions_resource_view(resource);
const auto keyspace = functions_view.keyspace();
if (!keyspace) {
// This is an "ALL FUNCTIONS" resource.
return;
}
if (!qp.db().has_keyspace(*keyspace)) {
throw exceptions::invalid_request_exception(format("{} doesn't exist.", resource));
}
if (functions_view.function_signature()) {
// The resource is already corrected.
return;
}
if (!functions_view.function_name()) {
// This is an "ALL FUNCTIONS IN KEYSPACE" resource.
return;
}
const auto& utm = qp.db().find_keyspace(*keyspace).user_types();
auto function_name = *functions_view.function_name();
auto function_args = functions_view.function_args();
std::vector<data_type> parsed_types;
if (function_args) {
parsed_types = boost::copy_range<std::vector<data_type>>(
*function_args | boost::adaptors::transformed([&] (std::string_view raw_type) {
auto parsed = db::cql_type_parser::parse(sstring(keyspace->data(), keyspace->size()), sstring(raw_type.data(), raw_type.size()), utm);
return parsed->is_user_type() ? parsed->freeze() : parsed;
})
);
}
resource = auth::make_functions_resource(*keyspace, auth::encode_signature(function_name, parsed_types));
}
}

2 changes: 1 addition & 1 deletion cql3/statements/authorization_statement.hh
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ public:
void validate(query_processor&, const service::client_state& state) const override;

protected:
static void maybe_correct_resource(auth::resource&, const service::client_state&);
static void maybe_correct_resource(auth::resource&, const service::client_state&, query_processor&);
};

}
Expand Down
2 changes: 1 addition & 1 deletion cql3/statements/list_permissions_statement.cc
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ void cql3::statements::list_permissions_statement::validate(

future<> cql3::statements::list_permissions_statement::check_access(query_processor& qp, const service::client_state& state) const {
if (_resource) {
maybe_correct_resource(*_resource, state);
maybe_correct_resource(*_resource, state, qp);
return state.ensure_exists(*_resource);
}

Expand Down
2 changes: 1 addition & 1 deletion cql3/statements/permission_altering_statement.cc
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ void cql3::statements::permission_altering_statement::validate(query_processor&,

future<> cql3::statements::permission_altering_statement::check_access(query_processor& qp, const service::client_state& state) const {
state.ensure_not_anonymous();
maybe_correct_resource(_resource, state);
maybe_correct_resource(_resource, state, qp);

return state.ensure_exists(_resource).then([this, &state] {
// check that the user has AUTHORIZE permission on the resource or its parents, otherwise reject
Expand Down
22 changes: 19 additions & 3 deletions test/cql-pytest/test_permissions.py
Original file line number Diff line number Diff line change
Expand Up @@ -380,8 +380,6 @@ def grant_idempotent():
cql.execute(f"DROP AGGREGATE IF EXISTS {keyspace}.{custom_avg}(bigint)")

# Test that permissions for user-defined functions created on top of user-defined types work
# Fails on Scylla, because we currently don't properly support user-defined types in function permissions
@pytest.mark.xfail(reason='user-defined types not supported in function permissions yet')
def test_udf_permissions_with_udt(cql):
with new_test_keyspace(cql, "WITH REPLICATION = { 'class': 'SimpleStrategy', 'replication_factor': 1 }") as keyspace:
with new_type(cql, keyspace, '(v int)') as udt:
Expand All @@ -401,4 +399,22 @@ def test_udf_permissions_with_udt(cql):
cql.execute(f"INSERT INTO {table}(a) VALUES ((7))")
grant(cql, 'SELECT', table, username)
grant(cql, 'EXECUTE', f'FUNCTION {keyspace}.{fun}({udt})', username)
user_session.execute(f'SELECT {keyspace}.{fun}(a) FROM {table}')
user_session.execute(f'SELECT {keyspace}.{fun}(a) FROM {table}')

# Test that permissions on user-defined functions with no arguments work
def test_udf_permissions_no_args(cql):
with new_test_keyspace(cql, "WITH REPLICATION = { 'class': 'SimpleStrategy', 'replication_factor': 1 }") as keyspace:
with new_test_table(cql, keyspace, schema="a int primary key") as table, new_user(cql) as username:
with new_session(cql, username) as user_session:
fun_body_lua = f"() CALLED ON NULL INPUT RETURNS int LANGUAGE lua AS 'return 42;'"
fun_body_java = f"() CALLED ON NULL INPUT RETURNS int LANGUAGE java AS 'return 42;'"
fun_body = fun_body_lua
try:
with new_function(cql, keyspace, fun_body):
pass
except:
fun_body = fun_body_java
with new_function(cql, keyspace, fun_body) as fun:
grant(cql, 'SELECT', table, username)
check_enforced(cql, username, permission='EXECUTE', resource=f'function {keyspace}.{fun}()',
function=lambda: user_session.execute(f'SELECT {keyspace}.{fun}() FROM {table}'))

0 comments on commit 6b8c182

Please sign in to comment.