Merge pull request #70090 from arthurpassos/valid_until_per_auth_method

Allow each authentication method to have its own expiration date, remove from user entity.
This commit is contained in:
Konstantin Bogdanov 2024-10-30 13:26:56 +00:00 committed by GitHub
commit bceb1dce69
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
19 changed files with 257 additions and 95 deletions

View File

@ -12,7 +12,7 @@ Syntax:
``` sql
ALTER USER [IF EXISTS] name1 [RENAME TO new_name |, name2 [,...]]
[ON CLUSTER cluster_name]
[NOT IDENTIFIED | RESET AUTHENTICATION METHODS TO NEW | {IDENTIFIED | ADD IDENTIFIED} {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}
[NOT IDENTIFIED | RESET AUTHENTICATION METHODS TO NEW | {IDENTIFIED | ADD IDENTIFIED} {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']} [VALID UNTIL datetime]
[, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]]
[[ADD | DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[VALID UNTIL datetime]
@ -91,3 +91,15 @@ Reset authentication methods and keep the most recent added one:
``` sql
ALTER USER user1 RESET AUTHENTICATION METHODS TO NEW
```
## VALID UNTIL Clause
Allows you to specify the expiration date and, optionally, the time for an authentication method. It accepts a string as a parameter. It is recommended to use the `YYYY-MM-DD [hh:mm:ss] [timezone]` format for datetime. By default, this parameter equals `'infinity'`.
The `VALID UNTIL` clause can only be specified along with an authentication method, except for the case where no authentication method has been specified in the query. In this scenario, the `VALID UNTIL` clause will be applied to all existing authentication methods.
Examples:
- `ALTER USER name1 VALID UNTIL '2025-01-01'`
- `ALTER USER name1 VALID UNTIL '2025-01-01 12:00:00 UTC'`
- `ALTER USER name1 VALID UNTIL 'infinity'`
- `ALTER USER name1 IDENTIFIED WITH plaintext_password BY 'no_expiration', bcrypt_password BY 'expiration_set' VALID UNTIL'2025-01-01''`

View File

@ -11,7 +11,7 @@ Syntax:
``` sql
CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [, name2 [,...]] [ON CLUSTER cluster_name]
[NOT IDENTIFIED | IDENTIFIED {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']}
[NOT IDENTIFIED | IDENTIFIED {[WITH {plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | WITH NO_PASSWORD | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name' | SAN 'TYPE:subject_alt_name'} | {WITH ssh_key BY KEY 'public_key' TYPE 'ssh-rsa|...'} | {WITH http SERVER 'server_name' [SCHEME 'Basic']} [VALID UNTIL datetime]
[, {[{plaintext_password | sha256_password | sha256_hash | ...}] BY {'password' | 'hash'}} | {ldap SERVER 'server_name'} | {...} | ... [,...]]]
[HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]
[VALID UNTIL datetime]
@ -178,7 +178,8 @@ ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technica
## VALID UNTIL Clause
Allows you to specify the expiration date and, optionally, the time for user credentials. It accepts a string as a parameter. It is recommended to use the `YYYY-MM-DD [hh:mm:ss] [timezone]` format for datetime. By default, this parameter equals `'infinity'`.
Allows you to specify the expiration date and, optionally, the time for an authentication method. It accepts a string as a parameter. It is recommended to use the `YYYY-MM-DD [hh:mm:ss] [timezone]` format for datetime. By default, this parameter equals `'infinity'`.
The `VALID UNTIL` clause can only be specified along with an authentication method, except for the case where no authentication method has been specified in the query. In this scenario, the `VALID UNTIL` clause will be applied to all existing authentication methods.
Examples:
@ -186,6 +187,7 @@ Examples:
- `CREATE USER name1 VALID UNTIL '2025-01-01 12:00:00 UTC'`
- `CREATE USER name1 VALID UNTIL 'infinity'`
- ```CREATE USER name1 VALID UNTIL '2025-01-01 12:00:00 `Asia/Tokyo`'```
- `CREATE USER name1 IDENTIFIED WITH plaintext_password BY 'no_expiration', bcrypt_password BY 'expiration_set' VALID UNTIL '2025-01-01''`
## GRANTEES Clause

