Add TOTP for authentification

Co-authored-by: Denis Kamenskii <dekamenskii@mail.ru>
This commit is contained in:
vdimir 2024-10-30 16:35:01 +00:00
parent 2c4d93f552
commit 4c0e620b92
No known key found for this signature in database
GPG Key ID: 6EE4CE2BEDC51862
25 changed files with 499 additions and 10 deletions

3
.gitmodules vendored
View File

@ -369,3 +369,6 @@
[submodule "contrib/postgres"]
path = contrib/postgres
url = https://github.com/ClickHouse/postgres.git
[submodule "contrib/libcotp"]
path = contrib/libcotp
url = git@github.com:paolostivanin/libcotp.git

View File

@ -219,6 +219,8 @@ add_contrib (prometheus-protobufs-cmake prometheus-protobufs prometheus-protobuf
add_contrib(numactl-cmake numactl)
add_contrib (libcotp-cmake libcotp)
# Put all targets defined here and in subdirectories under "contrib/<immediate-subdir>" folders in GUI-based IDEs.
# Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear
# in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually,

1
contrib/libcotp vendored Submodule

@ -0,0 +1 @@
Subproject commit 78a3783ac19604e9e3ad7053f1c43c761066bfb4

View File

@ -0,0 +1,22 @@
if(NOT ENABLE_SSL)
message(STATUS "libcotp: DISABLED because ENABLE_SSL=OFF")
return()
endif()
set (LIBCOTP_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libcotp")
set (LIBCOTP_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/libcotp")
set(SRCS
"${LIBCOTP_SOURCE_DIR}/src/otp.c"
"${LIBCOTP_SOURCE_DIR}/src/utils/base32.c"
"${LIBCOTP_SOURCE_DIR}/src/utils/whmac_openssl.c"
)
add_library (_libcotp ${SRCS})
file(MAKE_DIRECTORY "${LIBCOTP_BINARY_DIR}/include")
file(COPY "${LIBCOTP_SOURCE_DIR}/src/" DESTINATION "${LIBCOTP_BINARY_DIR}/include" FILES_MATCHING PATTERN "*.h")
target_include_directories(_libcotp SYSTEM BEFORE PUBLIC "${LIBCOTP_BINARY_DIR}/include")
target_link_libraries(_libcotp PRIVATE OpenSSL::Crypto)
add_library(ch_contrib::libcotp ALIAS _libcotp)

View File

@ -16,4 +16,5 @@ The following external authenticators and directories are supported:
- [LDAP](./ldap.md#external-authenticators-ldap) [Authenticator](./ldap.md#ldap-external-authenticator) and [Directory](./ldap.md#ldap-external-user-directory)
- Kerberos [Authenticator](./kerberos.md#external-authenticators-kerberos)
- [SSL X.509 authentication](./ssl-x509.md#ssl-external-authentication)
- HTTP [Authenticator](./http.md)
- HTTP [Authenticator](./http.md)
- [Time-based one-time password](./totp.md)

View File

@ -0,0 +1,7 @@
---
slug: /en/operations/external-authenticators/
title: "HTTP"
---
import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md';
<SelfManaged />

View File

@ -32,6 +32,24 @@
If you want to specify double SHA1, place it in 'password_double_sha1_hex' element.
Example: <password_double_sha1_hex>e395796d6546b1b65db9d665cd43f0e858dd4303</password_double_sha1_hex>
If you want to specify secret for one time passwords, place it in 'time_based_one_time_password' element.
Example:
<time_based_one_time_password>
<secret>ALLT7V6M3NNSQXMEMPEAUSX77OOUTBSO</secret>
<period>30</period>
<digits>6</digits>
<algorithm>SHA1</algorithm>
</time_based_one_time_password>
secret - base32 encoded secret key
period - password validity duration in seconds, default is 30
digits - number of digits in password, default is 6
algorithm - hash algorithm: SHA1/SHA256/SHA512, default is SHA1
How to generate secret:
Execute: echo `base32 < /dev/urandom | head -c32`
More info: https://datatracker.ietf.org/doc/html/rfc6238
If you want to specify a previously defined LDAP server (see 'ldap_servers' in the main config) for authentication,
place its name in 'server' element inside 'ldap' element.
Example: <ldap><server>my_ldap_server</server></ldap>

View File

@ -611,7 +611,7 @@ AuthResult AccessControl::authenticate(const Credentials & credentials, const Po
tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed");
WriteBufferFromOwnString message;
message << credentials.getUserName() << ": Authentication failed: password is incorrect, or there is no user with such name.";
message << credentials.getUserName() << ": Authentication failed: password is incorrect, or there is no user with such name";
/// Better exception message for usability.
/// It is typical when users install ClickHouse, type some password and instantly forget it.

View File

@ -9,6 +9,7 @@
#include <Common/SSHWrapper.h>
#include <Common/typeid_cast.h>
#include <Access/Common/SSLCertificateSubjects.h>
#include <Access/Common/OneTimePassword.h>
#include "config.h"
@ -136,6 +137,13 @@ namespace
{
return checkPasswordDoubleSHA1(basic_credentials->getPassword(), authentication_method.getPasswordHashBinary());
}
case AuthenticationType::ONE_TIME_PASSWORD:
{
return checkOneTimePassword(
/* password */ basic_credentials->getPassword(),
/* secret */ authentication_method.getPassword(),
/* config */ authentication_method.getOneTimePasswordConfig());
}
case AuthenticationType::LDAP:
{
return external_authenticators.checkLDAPCredentials(authentication_method.getLDAPServerName(), *basic_credentials);

View File

@ -133,6 +133,10 @@ void AuthenticationData::setPassword(const String & password_, bool validate)
setPasswordHashBinary(Util::encodeDoubleSHA1(password_), validate);
return;
case AuthenticationType::ONE_TIME_PASSWORD:
setPasswordHashBinary(Util::stringToDigest(normalizeOneTimePasswordSecret(password_)), validate);
return;
case AuthenticationType::BCRYPT_PASSWORD:
case AuthenticationType::NO_PASSWORD:
case AuthenticationType::LDAP:
@ -149,6 +153,15 @@ void AuthenticationData::setPassword(const String & password_, bool validate)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "setPassword(): authentication type {} not supported", toString(type));
}
void AuthenticationData::setOneTimePassword(const String & password_, OneTimePasswordConfig config, bool validate)
{
if (type != AuthenticationType::ONE_TIME_PASSWORD)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify one-time password for authentication type {}", toString(type));
otp_config = config;
setPasswordHashBinary(Util::stringToDigest(normalizeOneTimePasswordSecret(password_)), validate);
}
void AuthenticationData::setPasswordBcrypt(const String & password_, int workfactor_, bool validate)
{
if (type != AuthenticationType::BCRYPT_PASSWORD)
@ -159,8 +172,9 @@ void AuthenticationData::setPasswordBcrypt(const String & password_, int workfac
String AuthenticationData::getPassword() const
{
if (type != AuthenticationType::PLAINTEXT_PASSWORD)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot decode the password");
if (type != AuthenticationType::PLAINTEXT_PASSWORD
&& type != AuthenticationType::ONE_TIME_PASSWORD)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot decode the password for authentication type {}", type);
return String(password_hash.data(), password_hash.data() + password_hash.size());
}
@ -205,6 +219,12 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash, bool validat
return;
}
case AuthenticationType::ONE_TIME_PASSWORD:
{
password_hash = hash;
return;
}
case AuthenticationType::SHA256_PASSWORD:
{
if (hash.size() != 32)
@ -307,6 +327,12 @@ std::shared_ptr<ASTAuthenticationData> AuthenticationData::toAST() const
node->children.push_back(std::make_shared<ASTLiteral>(getPassword()));
break;
}
case AuthenticationType::ONE_TIME_PASSWORD:
{
node->contains_password = true;
node->children.push_back(std::make_shared<ASTLiteral>(getPassword()));
break;
}
case AuthenticationType::SHA256_PASSWORD:
{
node->contains_hash = true;

View File

@ -3,6 +3,7 @@
#include <Access/Common/AuthenticationType.h>
#include <Access/Common/HTTPAuthenticationScheme.h>
#include <Access/Common/SSLCertificateSubjects.h>
#include <Access/Common/OneTimePassword.h>
#include <Common/SSHWrapper.h>
#include <Interpreters/Context_fwd.h>
#include <Parsers/Access/ASTAuthenticationData.h>
@ -36,6 +37,9 @@ public:
/// Returns the password. Allowed to use only for Type::PLAINTEXT_PASSWORD.
String getPassword() const;
void setOneTimePassword(const String & password_, OneTimePasswordConfig config, bool validate);
const OneTimePasswordConfig & getOneTimePasswordConfig() const { return otp_config; }
/// Sets the password as a string of hexadecimal digits.
void setPasswordHashHex(const String & hash, bool validate);
String getPasswordHashHex() const;
@ -96,6 +100,7 @@ public:
private:
AuthenticationType type = AuthenticationType::NO_PASSWORD;
Digest password_hash;
OneTimePasswordConfig otp_config;
String ldap_server_name;
String kerberos_realm;
SSLCertificateSubjects ssl_certificate_subjects;

View File

@ -42,6 +42,11 @@ const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType ty
static const auto info = make_info(Keyword::DOUBLE_SHA1_PASSWORD, true);
return info;
}
case AuthenticationType::ONE_TIME_PASSWORD:
{
static const auto info = make_info(Keyword::ONE_TIME_PASSWORD, true);
return info;
}
case AuthenticationType::LDAP:
{
static const auto info = make_info(Keyword::LDAP);

View File

@ -21,6 +21,10 @@ enum class AuthenticationType : uint8_t
/// This kind of hash is used by the `mysql_native_password` authentication plugin.
DOUBLE_SHA1_PASSWORD,
/// A secret is stored instead of a password
/// Users are authenticated using time-based one-time passwords
ONE_TIME_PASSWORD,
/// Password is checked by a [remote] LDAP server. Connection will be made at each authentication attempt.
LDAP,

View File

@ -3,3 +3,7 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
add_headers_and_sources(clickhouse_common_access .)
add_library(clickhouse_common_access ${clickhouse_common_access_headers} ${clickhouse_common_access_sources})
target_link_libraries(clickhouse_common_access PUBLIC clickhouse_common_io)
if (TARGET ch_contrib::libcotp)
target_link_libraries (clickhouse_common_access PRIVATE ch_contrib::libcotp)
endif()

View File

@ -0,0 +1,179 @@
#include <Access/Common/OneTimePassword.h>
#include <Common/Exception.h>
#include <Common/ErrorCodes.h>
#include <Common/logger_useful.h>
#include <fmt/format.h>
#include <Poco/String.h>
#include "config.h"
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
static const UInt8 b32_alphabet[] = u8"ABCDEFGHIJKLMNOPQRSTUVWXYZ234567";
String normalizeOneTimePasswordSecret(const String & secret)
{
static const UInt8 b32_lower_alphabet[] = u8"abcdefghijklmnopqrstuvwxyz";
constexpr static UInt8 CHAR_IS_VALID = 1;
constexpr static UInt8 CHAR_IS_LOWER = 2;
std::array<UInt8, 128> table = {};
for (const auto * p = b32_alphabet; *p; p++)
table[*p] = CHAR_IS_VALID;
for (const auto * p = b32_lower_alphabet; *p; p++)
table[*p] = CHAR_IS_LOWER;
String result = secret;
size_t i = 0;
size_t n = 0;
for (; i < secret.size(); ++i)
{
if (secret[i] == ' ' || secret[i] == '=')
continue;
size_t idx = static_cast<UInt8>(secret[i]);
if (idx >= table.size() || table[idx] == 0)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid character in base32 secret: '{}'", secret[i]);
if (table[idx] == CHAR_IS_VALID)
result[n] = secret[i];
if (table[idx] == CHAR_IS_LOWER)
result[n] = std::toupper(secret[i]);
++n;
}
result.resize(n);
if (result.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty secret for one-time password");
return result;
}
static bool validateBase32Secret(const String & secret)
{
if (secret.empty())
return false;
std::array<UInt8, 128> table = {};
for (const auto * p = b32_alphabet; *p; p++)
table[*p] = 1;
for (const auto c : secret)
{
size_t idx = static_cast<UInt8>(c);
if (idx >= table.size() || table[idx] == 0)
return false;
}
return true;
}
static std::string_view toString(OneTimePasswordConfig::Algorithm algorithm)
{
switch (algorithm)
{
case OneTimePasswordConfig::Algorithm::SHA1: return "SHA1";
case OneTimePasswordConfig::Algorithm::SHA256: return "SHA256";
case OneTimePasswordConfig::Algorithm::SHA512: return "SHA512";
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown algorithm for one-time password: {}", static_cast<UInt32>(algorithm));
}
static OneTimePasswordConfig::Algorithm hashingAlgorithmFromString(const String & algorithm_name)
{
for (auto alg : {OneTimePasswordConfig::Algorithm::SHA1, OneTimePasswordConfig::Algorithm::SHA256, OneTimePasswordConfig::Algorithm::SHA512})
{
if (Poco::toUpper(algorithm_name) == toString(alg))
return alg;
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown algorithm for one-time password: '{}'", algorithm_name);
}
OneTimePasswordConfig::OneTimePasswordConfig(Int32 num_digits_, Int32 period_, const String & algorithm_name_)
{
if (num_digits_)
num_digits = num_digits_;
if (period_)
period = period_;
if (!algorithm_name_.empty())
algorithm = hashingAlgorithmFromString(algorithm_name_);
if (num_digits < 4 || 10 < num_digits)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid number of digits for one-time password: {}", num_digits);
if (period <= 0 || 120 < period)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid period for one-time password: {}", period);
}
std::string_view OneTimePasswordConfig::getAlgorithmName() const { return toString(algorithm); }
String getOneTimePasswordLink(const String & secret, const OneTimePasswordConfig & config)
{
validateBase32Secret(secret);
if (config == OneTimePasswordConfig{})
return fmt::format("otpauth://totp/ClickHouse?issuer=ClickHouse&secret={}", secret);
return fmt::format("otpauth://totp/ClickHouse?issuer=ClickHouse&secret={}&digits={}&period={}&algorithm={}",
secret, config.num_digits, config.period, toString(config.algorithm));
}
bool checkOneTimePassword(const String & password, const String & secret, const OneTimePasswordConfig & config)
{
return password == getOneTimePassword(secret, config);
}
}
#if USE_SSL
#include <cotp.h>
constexpr int TOTP_SHA512 = SHA512;
constexpr int TOTP_SHA256 = SHA256;
constexpr int TOTP_SHA1 = SHA1;
#undef SHA512
#undef SHA256
#undef SHA1
namespace DB
{
String getOneTimePassword(const String & secret, const OneTimePasswordConfig & config)
{
validateBase32Secret(secret);
cotp_error_t error;
int sha_algo = config.algorithm == OneTimePasswordConfig::Algorithm::SHA512 ? TOTP_SHA512
: config.algorithm == OneTimePasswordConfig::Algorithm::SHA256 ? TOTP_SHA256
: TOTP_SHA1;
auto result = std::unique_ptr<char>(get_totp(secret.c_str(), config.num_digits, config.period, sha_algo, &error));
if (result == nullptr || (error != NO_ERROR && error != VALID))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while retrieving one-time password, code: {}",
static_cast<std::underlying_type_t<cotp_error_t>>(error));
return String(result.get(), strlen(result.get()));
}
}
#else
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
}
String getOneTimePassword(const String & secret)
{
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "One-time password support is disabled, because ClickHouse was built without openssl library");
}
}
#endif

View File

@ -0,0 +1,37 @@
#pragma once
#include <base/types.h>
namespace DB
{
struct OneTimePasswordConfig
{
Int32 num_digits = 6;
Int32 period = 30;
enum class Algorithm : UInt8
{
SHA1,
SHA256,
SHA512,
} algorithm = Algorithm::SHA1;
explicit OneTimePasswordConfig(Int32 num_digits_ = 0, Int32 period_ = 0, const String & algorithm_name_ = "");
bool operator==(const OneTimePasswordConfig &) const = default;
std::string_view getAlgorithmName() const;
};
String getOneTimePasswordLink(const String & secret, const OneTimePasswordConfig & config);
String getOneTimePassword(const String & secret, const OneTimePasswordConfig & config);
bool checkOneTimePassword(const String & password, const String & secret, const OneTimePasswordConfig & config);
/// Checks if the secret contains only valid base32 characters.
/// The secret may contain spaces, which are ignored and lower-case characters, which are converted to upper-case.
String normalizeOneTimePasswordSecret(const String & secret);
}

View File

@ -7,6 +7,8 @@
#include <Access/SettingsProfile.h>
#include <Access/AccessControl.h>
#include <Access/resolveSetting.h>
#include <Access/Common/AuthenticationType.h>
#include <Access/Common/OneTimePassword.h>
#include <Access/AccessChangesNotifier.h>
#include <Dictionaries/IDictionary.h>
#include <Common/Config/ConfigReloader.h>
@ -127,6 +129,7 @@ namespace
String user_config = "users." + user_name;
bool has_no_password = config.has(user_config + ".no_password");
bool has_password_plaintext = config.has(user_config + ".password");
bool has_otp_secret = config.has(user_config + ".time_based_one_time_password");
bool has_password_sha256_hex = config.has(user_config + ".password_sha256_hex");
bool has_password_double_sha1_hex = config.has(user_config + ".password_double_sha1_hex");
bool has_ldap = config.has(user_config + ".ldap");
@ -142,11 +145,11 @@ namespace
bool has_http_auth = config.has(http_auth_config);
size_t num_password_fields = has_no_password + has_password_plaintext + has_password_sha256_hex + has_password_double_sha1_hex
+ has_ldap + has_kerberos + has_certificates + has_ssh_keys + has_http_auth;
+ has_ldap + has_kerberos + has_certificates + has_ssh_keys + has_http_auth + has_otp_secret;
if (num_password_fields > 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "More than one field of 'password', 'password_sha256_hex', "
"'password_double_sha1_hex', 'no_password', 'ldap', 'kerberos', 'ssl_certificates', 'ssh_keys', "
"'password_double_sha1_hex', 'no_password', 'time_based_one_time_password', 'ldap', 'kerberos', 'ssl_certificates', 'ssh_keys', "
"'http_authentication' are used to specify authentication info for user {}. "
"Must be only one of them.", user_name);
@ -170,6 +173,16 @@ namespace
user->authentication_methods.emplace_back(AuthenticationType::DOUBLE_SHA1_PASSWORD);
user->authentication_methods.back().setPasswordHashHex(config.getString(user_config + ".password_double_sha1_hex"), validate);
}
else if (has_otp_secret)
{
String secret = config.getString(user_config + ".time_based_one_time_password.secret", "");
OneTimePasswordConfig otp_config(
config.getInt(user_config + ".time_based_one_time_password.digits", {}),
config.getInt(user_config + ".time_based_one_time_password.period", {}),
config.getString(user_config + ".time_based_one_time_password.algorithm", "")
);
user->authentication_methods.emplace_back(AuthenticationType::ONE_TIME_PASSWORD).setOneTimePassword(secret, otp_config, validate);
}
else if (has_ldap)
{
bool has_ldap_server = config.has(user_config + ".ldap.server");
@ -269,7 +282,7 @@ namespace
}
else
{
user->authentication_methods.emplace_back();
user->authentication_methods.emplace_back(AuthenticationType::NO_PASSWORD);
}
for (const auto & authentication_method : user->authentication_methods)

View File

@ -85,6 +85,7 @@ ColumnsDescription SessionLogElement::getColumnsDescription()
AUTH_TYPE_NAME_AND_VALUE(AuthType::PLAINTEXT_PASSWORD),
AUTH_TYPE_NAME_AND_VALUE(AuthType::SHA256_PASSWORD),
AUTH_TYPE_NAME_AND_VALUE(AuthType::DOUBLE_SHA1_PASSWORD),
AUTH_TYPE_NAME_AND_VALUE(AuthType::ONE_TIME_PASSWORD),
AUTH_TYPE_NAME_AND_VALUE(AuthType::LDAP),
AUTH_TYPE_NAME_AND_VALUE(AuthType::JWT),
AUTH_TYPE_NAME_AND_VALUE(AuthType::KERBEROS),
@ -94,7 +95,7 @@ ColumnsDescription SessionLogElement::getColumnsDescription()
AUTH_TYPE_NAME_AND_VALUE(AuthType::HTTP),
});
#undef AUTH_TYPE_NAME_AND_VALUE
static_assert(static_cast<int>(AuthenticationType::MAX) == 11);
static_assert(static_cast<int>(AuthenticationType::MAX) == 12);
auto interface_type_column = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values

View File

@ -89,6 +89,12 @@ void ASTAuthenticationData::formatImpl(const FormatSettings & settings, FormatSt
password = true;
break;
}
case AuthenticationType::ONE_TIME_PASSWORD:
{
prefix = "BY";
password = true;
break;
}
case AuthenticationType::JWT:
{
prefix = "CLAIMS";

View File

@ -561,6 +561,7 @@ namespace DB
MR_MACROS(DOUBLE_SHA1_PASSWORD, "DOUBLE_SHA1_PASSWORD") \
MR_MACROS(IS_OBJECT_ID, "IS_OBJECT_ID") \
MR_MACROS(NO_PASSWORD, "NO_PASSWORD") \
MR_MACROS(ONE_TIME_PASSWORD, "ONE_TIME_PASSWORD") \
MR_MACROS(PART_MOVE_TO_SHARD, "PART_MOVE_TO_SHARD") \
MR_MACROS(PLAINTEXT_PASSWORD, "PLAINTEXT_PASSWORD") \
MR_MACROS(SHA256_HASH, "SHA256_HASH") \

View File

@ -149,6 +149,16 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr conte
{
auth_params_json.set("realm", auth_data.getKerberosRealm());
}
else if (auth_data.getType() == AuthenticationType::ONE_TIME_PASSWORD)
{
const auto & config = auth_data.getOneTimePasswordConfig();
if (config != OneTimePasswordConfig{})
{
auth_params_json.set("algorithm", toString(config.algorithm));
auth_params_json.set("num_digits", toString(config.num_digits));
auth_params_json.set("period", toString(config.period));
}
}
else if (auth_data.getType() == AuthenticationType::SSL_CERTIFICATE)
{
Poco::JSON::Array::Ptr common_names = new Poco::JSON::Array();

View File

@ -341,14 +341,15 @@
"NULL"
"NULLS"
"OFFSET"
"ON"
"ON DELETE"
"ON UPDATE"
"ON VOLUME"
"ON"
"ONE_TIME_PASSWORD"
"ONLY"
"OPTIMIZE TABLE"
"OR"
"OR REPLACE"
"OR"
"ORDER BY"
"OUTER"
"OVER"

View File

@ -0,0 +1,23 @@
<clickhouse>
<profiles>
<default>
</default>
</profiles>
<users>
<totuser>
<time_based_one_time_password>
<secret>inwg sy3l jbxx k43f biaa ====</secret>
<period>10</period>
<digits>9</digits>
<algorithm>SHA1</algorithm>
</time_based_one_time_password>
<access_management>1</access_management>
<networks replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</totuser>
</users>
</clickhouse>

View File

@ -0,0 +1,112 @@
import pytest
import base64
import hmac
import struct
import time
import hashlib
from fnmatch import fnmatch
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
user_configs=["config/users.xml"],
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def generate_totp(secret, interval=30, digits=6):
key = base64.b32decode(secret, casefold=True)
time_step = int(time.time() / interval)
msg = struct.pack(">Q", time_step)
hmac_hash = hmac.new(key, msg, hashlib.sha1).digest()
offset = hmac_hash[-1] & 0x0F
binary_code = struct.unpack(">I", hmac_hash[offset : offset + 4])[0] & 0x7FFFFFFF
otp = binary_code % (10**digits)
return f"{otp:0{digits}d}"
def test_one_time_password(started_cluster):
query_text = "SELECT currentUser() || toString(42)"
totuser_secret = {"secret": "INWGSY3LJBXXK43FBIAA====", "interval": 10, "digits": 9}
old_password = generate_totp(**totuser_secret)
old_password_created = time.time()
assert "totuser42\n" == node.query(
query_text, user="totuser", password=old_password
)
assert "totuser42\n" == node.query(
query_text, user="totuser", password=generate_totp(**totuser_secret)
)
assert "CREATE USER totuser IDENTIFIED WITH one_time_password" in node.query(
"SHOW CREATE USER totuser",
user="totuser",
password=generate_totp(**totuser_secret),
)
for bad_secret, error_message in [
("i11egalbase32", "Invalid character in*secret"),
(" ", "Empty secret"),
(" =", "Empty secret"),
("", "Empty secret"),
]:
err_resp = node.query_and_get_error(
f"CREATE USER user2 IDENTIFIED WITH one_time_password BY '{bad_secret}'",
user="totuser",
password=generate_totp(**totuser_secret),
)
assert fnmatch(err_resp, f"*{error_message}*BAD_ARGUMENTS*"), err_resp
node.query(
"CREATE USER user2 IDENTIFIED WITH one_time_password BY 'WLFVSILVT3PKZPKONCMGAHN7KBPTUX2J'",
user="totuser",
password=generate_totp(**totuser_secret),
)
assert "user242\n" == node.query(
query_text,
user="user2",
password=generate_totp("WLFVSILVT3PKZPKONCMGAHN7KBPTUX2J"),
)
resp = node.query(
"""
SELECT
name,
auth_type[1],
JSONExtractString(auth_params[1], 'algorithm'),
JSONExtractString(auth_params[1], 'num_digits'),
JSONExtractString(auth_params[1], 'period')
FROM system.users WHERE name IN ('totuser', 'user2')
ORDER BY 1
""",
user="totuser",
password=generate_totp(**totuser_secret),
).splitlines()
assert resp[0].startswith("totuser\tone_time_password\tSHA1\t9\t10"), resp
assert resp[1].startswith("user2\tone_time_password"), resp
# check that old password invalidated
elapsed = int(time.time() - old_password_created)
for _ in range(20 - elapsed):
time.sleep(1)
print(".", end="", flush=True)
print()
assert "AUTHENTICATION_FAILED" in node.query_and_get_error(
query_text, user="totuser", password=old_password
)