View File

@ -1,12 +1,16 @@
#include <Access/AccessControl.h>
#include <Access/AuthenticationData.h>
#include <Common/Exception.h>
#include <Interpreters/Access/getValidUntilFromAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/Access/ASTPublicSSHKey.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <IO/parseDateTimeBestEffort.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <Common/OpenSSLHelpers.h>
#include <Poco/SHA1Engine.h>
@ -113,7 +117,8 @@ bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs)
&& (lhs.ssh_keys == rhs.ssh_keys)
#endif
&& (lhs.http_auth_scheme == rhs.http_auth_scheme)
&& (lhs.http_auth_server_name == rhs.http_auth_server_name);
&& (lhs.http_auth_server_name == rhs.http_auth_server_name)
&& (lhs.valid_until == rhs.valid_until);
}
@ -384,14 +389,34 @@ std::shared_ptr<ASTAuthenticationData> AuthenticationData::toAST() const
throw Exception(ErrorCodes::LOGICAL_ERROR, "AST: Unexpected authentication type {}", toString(auth_type));
}
if (valid_until)
{
WriteBufferFromOwnString out;
writeDateTimeText(valid_until, out);
node->valid_until = std::make_shared<ASTLiteral>(out.str());
}
return node;
}
AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & query, ContextPtr context, bool validate)
{
time_t valid_until = 0;
if (query.valid_until)
{
valid_until = getValidUntilFromAST(query.valid_until, context);
}
if (query.type && query.type == AuthenticationType::NO_PASSWORD)
return AuthenticationData();
{
AuthenticationData auth_data;
auth_data.setValidUntil(valid_until);
return auth_data;
}
/// For this type of authentication we have ASTPublicSSHKey as children for ASTAuthenticationData
if (query.type && query.type == AuthenticationType::SSH_KEY)
@ -418,6 +443,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
}
auth_data.setSSHKeys(std::move(keys));
auth_data.setValidUntil(valid_until);
return auth_data;
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
@ -451,6 +477,8 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
AuthenticationData auth_data(current_type);
auth_data.setValidUntil(valid_until);
if (validate)
context->getAccessControl().checkPasswordComplexityRules(value);
@ -494,6 +522,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
}
AuthenticationData auth_data(*query.type);
auth_data.setValidUntil(valid_until);
if (query.contains_hash)
{

View File

@ -74,6 +74,9 @@ public:
const String & getHTTPAuthenticationServerName() const { return http_auth_server_name; }
void setHTTPAuthenticationServerName(const String & name) { http_auth_server_name = name; }
time_t getValidUntil() const { return valid_until; }
void setValidUntil(time_t valid_until_) { valid_until = valid_until_; }
friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs);
friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); }
@ -106,6 +109,7 @@ private:
/// HTTP authentication properties
String http_auth_server_name;
HTTPAuthenticationScheme http_auth_scheme = HTTPAuthenticationScheme::BASIC;
time_t valid_until = 0;
};
}

View File

@ -554,7 +554,7 @@ std::optional<AuthResult> IAccessStorage::authenticateImpl(
continue;
}
if (areCredentialsValid(user->getName(), user->valid_until, auth_method, credentials, external_authenticators, auth_result.settings))
if (areCredentialsValid(user->getName(), auth_method, credentials, external_authenticators, auth_result.settings))
{
auth_result.authentication_data = auth_method;
return auth_result;
@ -579,7 +579,6 @@ std::optional<AuthResult> IAccessStorage::authenticateImpl(
bool IAccessStorage::areCredentialsValid(
const std::string & user_name,
time_t valid_until,
const AuthenticationData & authentication_method,
const Credentials & credentials,
const ExternalAuthenticators & external_authenticators,
@ -591,6 +590,7 @@ bool IAccessStorage::areCredentialsValid(
if (credentials.getUserName() != user_name)
return false;
auto valid_until = authentication_method.getValidUntil();
if (valid_until)
{
const time_t now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());

View File

@ -236,7 +236,6 @@ protected:
bool allow_plaintext_password) const;
virtual bool areCredentialsValid(
const std::string & user_name,
time_t valid_until,
const AuthenticationData & authentication_method,
const Credentials & credentials,
const ExternalAuthenticators & external_authenticators,

View File

@ -19,8 +19,7 @@ bool User::equal(const IAccessEntity & other) const
return (authentication_methods == other_user.authentication_methods)
&& (allowed_client_hosts == other_user.allowed_client_hosts)
&& (access == other_user.access) && (granted_roles == other_user.granted_roles) && (default_roles == other_user.default_roles)
&& (settings == other_user.settings) && (grantees == other_user.grantees) && (default_database == other_user.default_database)
&& (valid_until == other_user.valid_until);
&& (settings == other_user.settings) && (grantees == other_user.grantees) && (default_database == other_user.default_database);
}
void User::setName(const String & name_)
@ -88,7 +87,6 @@ void User::clearAllExceptDependencies()
access = {};
settings.removeSettingsKeepProfiles();
default_database = {};
valid_until = 0;
}
}

View File

@ -23,7 +23,6 @@ struct User : public IAccessEntity
SettingsProfileElements settings;
RolesOrUsersSet grantees = RolesOrUsersSet::AllTag{};
String default_database;
time_t valid_until = 0;
bool equal(const IAccessEntity & other) const override;
std::shared_ptr<IAccessEntity> clone() const override { return cloneImpl<User>(); }

View File

@ -8,6 +8,7 @@
#include <Common/logger_useful.h>
#include <Core/ServerSettings.h>
#include <Interpreters/Access/InterpreterSetRoleQuery.h>
#include <Interpreters/Access/getValidUntilFromAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/removeOnClusterClauseIfNeeded.h>
@ -44,7 +45,7 @@ namespace
const std::optional<RolesOrUsersSet> & override_default_roles,
const std::optional<SettingsProfileElements> & override_settings,
const std::optional<RolesOrUsersSet> & override_grantees,
const std::optional<time_t> & valid_until,
const std::optional<time_t> & global_valid_until,
bool reset_authentication_methods,
bool replace_authentication_methods,
bool allow_implicit_no_password,
@ -105,12 +106,20 @@ namespace
user.authentication_methods.emplace_back(authentication_method);
}
bool has_no_password_authentication_method = std::find_if(user.authentication_methods.begin(),
user.authentication_methods.end(),
[](const AuthenticationData & authentication_data)
{
return authentication_data.getType() == AuthenticationType::NO_PASSWORD;
}) != user.authentication_methods.end();
bool has_no_password_authentication_method = false;
for (auto & authentication_method : user.authentication_methods)
{
if (global_valid_until)
{
authentication_method.setValidUntil(*global_valid_until);
}
if (authentication_method.getType() == AuthenticationType::NO_PASSWORD)
{
has_no_password_authentication_method = true;
}
}
if (has_no_password_authentication_method && user.authentication_methods.size() > 1)
{
@ -133,9 +142,6 @@ namespace
}
}
if (valid_until)
user.valid_until = *valid_until;
if (override_name && !override_name->host_pattern.empty())
{
user.allowed_client_hosts = AllowedClientHosts{};
@ -175,34 +181,6 @@ namespace
else if (query.grantees)
user.grantees = *query.grantees;
}
time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context)
{
if (context)
valid_until = evaluateConstantExpressionAsLiteral(valid_until, context);
const String valid_until_str = checkAndGetLiteralArgument<String>(valid_until, "valid_until");
if (valid_until_str == "infinity")
return 0;
time_t time = 0;
ReadBufferFromString in(valid_until_str);
if (context)
{
const auto & time_zone = DateLUT::instance("");
const auto & utc_time_zone = DateLUT::instance("UTC");
parseDateTimeBestEffort(time, in, time_zone, utc_time_zone);
}
else
{
readDateTimeText(time, in);
}
return time;
}
}
BlockIO InterpreterCreateUserQuery::execute()
@ -226,9 +204,9 @@ BlockIO InterpreterCreateUserQuery::execute()
}
}
std::optional<time_t> valid_until;
if (query.valid_until)
valid_until = getValidUntilFromAST(query.valid_until, getContext());
std::optional<time_t> global_valid_until;
if (query.global_valid_until)
global_valid_until = getValidUntilFromAST(query.global_valid_until, getContext());
std::optional<RolesOrUsersSet> default_roles_from_query;
if (query.default_roles)
@ -274,7 +252,7 @@ BlockIO InterpreterCreateUserQuery::execute()
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserFromQueryImpl(
*updated_user, query, authentication_methods, {}, default_roles_from_query, settings_from_query, grantees_from_query,
valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods,
global_valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods,
implicit_no_password_allowed, no_password_allowed,
plaintext_password_allowed, getContext()->getServerSettings()[ServerSetting::max_authentication_methods_per_user]);
return updated_user;
@ -296,7 +274,7 @@ BlockIO InterpreterCreateUserQuery::execute()
auto new_user = std::make_shared<User>();
updateUserFromQueryImpl(
*new_user, query, authentication_methods, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{},
valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods,
global_valid_until, query.reset_authentication_methods_to_new, query.replace_authentication_methods,
implicit_no_password_allowed, no_password_allowed,
plaintext_password_allowed, getContext()->getServerSettings()[ServerSetting::max_authentication_methods_per_user]);
new_users.emplace_back(std::move(new_user));
@ -351,9 +329,9 @@ void InterpreterCreateUserQuery::updateUserFromQuery(
}
}
std::optional<time_t> valid_until;
if (query.valid_until)
valid_until = getValidUntilFromAST(query.valid_until, {});
std::optional<time_t> global_valid_until;
if (query.global_valid_until)
global_valid_until = getValidUntilFromAST(query.global_valid_until, {});
updateUserFromQueryImpl(
user,
@ -363,7 +341,7 @@ void InterpreterCreateUserQuery::updateUserFromQuery(
{},
{},
{},
valid_until,
global_valid_until,
query.reset_authentication_methods_to_new,
query.replace_authentication_methods,
allow_no_password,

View File

@ -69,13 +69,6 @@ namespace
query->authentication_methods.push_back(authentication_method.toAST());
}
if (user.valid_until)
{
WriteBufferFromOwnString out;
writeDateTimeText(user.valid_until, out);
query->valid_until = std::make_shared<ASTLiteral>(out.str());
}
if (!user.settings.empty())
{
if (attach_mode)

View File

@ -0,0 +1,37 @@
#include <Interpreters/Access/getValidUntilFromAST.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <IO/parseDateTimeBestEffort.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <Storages/checkAndGetLiteralArgument.h>
namespace DB
{
time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context)
{
if (context)
valid_until = evaluateConstantExpressionAsLiteral(valid_until, context);
const String valid_until_str = checkAndGetLiteralArgument<String>(valid_until, "valid_until");
if (valid_until_str == "infinity")
return 0;
time_t time = 0;
ReadBufferFromString in(valid_until_str);
if (context)
{
const auto & time_zone = DateLUT::instance("");
const auto & utc_time_zone = DateLUT::instance("UTC");
parseDateTimeBestEffort(time, in, time_zone, utc_time_zone);
}
else
{
readDateTimeText(time, in);
}
return time;
}
}

View File

@ -0,0 +1,9 @@
#pragma once
#include <Parsers/IAST.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{
time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context);
}

View File

@ -383,12 +383,12 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So
void Session::checkIfUserIsStillValid()
{
if (user && user->valid_until)
if (const auto valid_until = user_authenticated_with.getValidUntil())
{
const time_t now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
if (now > user->valid_until)
throw Exception(ErrorCodes::USER_EXPIRED, "User expired");
if (now > valid_until)
throw Exception(ErrorCodes::USER_EXPIRED, "Authentication method used has expired");
}
}

View File

@ -14,6 +14,15 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
namespace
{
void formatValidUntil(const IAST & valid_until, const IAST::FormatSettings & settings)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " VALID UNTIL " << (settings.hilite ? IAST::hilite_none : "");
valid_until.format(settings);
}
}
std::optional<String> ASTAuthenticationData::getPassword() const
{
if (contains_password)
@ -46,6 +55,12 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " no_password"
<< (settings.hilite ? IAST::hilite_none : "");
if (valid_until)
{
formatValidUntil(*valid_until, settings);
}
return;
}
@ -205,6 +220,11 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt
children[1]->format(settings);
}
if (valid_until)
{
formatValidUntil(*valid_until, settings);
}
}
bool ASTAuthenticationData::hasSecretParts() const

View File

@ -41,6 +41,7 @@ public:
bool contains_password = false;
bool contains_hash = false;
ASTPtr valid_until;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;

View File

@ -260,8 +260,10 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState &
formatAuthenticationData(authentication_methods, format);
}
if (valid_until)
formatValidUntil(*valid_until, format);
if (global_valid_until)
{
formatValidUntil(*global_valid_until, format);
}
if (hosts)
formatHosts(nullptr, *hosts, format);

View File

@ -62,7 +62,7 @@ public:
std::shared_ptr<ASTDatabaseOrNone> default_database;
ASTPtr valid_until;
ASTPtr global_valid_until;
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -43,6 +43,19 @@ namespace
});
}
bool parseValidUntil(IParserBase::Pos & pos, Expected & expected, ASTPtr & valid_until)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{Keyword::VALID_UNTIL}.ignore(pos, expected))
return false;
ParserStringAndSubstitution until_p;
return until_p.parse(pos, valid_until, expected);
});
}
bool parseAuthenticationData(
IParserBase::Pos & pos,
Expected & expected,
@ -223,6 +236,8 @@ namespace
if (http_auth_scheme)
auth_data->children.push_back(std::move(http_auth_scheme));
parseValidUntil(pos, expected, auth_data->valid_until);
return true;
});
}
@ -283,6 +298,8 @@ namespace
authentication_methods.emplace_back(std::make_shared<ASTAuthenticationData>());
authentication_methods.back()->type = AuthenticationType::NO_PASSWORD;
parseValidUntil(pos, expected, authentication_methods.back()->valid_until);
return true;
}
@ -471,19 +488,6 @@ namespace
});
}
bool parseValidUntil(IParserBase::Pos & pos, Expected & expected, ASTPtr & valid_until)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{Keyword::VALID_UNTIL}.ignore(pos, expected))
return false;
ParserStringAndSubstitution until_p;
return until_p.parse(pos, valid_until, expected);
});
}
bool parseAddIdentifiedWith(IParserBase::Pos & pos, Expected & expected, std::vector<std::shared_ptr<ASTAuthenticationData>> & auth_data)
{
return IParserBase::wrapParseImpl(pos, [&]
@ -554,7 +558,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
std::shared_ptr<ASTSettingsProfileElements> settings;
std::shared_ptr<ASTRolesOrUsersSet> grantees;
std::shared_ptr<ASTDatabaseOrNone> default_database;
ASTPtr valid_until;
ASTPtr global_valid_until;
String cluster;
String storage_name;
bool reset_authentication_methods_to_new = false;
@ -568,20 +572,27 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
{
parsed_identified_with = parseIdentifiedOrNotIdentified(pos, expected, auth_data);
if (!parsed_identified_with && alter)
if (parsed_identified_with)
{
continue;
}
else if (alter)
{
parsed_add_identified_with = parseAddIdentifiedWith(pos, expected, auth_data);
if (parsed_add_identified_with)
{
continue;
}
}
}
if (!reset_authentication_methods_to_new && alter && auth_data.empty())
{
reset_authentication_methods_to_new = parseResetAuthenticationMethods(pos, expected);
}
if (!valid_until)
{
parseValidUntil(pos, expected, valid_until);
if (reset_authentication_methods_to_new)
{
continue;
}
}
AllowedClientHosts new_hosts;
@ -640,6 +651,14 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (storage_name.empty() && ParserKeyword{Keyword::IN}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
continue;
if (auth_data.empty() && !global_valid_until)
{
if (parseValidUntil(pos, expected, global_valid_until))
{
continue;
}
}
break;
}
@ -674,7 +693,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->settings = std::move(settings);
query->grantees = std::move(grantees);
query->default_database = std::move(default_database);
query->valid_until = std::move(valid_until);
query->global_valid_until = std::move(global_valid_until);
query->storage_name = std::move(storage_name);
query->reset_authentication_methods_to_new = reset_authentication_methods_to_new;
query->add_identified_with = parsed_add_identified_with;
@ -685,8 +704,8 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->children.push_back(authentication_method);
}
if (query->valid_until)
query->children.push_back(query->valid_until);
if (query->global_valid_until)
query->children.push_back(query->global_valid_until);
return true;
}

View File

@ -76,6 +76,18 @@ def test_basic(started_cluster):
node.query("DROP USER IF EXISTS user_basic")
# NOT IDENTIFIED test to make sure valid until is also parsed on its short-circuit
node.query("CREATE USER user_basic NOT IDENTIFIED VALID UNTIL '01/01/2010'")
assert (
node.query("SHOW CREATE USER user_basic")
== "CREATE USER user_basic IDENTIFIED WITH no_password VALID UNTIL \\'2010-01-01 00:00:00\\'\n"
)
assert error in node.query_and_get_error("SELECT 1", user="user_basic")
node.query("DROP USER IF EXISTS user_basic")
def test_details(started_cluster):
node.query("DROP USER IF EXISTS user_details_infinity, user_details_time_only")
@ -124,3 +136,51 @@ def test_restart(started_cluster):
assert error in node.query_and_get_error("SELECT 1", user="user_restart")
node.query("DROP USER IF EXISTS user_restart")
def test_multiple_authentication_methods(started_cluster):
node.query("DROP USER IF EXISTS user_basic")
node.query(
"CREATE USER user_basic IDENTIFIED WITH plaintext_password BY 'no_expiration',"
"plaintext_password by 'not_expired' VALID UNTIL '06/11/2040', plaintext_password by 'expired' VALID UNTIL '06/11/2010',"
"plaintext_password by 'infinity' VALID UNTIL 'infinity'"
)
assert (
node.query("SHOW CREATE USER user_basic")
== "CREATE USER user_basic IDENTIFIED WITH plaintext_password, plaintext_password VALID UNTIL \\'2040-11-06 00:00:00\\', "
"plaintext_password VALID UNTIL \\'2010-11-06 00:00:00\\', plaintext_password\n"
)
assert node.query("SELECT 1", user="user_basic", password="no_expiration") == "1\n"
assert node.query("SELECT 1", user="user_basic", password="not_expired") == "1\n"
assert node.query("SELECT 1", user="user_basic", password="infinity") == "1\n"
error = "Authentication failed"
assert error in node.query_and_get_error(
"SELECT 1", user="user_basic", password="expired"
)
# Expire them all
node.query("ALTER USER user_basic VALID UNTIL '06/11/2010 08:03:20'")
assert (
node.query("SHOW CREATE USER user_basic")
== "CREATE USER user_basic IDENTIFIED WITH plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\',"
" plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\',"
" plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\',"
" plaintext_password VALID UNTIL \\'2010-11-06 08:03:20\\'\n"
)
assert error in node.query_and_get_error(
"SELECT 1", user="user_basic", password="no_expiration"
)
assert error in node.query_and_get_error(
"SELECT 1", user="user_basic", password="not_expired"
)
assert error in node.query_and_get_error(
"SELECT 1", user="user_basic", password="infinity"
)
assert error in node.query_and_get_error(
"SELECT 1", user="user_basic", password="expired"
)