Merge branch 'master' into intdiv-ubsan

This commit is contained in:
Alexey Milovidov 2021-02-01 07:46:40 +03:00
commit 1c0c3b3aab
50 changed files with 1391 additions and 366 deletions

2
contrib/hyperscan vendored

@ -1 +1 @@
Subproject commit 3907fd00ee8b2538739768fa9533f8635a276531
Subproject commit e9f08df0213fc637aac0a5bbde9beeaeba2fe9fa

2
contrib/poco vendored

@ -1 +1 @@
Subproject commit 2c32e17c7dfee1f8bf24227b697cdef5fddf0823
Subproject commit e11f3c971570cf6a31006cd21cadf41a259c360a

View File

@ -251,8 +251,12 @@ function run_tests
00701_rollup
00834_cancel_http_readonly_queries_on_client_close
00911_tautological_compare
# Hyperscan
00926_multimatch
00929_multi_match_edit_distance
01681_hyperscan_debug_assertion
01031_mutations_interpreter_and_context
01053_ssd_dictionary # this test mistakenly requires acces to /var/lib/clickhouse -- can't run this locally, disabled
01083_expressions_in_engine_arguments

View File

@ -115,9 +115,20 @@ LIMIT 10
## IPv6StringToNum(s) {#ipv6stringtonums}
The reverse function of IPv6NumToString. If the IPv6 address has an invalid format, it returns a string of null bytes.
The reverse function of IPv6NumToString. If the IPv6 address has an invalid format, it returns a string of null bytes.
If the IP address is a valid IPv4 address then the IPv6 equivalent of the IPv4 address is returned.
HEX can be uppercase or lowercase.
``` sql
SELECT cutIPv6(IPv6StringToNum('127.0.0.1'), 0, 0);
```
``` text
┌─cutIPv6(IPv6StringToNum('127.0.0.1'), 0, 0)─┐
│ ::ffff:127.0.0.1 │
└─────────────────────────────────────────────┘
```
## IPv4ToIPv6(x) {#ipv4toipv6x}
Takes a `UInt32` number. Interprets it as an IPv4 address in [big endian](https://en.wikipedia.org/wiki/Endianness). Returns a `FixedString(16)` value containing the IPv6 address in binary format. Examples:
@ -214,6 +225,7 @@ SELECT
## toIPv6(string) {#toipv6string}
An alias to `IPv6StringToNum()` that takes a string form of IPv6 address and returns value of [IPv6](../../sql-reference/data-types/domains/ipv6.md) type, which is binary equal to value returned by `IPv6StringToNum()`.
If the IP address is a valid IPv4 address then the IPv6 equivalent of the IPv4 address is returned.
``` sql
WITH
@ -243,6 +255,15 @@ SELECT
└───────────────────────────────────┴──────────────────────────────────┘
```
``` sql
SELECT toIPv6('127.0.0.1')
```
``` text
┌─toIPv6('127.0.0.1')─┐
│ ::ffff:127.0.0.1 │
└─────────────────────┘
```
## isIPv4String

View File

@ -62,12 +62,12 @@ public:
bool randomize_, size_t max_iterations_, double max_time_,
const String & json_path_, size_t confidence_,
const String & query_id_, const String & query_to_execute_, bool continue_on_errors_,
bool print_stacktrace_, const Settings & settings_)
bool reconnect_, bool print_stacktrace_, const Settings & settings_)
:
concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_),
cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_),
json_path(json_path_), confidence(confidence_), query_id(query_id_),
query_to_execute(query_to_execute_), continue_on_errors(continue_on_errors_),
query_to_execute(query_to_execute_), continue_on_errors(continue_on_errors_), reconnect(reconnect_),
print_stacktrace(print_stacktrace_), settings(settings_),
shared_context(Context::createShared()), global_context(Context::createGlobal(shared_context.get())),
pool(concurrency)
@ -155,6 +155,7 @@ private:
String query_id;
String query_to_execute;
bool continue_on_errors;
bool reconnect;
bool print_stacktrace;
const Settings & settings;
SharedContextHolder shared_context;
@ -404,9 +405,14 @@ private:
void execute(EntryPtrs & connection_entries, Query & query, size_t connection_index)
{
Stopwatch watch;
Connection & connection = **connection_entries[connection_index];
if (reconnect)
connection.disconnect();
RemoteBlockInputStream stream(
*(*connection_entries[connection_index]),
query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage);
connection, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage);
if (!query_id.empty())
stream.setQueryId(query_id);
@ -589,6 +595,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
("confidence", value<size_t>()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)")
("query_id", value<std::string>()->default_value(""), "")
("continue_on_errors", "continue testing even if a query fails")
("reconnect", "establish new connection for every query")
;
Settings settings;
@ -638,7 +645,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
options["confidence"].as<size_t>(),
options["query_id"].as<std::string>(),
options["query"].as<std::string>(),
options.count("continue_on_errors") > 0,
options.count("continue_on_errors"),
options.count("reconnect"),
print_stacktrace,
settings);
return benchmark.run();

View File

@ -513,7 +513,7 @@ private:
}
protected:
void extractColumns(const IColumn ** columns, const IColumn ** aggr_columns) const
ssize_t extractColumns(const IColumn ** columns, const IColumn ** aggr_columns, ssize_t if_argument_pos) const
{
if (tuple_argument)
{
@ -526,6 +526,13 @@ protected:
for (size_t i = 0; i < args_count; ++i)
columns[i] = aggr_columns[i];
}
if (if_argument_pos >= 0)
{
columns[args_count] = aggr_columns[if_argument_pos];
return args_count;
}
else
return -1;
}
bool tuple_argument;
@ -551,8 +558,8 @@ public:
Arena * arena,
ssize_t if_argument_pos = -1) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatch(batch_size, places, place_offset, ex_columns, arena, if_argument_pos);
}
@ -560,8 +567,8 @@ public:
void addBatchSinglePlace(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatchSinglePlace(batch_size, place, ex_columns, arena, if_argument_pos);
}
@ -574,8 +581,8 @@ public:
Arena * arena,
ssize_t if_argument_pos = -1) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatchSinglePlaceNotNull(batch_size, place, ex_columns, null_map, arena, if_argument_pos);
}
@ -584,8 +591,8 @@ public:
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1)
const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
const IColumn * ex_columns[args_count + (if_argument_pos >= 0)];
if_argument_pos = extractColumns(ex_columns, columns, if_argument_pos);
Base::addBatchSinglePlaceFromInterval(batch_begin, batch_end, place, ex_columns, arena, if_argument_pos);
}
@ -595,7 +602,7 @@ public:
const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
extractColumns(ex_columns, columns, -1);
Base::addBatchArray(batch_size, places, place_offset, ex_columns, offsets, arena);
}
@ -610,7 +617,7 @@ public:
Arena * arena) const override
{
const IColumn * ex_columns[args_count];
extractColumns(ex_columns, columns);
extractColumns(ex_columns, columns, -1);
Base::addBatchLookupTable8(batch_size, map, place_offset, init, key, ex_columns, arena);
}

View File

@ -98,14 +98,31 @@ public:
}
else
{
const auto first_u32 = UTF8::convert(needle);
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
auto first_u32 = UTF8::convertUTF8ToCodePoint(needle, needle_size);
/// Invalid UTF-8
if (!first_u32)
{
/// Process it verbatim as a sequence of bytes.
size_t src_len = UTF8::seqLength(*needle);
memcpy(l_seq, needle, src_len);
memcpy(u_seq, needle, src_len);
}
else
{
uint32_t first_l_u32 = Poco::Unicode::toLower(*first_u32);
uint32_t first_u_u32 = Poco::Unicode::toUpper(*first_u32);
/// lower and uppercase variants of the first octet of the first character in `needle`
size_t length_l = UTF8::convertCodePointToUTF8(first_l_u32, l_seq, sizeof(l_seq));
size_t length_r = UTF8::convertCodePointToUTF8(first_u_u32, u_seq, sizeof(u_seq));
if (length_l != length_r)
throw Exception{"UTF8 sequences with different lowercase and uppercase lengths are not supported", ErrorCodes::UNSUPPORTED_PARAMETER};
}
/// lower and uppercase variants of the first octet of the first character in `needle`
UTF8::convert(first_l_u32, l_seq, sizeof(l_seq));
l = l_seq[0];
UTF8::convert(first_u_u32, u_seq, sizeof(u_seq));
u = u_seq[0];
}
@ -128,18 +145,21 @@ public:
continue;
}
const auto src_len = UTF8::seqLength(*needle_pos);
const auto c_u32 = UTF8::convert(needle_pos);
size_t src_len = std::min<size_t>(needle_end - needle_pos, UTF8::seqLength(*needle_pos));
auto c_u32 = UTF8::convertUTF8ToCodePoint(needle_pos, src_len);
const auto c_l_u32 = Poco::Unicode::toLower(c_u32);
const auto c_u_u32 = Poco::Unicode::toUpper(c_u32);
if (c_u32)
{
int c_l_u32 = Poco::Unicode::toLower(*c_u32);
int c_u_u32 = Poco::Unicode::toUpper(*c_u32);
const auto dst_l_len = static_cast<uint8_t>(UTF8::convert(c_l_u32, l_seq, sizeof(l_seq)));
const auto dst_u_len = static_cast<uint8_t>(UTF8::convert(c_u_u32, u_seq, sizeof(u_seq)));
uint8_t dst_l_len = static_cast<uint8_t>(UTF8::convertCodePointToUTF8(c_l_u32, l_seq, sizeof(l_seq)));
uint8_t dst_u_len = static_cast<uint8_t>(UTF8::convertCodePointToUTF8(c_u_u32, u_seq, sizeof(u_seq)));
/// @note Unicode standard states it is a rare but possible occasion
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
throw Exception{"UTF8 sequences with different lowercase and uppercase lengths are not supported", ErrorCodes::UNSUPPORTED_PARAMETER};
/// @note Unicode standard states it is a rare but possible occasion
if (!(dst_l_len == dst_u_len && dst_u_len == src_len))
throw Exception{"UTF8 sequences with different lowercase and uppercase lengths are not supported", ErrorCodes::UNSUPPORTED_PARAMETER};
}
cache_actual_len += src_len;
if (cache_actual_len < n)
@ -164,7 +184,7 @@ public:
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
ALWAYS_INLINE bool compare(const CharT * /*haystack*/, const CharT * /*haystack_end*/, const CharT * pos) const
ALWAYS_INLINE bool compare(const CharT * /*haystack*/, const CharT * haystack_end, const CharT * pos) const
{
#ifdef __SSE4_1__
@ -183,11 +203,20 @@ public:
pos += cache_valid_len;
auto needle_pos = needle + cache_valid_len;
while (needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (needle_pos < needle_end)
{
/// @note assuming sequences for lowercase and uppercase have exact same length
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(pos, haystack_end - pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
/// @note assuming sequences for lowercase and uppercase have exact same length (that is not always true)
const auto len = UTF8::seqLength(*pos);
pos += len;
needle_pos += len;
@ -209,10 +238,19 @@ public:
pos += first_needle_symbol_is_ascii;
auto needle_pos = needle + first_needle_symbol_is_ascii;
while (needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (needle_pos < needle_end)
{
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(pos, haystack_end - pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
const auto len = UTF8::seqLength(*pos);
pos += len;
needle_pos += len;
@ -270,11 +308,20 @@ public:
auto haystack_pos = haystack + cache_valid_len;
auto needle_pos = needle + cache_valid_len;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(haystack_pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (haystack_pos < haystack_end && needle_pos < needle_end)
{
/// @note assuming sequences for lowercase and uppercase have exact same length
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
/// @note assuming sequences for lowercase and uppercase have exact same length (that is not always true)
const auto len = UTF8::seqLength(*haystack_pos);
haystack_pos += len;
needle_pos += len;
@ -302,10 +349,19 @@ public:
auto haystack_pos = haystack + first_needle_symbol_is_ascii;
auto needle_pos = needle + first_needle_symbol_is_ascii;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
Poco::Unicode::toLower(UTF8::convert(haystack_pos)) ==
Poco::Unicode::toLower(UTF8::convert(needle_pos)))
while (haystack_pos < haystack_end && needle_pos < needle_end)
{
auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos);
auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos);
/// Invalid UTF-8, should not compare equals
if (!haystack_code_point || !needle_code_point)
break;
/// Not equals case insensitive.
if (Poco::Unicode::toLower(*haystack_code_point) != Poco::Unicode::toLower(*needle_code_point))
break;
const auto len = UTF8::seqLength(*haystack_pos);
haystack_pos += len;
needle_pos += len;

View File

@ -1,5 +1,6 @@
#pragma once
#include <optional>
#include <common/types.h>
#include <Common/BitHelpers.h>
#include <Poco/UTF8Encoding.h>
@ -73,26 +74,27 @@ inline size_t countCodePoints(const UInt8 * data, size_t size)
return res;
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
int convert(const CharT * bytes)
size_t convertCodePointToUTF8(uint32_t code_point, CharT * out_bytes, size_t out_length)
{
static const Poco::UTF8Encoding utf8;
return utf8.convert(reinterpret_cast<const uint8_t *>(bytes));
int res = utf8.convert(code_point, reinterpret_cast<uint8_t *>(out_bytes), out_length);
assert(res >= 0);
return res;
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
int convert(int ch, CharT * bytes, int length)
std::optional<uint32_t> convertUTF8ToCodePoint(const CharT * in_bytes, size_t in_length)
{
static const Poco::UTF8Encoding utf8;
return utf8.convert(ch, reinterpret_cast<uint8_t *>(bytes), length);
int res = utf8.queryConvert(reinterpret_cast<const uint8_t *>(in_bytes), in_length);
if (res >= 0)
return res;
return {};
}
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
int queryConvert(const CharT * bytes, int length)
{
static const Poco::UTF8Encoding utf8;
return utf8.queryConvert(reinterpret_cast<const uint8_t *>(bytes), length);
}
/// returns UTF-8 wcswidth. Invalid sequence is treated as zero width character.
/// `prefix` is used to compute the `\t` width which extends the string before

View File

@ -60,7 +60,7 @@ namespace VolnitskyTraits
static inline Ngram toNGram(const UInt8 * const pos) { return unalignedLoad<Ngram>(pos); }
template <typename Callback>
static inline void putNGramASCIICaseInsensitive(const UInt8 * const pos, const int offset, const Callback & putNGramBase)
static inline void putNGramASCIICaseInsensitive(const UInt8 * pos, int offset, Callback && putNGramBase)
{
struct Chars
{
@ -109,199 +109,234 @@ namespace VolnitskyTraits
putNGramBase(n, offset);
}
template <bool CaseSensitive, bool ASCII, typename Callback>
static inline void putNGram(const UInt8 * const pos, const int offset, [[maybe_unused]] const UInt8 * const begin, const Callback & putNGramBase)
template <typename Callback>
static inline void putNGramUTF8CaseInsensitive(
const UInt8 * pos, int offset, const UInt8 * begin, size_t size, Callback && putNGramBase)
{
if constexpr (CaseSensitive)
const UInt8 * end = begin + size;
struct Chars
{
putNGramBase(toNGram(pos), offset);
UInt8 c0;
UInt8 c1;
};
union
{
VolnitskyTraits::Ngram n;
Chars chars;
};
n = toNGram(pos);
if (isascii(chars.c0) && isascii(chars.c1))
{
putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
}
else
{
if constexpr (ASCII)
/** n-gram (in the case of n = 2)
* can be entirely located within one code point,
* or intersect with two code points.
*
* In the first case, you need to consider up to two alternatives - this code point in upper and lower case,
* and in the second case - up to four alternatives - fragments of two code points in all combinations of cases.
*
* It does not take into account the dependence of the case-transformation from the locale (for example - Turkish `Ii`)
* as well as composition / decomposition and other features.
*
* It also does not work if characters with lower and upper cases are represented by different number of bytes or code points.
*/
using Seq = UInt8[6];
if (UTF8::isContinuationOctet(chars.c1))
{
putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
}
else
{
struct Chars
/// ngram is inside a sequence
auto seq_pos = pos;
UTF8::syncBackward(seq_pos, begin);
auto u32 = UTF8::convertUTF8ToCodePoint(seq_pos, end - seq_pos);
/// Invalid UTF-8
if (!u32)
{
UInt8 c0;
UInt8 c1;
};
union
{
VolnitskyTraits::Ngram n;
Chars chars;
};
n = toNGram(pos);
if (isascii(chars.c0) && isascii(chars.c1))
putNGramASCIICaseInsensitive(pos, offset, putNGramBase);
putNGramBase(n, offset);
}
else
{
/** n-gram (in the case of n = 2)
* can be entirely located within one code point,
* or intersect with two code points.
*
* In the first case, you need to consider up to two alternatives - this code point in upper and lower case,
* and in the second case - up to four alternatives - fragments of two code points in all combinations of cases.
*
* It does not take into account the dependence of the case-transformation from the locale (for example - Turkish `Ii`)
* as well as composition / decomposition and other features.
*
* It also does not work if characters with lower and upper cases are represented by different number of bytes or code points.
*/
int l_u32 = Poco::Unicode::toLower(*u32);
int u_u32 = Poco::Unicode::toUpper(*u32);
using Seq = UInt8[6];
if (UTF8::isContinuationOctet(chars.c1))
/// symbol is case-independent
if (l_u32 == u_u32)
{
/// ngram is inside a sequence
auto seq_pos = pos;
UTF8::syncBackward(seq_pos, begin);
const auto u32 = UTF8::convert(seq_pos);
const auto l_u32 = Poco::Unicode::toLower(u32);
const auto u_u32 = Poco::Unicode::toUpper(u32);
/// symbol is case-independent
if (l_u32 == u_u32)
putNGramBase(n, offset);
else
{
/// where is the given ngram in respect to the start of UTF-8 sequence?
const auto seq_ngram_offset = pos - seq_pos;
Seq seq;
/// put ngram for lowercase
UTF8::convert(l_u32, seq, sizeof(seq));
chars.c0 = seq[seq_ngram_offset];
chars.c1 = seq[seq_ngram_offset + 1];
putNGramBase(n, offset);
/// put ngram for uppercase
UTF8::convert(u_u32, seq, sizeof(seq));
chars.c0 = seq[seq_ngram_offset]; //-V519
chars.c1 = seq[seq_ngram_offset + 1]; //-V519
putNGramBase(n, offset);
}
putNGramBase(n, offset);
}
else
{
/// ngram is on the boundary of two sequences
/// first sequence may start before u_pos if it is not ASCII
auto first_seq_pos = pos;
UTF8::syncBackward(first_seq_pos, begin);
/// where is the given ngram in respect to the start of first UTF-8 sequence?
const auto seq_ngram_offset = pos - first_seq_pos;
/// where is the given ngram in respect to the start of UTF-8 sequence?
size_t seq_ngram_offset = pos - seq_pos;
const auto first_u32 = UTF8::convert(first_seq_pos);
const auto first_l_u32 = Poco::Unicode::toLower(first_u32);
const auto first_u_u32 = Poco::Unicode::toUpper(first_u32);
Seq seq;
/// second sequence always start immediately after u_pos
auto second_seq_pos = pos + 1;
/// put ngram for lowercase
size_t length_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(l_u32, seq, sizeof(seq));
assert(length_l >= 2);
chars.c0 = seq[seq_ngram_offset];
chars.c1 = seq[seq_ngram_offset + 1];
putNGramBase(n, offset);
const auto second_u32 = UTF8::convert(second_seq_pos); /// TODO This assumes valid UTF-8 or zero byte after needle.
const auto second_l_u32 = Poco::Unicode::toLower(second_u32);
const auto second_u_u32 = Poco::Unicode::toUpper(second_u32);
/// put ngram for uppercase
size_t length_r [[maybe_unused]] = UTF8::convertCodePointToUTF8(u_u32, seq, sizeof(seq));
assert(length_r >= 2);
chars.c0 = seq[seq_ngram_offset]; //-V519
chars.c1 = seq[seq_ngram_offset + 1]; //-V519
putNGramBase(n, offset);
}
}
}
else
{
/// ngram is on the boundary of two sequences
/// first sequence may start before u_pos if it is not ASCII
auto first_seq_pos = pos;
UTF8::syncBackward(first_seq_pos, begin);
/// where is the given ngram in respect to the start of first UTF-8 sequence?
size_t seq_ngram_offset = pos - first_seq_pos;
/// both symbols are case-independent
if (first_l_u32 == first_u_u32 && second_l_u32 == second_u_u32)
{
putNGramBase(n, offset);
}
else if (first_l_u32 == first_u_u32)
{
/// first symbol is case-independent
Seq seq;
auto first_u32 = UTF8::convertUTF8ToCodePoint(first_seq_pos, end - first_seq_pos);
int first_l_u32 = 0;
int first_u_u32 = 0;
/// put ngram for lowercase
UTF8::convert(second_l_u32, seq, sizeof(seq));
chars.c1 = seq[0];
putNGramBase(n, offset);
if (first_u32)
{
first_l_u32 = Poco::Unicode::toLower(*first_u32);
first_u_u32 = Poco::Unicode::toUpper(*first_u32);
}
/// put ngram from uppercase, if it is different
UTF8::convert(second_u_u32, seq, sizeof(seq));
if (chars.c1 != seq[0])
{
chars.c1 = seq[0];
putNGramBase(n, offset);
}
}
else if (second_l_u32 == second_u_u32)
{
/// second symbol is case-independent
Seq seq;
/// second sequence always start immediately after u_pos
auto second_seq_pos = pos + 1;
/// put ngram for lowercase
UTF8::convert(first_l_u32, seq, sizeof(seq));
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
auto second_u32 = UTF8::convertUTF8ToCodePoint(second_seq_pos, end - second_seq_pos);
int second_l_u32 = 0;
int second_u_u32 = 0;
/// put ngram for uppercase, if it is different
UTF8::convert(first_u_u32, seq, sizeof(seq));
if (chars.c0 != seq[seq_ngram_offset])
{
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
}
}
else
{
Seq first_l_seq;
Seq first_u_seq;
Seq second_l_seq;
Seq second_u_seq;
if (second_u32)
{
second_l_u32 = Poco::Unicode::toLower(*second_u32);
second_u_u32 = Poco::Unicode::toUpper(*second_u32);
}
UTF8::convert(first_l_u32, first_l_seq, sizeof(first_l_seq));
UTF8::convert(first_u_u32, first_u_seq, sizeof(first_u_seq));
UTF8::convert(second_l_u32, second_l_seq, sizeof(second_l_seq));
UTF8::convert(second_u_u32, second_u_seq, sizeof(second_u_seq));
/// both symbols are case-independent
if (first_l_u32 == first_u_u32 && second_l_u32 == second_u_u32)
{
putNGramBase(n, offset);
}
else if (first_l_u32 == first_u_u32)
{
/// first symbol is case-independent
Seq seq;
auto c0l = first_l_seq[seq_ngram_offset];
auto c0u = first_u_seq[seq_ngram_offset];
auto c1l = second_l_seq[0];
auto c1u = second_u_seq[0];
/// put ngram for lowercase
size_t size_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_l_u32, seq, sizeof(seq));
assert(size_l >= 1);
chars.c1 = seq[0];
putNGramBase(n, offset);
/// ngram for ll
chars.c0 = c0l;
chars.c1 = c1l;
putNGramBase(n, offset);
/// put ngram from uppercase, if it is different
size_t size_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_u_u32, seq, sizeof(seq));
assert(size_u >= 1);
if (chars.c1 != seq[0])
{
chars.c1 = seq[0];
putNGramBase(n, offset);
}
}
else if (second_l_u32 == second_u_u32)
{
/// second symbol is case-independent
Seq seq;
if (c0l != c0u)
{
/// ngram for Ul
chars.c0 = c0u;
chars.c1 = c1l;
putNGramBase(n, offset);
}
/// put ngram for lowercase
size_t size_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_l_u32, seq, sizeof(seq));
assert(size_l > seq_ngram_offset);
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
if (c1l != c1u)
{
/// ngram for lU
chars.c0 = c0l;
chars.c1 = c1u;
putNGramBase(n, offset);
}
/// put ngram for uppercase, if it is different
size_t size_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_u_u32, seq, sizeof(seq));
assert(size_u > seq_ngram_offset);
if (chars.c0 != seq[seq_ngram_offset])
{
chars.c0 = seq[seq_ngram_offset];
putNGramBase(n, offset);
}
}
else
{
Seq first_l_seq;
Seq first_u_seq;
Seq second_l_seq;
Seq second_u_seq;
if (c0l != c0u && c1l != c1u)
{
/// ngram for UU
chars.c0 = c0u;
chars.c1 = c1u;
putNGramBase(n, offset);
}
}
size_t size_first_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_l_u32, first_l_seq, sizeof(first_l_seq));
size_t size_first_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(first_u_u32, first_u_seq, sizeof(first_u_seq));
size_t size_second_l [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_l_u32, second_l_seq, sizeof(second_l_seq));
size_t size_second_u [[maybe_unused]] = UTF8::convertCodePointToUTF8(second_u_u32, second_u_seq, sizeof(second_u_seq));
assert(size_first_l > seq_ngram_offset);
assert(size_first_u > seq_ngram_offset);
assert(size_second_l > 0);
assert(size_second_u > 0);
auto c0l = first_l_seq[seq_ngram_offset];
auto c0u = first_u_seq[seq_ngram_offset];
auto c1l = second_l_seq[0];
auto c1u = second_u_seq[0];
/// ngram for ll
chars.c0 = c0l;
chars.c1 = c1l;
putNGramBase(n, offset);
if (c0l != c0u)
{
/// ngram for Ul
chars.c0 = c0u;
chars.c1 = c1l;
putNGramBase(n, offset);
}
if (c1l != c1u)
{
/// ngram for lU
chars.c0 = c0l;
chars.c1 = c1u;
putNGramBase(n, offset);
}
if (c0l != c0u && c1l != c1u)
{
/// ngram for UU
chars.c0 = c0u;
chars.c1 = c1u;
putNGramBase(n, offset);
}
}
}
}
}
template <bool CaseSensitive, bool ASCII, typename Callback>
static inline void putNGram(const UInt8 * pos, int offset, [[maybe_unused]] const UInt8 * begin, size_t size, Callback && putNGramBase)
{
if constexpr (CaseSensitive)
putNGramBase(toNGram(pos), offset);
else if constexpr (ASCII)
putNGramASCIICaseInsensitive(pos, offset, std::forward<Callback>(putNGramBase));
else
putNGramUTF8CaseInsensitive(pos, offset, begin, size, std::forward<Callback>(putNGramBase));
}
}
@ -310,17 +345,17 @@ template <bool CaseSensitive, bool ASCII, typename FallbackSearcher>
class VolnitskyBase
{
protected:
const UInt8 * const needle;
const size_t needle_size;
const UInt8 * const needle_end = needle + needle_size;
const UInt8 * needle;
size_t needle_size;
const UInt8 * needle_end = needle + needle_size;
/// For how long we move, if the n-gram from haystack is not found in the hash table.
const size_t step = needle_size - sizeof(VolnitskyTraits::Ngram) + 1;
size_t step = needle_size - sizeof(VolnitskyTraits::Ngram) + 1;
/** max needle length is 255, max distinct ngrams for case-sensitive is (255 - 1), case-insensitive is 4 * (255 - 1)
* storage of 64K ngrams (n = 2, 128 KB) should be large enough for both cases */
std::unique_ptr<VolnitskyTraits::Offset[]> hash; /// Hash table.
const bool fallback; /// Do we need to use the fallback algorithm.
bool fallback; /// Do we need to use the fallback algorithm.
FallbackSearcher fallback_searcher;
@ -346,7 +381,7 @@ public:
/// ssize_t is used here because unsigned can't be used with condition like `i >= 0`, unsigned always >= 0
/// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first.
for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
VolnitskyTraits::putNGram<CaseSensitive, ASCII>(this->needle + i, i + 1, this->needle, callback);
VolnitskyTraits::putNGram<CaseSensitive, ASCII>(needle + i, i + 1, needle, needle_size, callback);
}
@ -493,6 +528,7 @@ public:
reinterpret_cast<const UInt8 *>(cur_needle_data) + i,
i + 1,
reinterpret_cast<const UInt8 *>(cur_needle_data),
cur_needle_size,
callback);
}
}

View File

@ -36,6 +36,7 @@
#define DEFAULT_MERGE_BLOCK_SIZE 8192
#define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5
#define DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC 60
#define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160)
#define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15
#define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024

View File

@ -391,6 +391,7 @@ class IColumn;
M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \
M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \
M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \
M(Seconds, periodic_live_view_refresh, DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC, "Interval after which periodically refreshed live view is forced to refresh.", 0) \
M(Bool, transform_null_in, false, "If enabled, NULL values will be matched with 'IN' operator as if they are considered equal.", 0) \
M(Bool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \
M(Seconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \

View File

@ -68,12 +68,12 @@ struct AddSecondsImpl : public AddOnDateTime64DefaultImpl<AddSecondsImpl>
static constexpr auto name = "addSeconds";
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &)
{
return t + delta;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum(d)) + delta;
}
@ -92,7 +92,7 @@ struct AddMinutesImpl : public AddOnDateTime64DefaultImpl<AddMinutesImpl>
return t + delta * 60;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum(d)) + delta * 60;
}
@ -111,7 +111,7 @@ struct AddHoursImpl : public AddOnDateTime64DefaultImpl<AddHoursImpl>
return t + delta * 3600;
}
static inline UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.fromDayNum(DayNum(d)) + delta * 3600;
}
@ -125,18 +125,12 @@ struct AddDaysImpl : public AddOnDateTime64DefaultImpl<AddDaysImpl>
static constexpr auto name = "addDays";
// static inline UInt32 execute(UInt64 t, Int64 delta, const DateLUTImpl & time_zone)
// {
// // TODO (nemkov): LUT does not support out-of range date values for now.
// return time_zone.addDays(t, delta);
// }
static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone)
{
return time_zone.addDays(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
{
return d + delta;
}
@ -155,7 +149,7 @@ struct AddWeeksImpl : public AddOnDateTime64DefaultImpl<AddWeeksImpl>
return time_zone.addWeeks(t, delta);
}
static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
static inline NO_SANITIZE_UNDEFINED UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl &)
{
return d + delta * 7;
}

View File

@ -263,6 +263,12 @@ public:
static constexpr auto name = "IPv6StringToNum";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv6StringToNum>(); }
static inline bool tryParseIPv4(const char * pos)
{
UInt32 result = 0;
return DB::parseIPv4(pos, reinterpret_cast<unsigned char *>(&result));
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
@ -270,8 +276,8 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeFixedString>(IPV6_BINARY_LENGTH);
}
@ -292,13 +298,27 @@ public:
const ColumnString::Chars & vec_src = col_in->getChars();
const ColumnString::Offsets & offsets_src = col_in->getOffsets();
size_t src_offset = 0;
char src_ipv4_buf[sizeof("::ffff:") + IPV4_MAX_TEXT_LENGTH + 1] = "::ffff:";
for (size_t out_offset = 0, i = 0;
out_offset < vec_res.size();
out_offset += IPV6_BINARY_LENGTH, ++i)
for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += IPV6_BINARY_LENGTH, ++i)
{
/// In case of failure, the function fills vec_res with zero bytes.
parseIPv6(reinterpret_cast<const char *>(&vec_src[src_offset]), reinterpret_cast<unsigned char *>(&vec_res[out_offset]));
/// For both cases below: In case of failure, the function parseIPv6 fills vec_res with zero bytes.
/// If the source IP address is parsable as an IPv4 address, then transform it into a valid IPv6 address.
/// Keeping it simple by just prefixing `::ffff:` to the IPv4 address to represent it as a valid IPv6 address.
if (tryParseIPv4(reinterpret_cast<const char *>(&vec_src[src_offset])))
{
std::memcpy(
src_ipv4_buf + std::strlen("::ffff:"),
reinterpret_cast<const char *>(&vec_src[src_offset]),
std::min<UInt64>(offsets_src[i] - src_offset, IPV4_MAX_TEXT_LENGTH + 1));
parseIPv6(src_ipv4_buf, reinterpret_cast<unsigned char *>(&vec_res[out_offset]));
}
else
{
parseIPv6(
reinterpret_cast<const char *>(&vec_src[src_offset]), reinterpret_cast<unsigned char *>(&vec_res[out_offset]));
}
src_offset = offsets_src[i];
}

View File

@ -465,7 +465,7 @@ std::vector<size_t> buildKMPPrefixFunction(const SliceType & pattern, const Equa
for (size_t i = 1; i < pattern.size; ++i)
{
result[i] = 0;
for (auto length = i; length > 0;)
for (size_t length = i; length > 0;)
{
length = result[length - 1];
if (isEqualFunc(pattern, i, length))
@ -695,7 +695,7 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
if (size >= 0)
{
auto length = static_cast<size_t>(size);
size_t length = static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
@ -711,7 +711,7 @@ void resizeDynamicSize(ArraySource && array_source, ValueSource && value_source,
}
else
{
auto length = static_cast<size_t>(-size);
size_t length = -static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
@ -744,7 +744,7 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source
if (size >= 0)
{
auto length = static_cast<size_t>(size);
size_t length = static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);
@ -760,7 +760,7 @@ void resizeConstantSize(ArraySource && array_source, ValueSource && value_source
}
else
{
auto length = static_cast<size_t>(-size);
size_t length = -static_cast<size_t>(size);
if (length > MAX_ARRAY_SIZE)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size: {}, maximum: {}",
length, MAX_ARRAY_SIZE);

View File

@ -135,15 +135,16 @@ struct LowerUpperUTF8Impl
{
static const Poco::UTF8Encoding utf8;
int src_sequence_length = UTF8::seqLength(*src);
size_t src_sequence_length = UTF8::seqLength(*src);
int src_code_point = UTF8::queryConvert(src, src_end - src);
if (src_code_point > 0)
auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src);
if (src_code_point)
{
int dst_code_point = to_case(src_code_point);
int dst_code_point = to_case(*src_code_point);
if (dst_code_point > 0)
{
int dst_sequence_length = UTF8::convert(dst_code_point, dst, src_end - src);
size_t dst_sequence_length = UTF8::convertCodePointToUTF8(dst_code_point, dst, src_end - src);
assert(dst_sequence_length <= 4);
/// We don't support cases when lowercase and uppercase characters occupy different number of bytes in UTF-8.
/// As an example, this happens for ß and ẞ.
@ -156,7 +157,9 @@ struct LowerUpperUTF8Impl
}
}
*dst++ = *src++;
*dst = *src;
++dst;
++src;
}
}

View File

@ -119,8 +119,13 @@ public:
UInt32 code_point2 = generate_code_point(rand >> 32);
/// We have padding in column buffers that we can overwrite.
pos += UTF8::convert(code_point1, pos, sizeof(int));
last_writen_bytes = UTF8::convert(code_point2, pos, sizeof(int));
size_t length1 = UTF8::convertCodePointToUTF8(code_point1, pos, sizeof(int));
assert(length1 <= 4);
pos += length1;
size_t length2 = UTF8::convertCodePointToUTF8(code_point2, pos, sizeof(int));
assert(length2 <= 4);
last_writen_bytes = length2;
pos += last_writen_bytes;
}
offset = pos - data_to.data() + 1;

View File

@ -269,6 +269,18 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (live_view_timeout)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH TIMEOUT " << (settings.hilite ? hilite_none : "")
<< *live_view_timeout;
if (live_view_periodic_refresh)
{
if (live_view_timeout)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AND" << (settings.hilite ? hilite_none : "");
else
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH" << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " PERIODIC REFRESH " << (settings.hilite ? hilite_none : "")
<< *live_view_periodic_refresh;
}
formatOnCluster(settings);
}
else

View File

@ -77,6 +77,8 @@ public:
ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.)
std::optional<UInt64> live_view_timeout; /// For CREATE LIVE VIEW ... WITH TIMEOUT ...
std::optional<UInt64> live_view_periodic_refresh; /// For CREATE LIVE VIEW ... WITH [PERIODIC] REFRESH ...
bool attach_short_syntax{false};
std::optional<String> attach_from_path = std::nullopt;

View File

@ -569,10 +569,14 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
ASTPtr as_table;
ASTPtr select;
ASTPtr live_view_timeout;
ASTPtr live_view_periodic_refresh;
String cluster_str;
bool attach = false;
bool if_not_exists = false;
bool with_and = false;
bool with_timeout = false;
bool with_periodic_refresh = false;
if (!s_create.ignore(pos, expected))
{
@ -594,10 +598,35 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
if (!table_name_p.parse(pos, table, expected))
return false;
if (ParserKeyword{"WITH TIMEOUT"}.ignore(pos, expected))
if (ParserKeyword{"WITH"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_timeout, expected))
live_view_timeout = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC));
if (ParserKeyword{"TIMEOUT"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_timeout, expected))
{
live_view_timeout = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC));
}
/// Optional - AND
if (ParserKeyword{"AND"}.ignore(pos, expected))
with_and = true;
with_timeout = true;
}
if (ParserKeyword{"REFRESH"}.ignore(pos, expected) || ParserKeyword{"PERIODIC REFRESH"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_periodic_refresh, expected))
live_view_periodic_refresh = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC));
with_periodic_refresh = true;
}
else if (with_and)
return false;
if (!with_timeout && !with_periodic_refresh)
return false;
}
if (ParserKeyword{"ON"}.ignore(pos, expected))
@ -656,6 +685,9 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
if (live_view_timeout)
query->live_view_timeout.emplace(live_view_timeout->as<ASTLiteral &>().value.safeGet<UInt64>());
if (live_view_periodic_refresh)
query->live_view_periodic_refresh.emplace(live_view_periodic_refresh->as<ASTLiteral &>().value.safeGet<UInt64>());
return true;
}

View File

@ -34,6 +34,7 @@ public:
{
new_blocks_metadata->hash = key_str;
new_blocks_metadata->version = storage.getBlocksVersion() + 1;
new_blocks_metadata->time = std::chrono::system_clock::now();
for (auto & block : *new_blocks)
{
@ -48,6 +49,15 @@ public:
storage.condition.notify_all();
}
else
{
// only update blocks time
new_blocks_metadata->hash = storage.getBlocksHashKey();
new_blocks_metadata->version = storage.getBlocksVersion();
new_blocks_metadata->time = std::chrono::system_clock::now();
(*storage.blocks_metadata_ptr) = new_blocks_metadata;
}
new_blocks.reset();
new_blocks_metadata.reset();

View File

@ -20,6 +20,7 @@ limitations under the License. */
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/SquashingBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <common/logger_useful.h>
#include <Common/typeid_cast.h>
#include <Common/SipHash.h>
@ -254,6 +255,8 @@ StorageLiveView::StorageLiveView(
live_view_context = std::make_unique<Context>(global_context);
live_view_context->makeQueryContext();
log = &Poco::Logger::get("StorageLiveView (" + table_id_.database_name + "." + table_id_.table_name + ")");
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
setInMemoryMetadata(storage_metadata);
@ -275,12 +278,21 @@ StorageLiveView::StorageLiveView(
if (query.live_view_timeout)
{
is_temporary = true;
temporary_live_view_timeout = std::chrono::seconds{*query.live_view_timeout};
temporary_live_view_timeout = Seconds {*query.live_view_timeout};
}
if (query.live_view_periodic_refresh)
{
is_periodically_refreshed = true;
periodic_live_view_refresh = Seconds {*query.live_view_periodic_refresh};
}
blocks_ptr = std::make_shared<BlocksPtr>();
blocks_metadata_ptr = std::make_shared<BlocksMetadataPtr>();
active_ptr = std::make_shared<bool>(true);
periodic_refresh_task = global_context.getSchedulePool().createTask("LieViewPeriodicRefreshTask", [this]{ periodicRefreshTaskFunc(); });
periodic_refresh_task->deactivate();
}
Block StorageLiveView::getHeader() const
@ -369,10 +381,21 @@ bool StorageLiveView::getNewBlocks()
}
new_blocks_metadata->hash = key.toHexString();
new_blocks_metadata->version = getBlocksVersion() + 1;
new_blocks_metadata->time = std::chrono::system_clock::now();
(*blocks_ptr) = new_blocks;
(*blocks_metadata_ptr) = new_blocks_metadata;
updated = true;
}
else
{
new_blocks_metadata->hash = getBlocksHashKey();
new_blocks_metadata->version = getBlocksVersion();
new_blocks_metadata->time = std::chrono::system_clock::now();
(*blocks_metadata_ptr) = new_blocks_metadata;
}
}
return updated;
}
@ -392,11 +415,18 @@ void StorageLiveView::startup()
{
if (is_temporary)
TemporaryLiveViewCleaner::instance().addView(std::static_pointer_cast<StorageLiveView>(shared_from_this()));
if (is_periodically_refreshed)
periodic_refresh_task->activate();
}
void StorageLiveView::shutdown()
{
shutdown_called = true;
if (is_periodically_refreshed)
periodic_refresh_task->deactivate();
DatabaseCatalog::instance().removeDependency(select_table_id, getStorageID());
}
@ -415,15 +445,55 @@ void StorageLiveView::drop()
condition.notify_all();
}
void StorageLiveView::refresh()
void StorageLiveView::scheduleNextPeriodicRefresh()
{
Seconds current_time = std::chrono::duration_cast<Seconds> (std::chrono::system_clock::now().time_since_epoch());
Seconds blocks_time = std::chrono::duration_cast<Seconds> (getBlocksTime().time_since_epoch());
if ((current_time - periodic_live_view_refresh) >= blocks_time)
{
refresh(false);
blocks_time = std::chrono::duration_cast<Seconds> (getBlocksTime().time_since_epoch());
}
current_time = std::chrono::duration_cast<Seconds> (std::chrono::system_clock::now().time_since_epoch());
auto next_refresh_time = blocks_time + periodic_live_view_refresh;
if (current_time >= next_refresh_time)
periodic_refresh_task->scheduleAfter(0);
else
{
auto schedule_time = std::chrono::duration_cast<MilliSeconds> (next_refresh_time - current_time);
periodic_refresh_task->scheduleAfter(static_cast<size_t>(schedule_time.count()));
}
}
void StorageLiveView::periodicRefreshTaskFunc()
{
LOG_TRACE(log, "periodic refresh task");
std::lock_guard lock(mutex);
if (hasActiveUsers())
scheduleNextPeriodicRefresh();
}
void StorageLiveView::refresh(bool grab_lock)
{
// Lock is already acquired exclusively from InterperterAlterQuery.cpp InterpreterAlterQuery::execute() method.
// So, reacquiring lock is not needed and will result in an exception.
if (grab_lock)
{
std::lock_guard lock(mutex);
if (getNewBlocks())
condition.notify_all();
}
else
{
if (getNewBlocks())
condition.notify_all();
}
}
Pipe StorageLiveView::read(
@ -435,15 +505,21 @@ Pipe StorageLiveView::read(
const size_t /*max_block_size*/,
const unsigned /*num_streams*/)
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
refresh(false);
else if (is_periodically_refreshed)
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
{
if (getNewBlocks())
condition.notify_all();
}
return Pipe(std::make_shared<BlocksSource>(blocks_ptr, getHeader()));
Seconds current_time = std::chrono::duration_cast<Seconds> (std::chrono::system_clock::now().time_since_epoch());
Seconds blocks_time = std::chrono::duration_cast<Seconds> (getBlocksTime().time_since_epoch());
if ((current_time - periodic_live_view_refresh) >= blocks_time)
refresh(false);
}
return Pipe(std::make_shared<BlocksSource>(blocks_ptr, getHeader()));
}
BlockInputStreams StorageLiveView::watch(
@ -458,6 +534,7 @@ BlockInputStreams StorageLiveView::watch(
bool has_limit = false;
UInt64 limit = 0;
BlockInputStreamPtr reader;
if (query.limit_length)
{
@ -466,45 +543,28 @@ BlockInputStreams StorageLiveView::watch(
}
if (query.is_watch_events)
{
auto reader = std::make_shared<LiveViewEventsBlockInputStream>(
reader = std::make_shared<LiveViewEventsBlockInputStream>(
std::static_pointer_cast<StorageLiveView>(shared_from_this()),
blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit,
context.getSettingsRef().live_view_heartbeat_interval.totalSeconds());
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
{
if (getNewBlocks())
condition.notify_all();
}
}
processed_stage = QueryProcessingStage::Complete;
return { reader };
}
else
{
auto reader = std::make_shared<LiveViewBlockInputStream>(
reader = std::make_shared<LiveViewBlockInputStream>(
std::static_pointer_cast<StorageLiveView>(shared_from_this()),
blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit,
context.getSettingsRef().live_view_heartbeat_interval.totalSeconds());
{
std::lock_guard lock(mutex);
if (!(*blocks_ptr))
{
if (getNewBlocks())
condition.notify_all();
}
}
{
std::lock_guard lock(mutex);
processed_stage = QueryProcessingStage::Complete;
if (!(*blocks_ptr))
refresh(false);
return { reader };
if (is_periodically_refreshed)
scheduleNextPeriodicRefresh();
}
processed_stage = QueryProcessingStage::Complete;
return { reader };
}
NamesAndTypesList StorageLiveView::getVirtuals() const

View File

@ -13,6 +13,7 @@ limitations under the License. */
#include <ext/shared_ptr_helper.h>
#include <Storages/IStorage.h>
#include <Core/BackgroundSchedulePool.h>
#include <mutex>
#include <condition_variable>
@ -21,10 +22,16 @@ limitations under the License. */
namespace DB
{
using Time = std::chrono::time_point<std::chrono::system_clock>;
using Seconds = std::chrono::seconds;
using MilliSeconds = std::chrono::milliseconds;
struct BlocksMetadata
{
String hash;
UInt64 version;
Time time;
};
struct MergeableBlocks
@ -75,8 +82,10 @@ public:
NamesAndTypesList getVirtuals() const override;
bool isTemporary() const { return is_temporary; }
std::chrono::seconds getTimeout() const { return temporary_live_view_timeout; }
bool isPeriodicallyRefreshed() const { return is_periodically_refreshed; }
Seconds getTimeout() const { return temporary_live_view_timeout; }
Seconds getPeriodicRefresh() const { return periodic_live_view_refresh; }
/// Check if we have any readers
/// must be called with mutex locked
@ -109,6 +118,15 @@ public:
return 0;
}
/// Get blocks time
/// must be called with mutex locked
Time getBlocksTime()
{
if (*blocks_metadata_ptr)
return (*blocks_metadata_ptr)->time;
return {};
}
/// Reset blocks
/// must be called with mutex locked
void reset()
@ -124,7 +142,7 @@ public:
void startup() override;
void shutdown() override;
void refresh();
void refresh(const bool grab_lock = true);
Pipe read(
const Names & column_names,
@ -176,8 +194,13 @@ private:
Context & global_context;
std::unique_ptr<Context> live_view_context;
Poco::Logger * log;
bool is_temporary = false;
std::chrono::seconds temporary_live_view_timeout;
bool is_periodically_refreshed = false;
Seconds temporary_live_view_timeout;
Seconds periodic_live_view_refresh;
/// Mutex to protect access to sample block and inner_blocks_query
mutable std::mutex sample_block_lock;
@ -199,6 +222,13 @@ private:
std::atomic<bool> shutdown_called = false;
/// Periodic refresh task used when [PERIODIC] REFRESH is specified in create statement
BackgroundSchedulePool::TaskHolder periodic_refresh_task;
void periodicRefreshTaskFunc();
/// Must be called with mutex locked
void scheduleNextPeriodicRefresh();
StorageLiveView(
const StorageID & table_id_,
Context & local_context,

View File

@ -49,3 +49,19 @@ FFFF:FFFF:FFFF:FFFF:FFFF:FFFF:FFFF:FFFF is ipv6 string: 1
::ffff:127.0.0.1 is ipv6 string: 1
::ffff:8.8.8.8 is ipv6 string: 1
2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D is ipv6 string: 1
::ffff:0.0.0.0
::ffff:127.0.0.1
::ffff:127.0.0.1
::ffff:127.0.0.0
::ffff:127.0.0.1
::ffff:127.0.0.2
::ffff:127.0.0.3
::ffff:127.0.0.4
::ffff:127.0.0.5
::ffff:127.0.0.6
::ffff:127.0.0.7
::ffff:127.0.0.8
::ffff:127.0.0.9
::ffff:127.0.0.10
::ffff:127.0.0.11
::ffff:127.0.0.12

View File

@ -84,3 +84,9 @@ SELECT '::ffff:127.0.0.1 is ipv6 string: ', isIPv6String(
SELECT '::ffff:8.8.8.8 is ipv6 string: ', isIPv6String('::ffff:8.8.8.8');
SELECT '2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D is ipv6 string: ', isIPv6String('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D');
-- IPV6 functions parse IPv4 addresses.
SELECT toIPv6('0.0.0.0');
SELECT toIPv6('127.0.0.1');
SELECT cutIPv6(IPv6StringToNum('127.0.0.1'), 0, 0);
SELECT toIPv6('127.0.0.' || toString(number)) FROM numbers(13);

View File

@ -0,0 +1,41 @@
#!/usr/bin/env python3
import os
import sys
import signal
CURDIR = os.path.dirname(os.path.realpath(__file__))
sys.path.insert(0, os.path.join(CURDIR, 'helpers'))
from client import client, prompt, end_of_block
log = None
# uncomment the line below for debugging
#log=sys.stdout
with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2:
client1.expect(prompt)
client2.expect(prompt)
client1.send('SET allow_experimental_live_view = 1')
client1.expect(prompt)
client2.send('SET allow_experimental_live_view = 1')
client2.expect(prompt)
client1.send('DROP TABLE IF EXISTS test.lv')
client1.expect(prompt)
client1.send("CREATE LIVE VIEW test.lv WITH REFRESH 1"
" AS SELECT value FROM system.events WHERE event = 'OSCPUVirtualTimeMicroseconds'")
client1.expect(prompt)
client1.send('WATCH test.lv FORMAT JSONEachRow')
client1.expect(r'"_version":' + end_of_block)
client1.expect(r'"_version":' + end_of_block)
client1.expect(r'"_version":' + end_of_block)
# send Ctrl-C
client1.send('\x03', eol='')
match = client1.expect('(%s)|([#\$] )' % prompt)
if match.groups()[1]:
client1.send(client1.command)
client1.expect(prompt)
client1.send('DROP TABLE test.lv')
client1.expect(prompt)

View File

@ -0,0 +1,52 @@
#!/usr/bin/env python3
import os
import sys
import time
import signal
CURDIR = os.path.dirname(os.path.realpath(__file__))
sys.path.insert(0, os.path.join(CURDIR, 'helpers'))
from client import client, prompt, end_of_block
log = None
# uncomment the line below for debugging
#log=sys.stdout
with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2:
client1.expect(prompt)
client2.expect(prompt)
client1.send('SET allow_experimental_live_view = 1')
client1.expect(prompt)
client2.send('SET allow_experimental_live_view = 1')
client2.expect(prompt)
client1.send('DROP TABLE IF EXISTS test.lv')
client1.expect(prompt)
client1.send("CREATE LIVE VIEW test.lv WITH TIMEOUT 60 AND REFRESH 1"
" AS SELECT value FROM system.events WHERE event = 'OSCPUVirtualTimeMicroseconds'")
client1.expect(prompt)
client1.send('WATCH test.lv FORMAT JSONEachRow')
client1.expect(r'"_version":' + end_of_block)
client1.expect(r'"_version":' + end_of_block)
client1.expect(r'"_version":' + end_of_block)
# send Ctrl-C
client1.send('\x03', eol='')
match = client1.expect('(%s)|([#\$] )' % prompt)
if match.groups()[1]:
client1.send(client1.command)
client1.expect(prompt)
# poll until live view table is dropped
start_time = time.time()
while True:
client1.send('SELECT * FROM test.lv FORMAT JSONEachRow')
client1.expect(prompt)
if 'Table test.lv doesn\'t exist' in client1.before:
break
if time.time() - start_time > 90:
break
# check table is dropped
client1.send('DROP TABLE test.lv')
client1.expect('Table test.lv doesn\'t exist')
client1.expect(prompt)

View File

@ -0,0 +1,68 @@
#!/usr/bin/env python3
import os
import sys
import signal
CURDIR = os.path.dirname(os.path.realpath(__file__))
sys.path.insert(0, os.path.join(CURDIR, 'helpers'))
from client import client, prompt, end_of_block
log = None
# uncomment the line below for debugging
#log=sys.stdout
with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2:
client1.expect(prompt)
client2.expect(prompt)
client1.send('SET allow_experimental_live_view = 1')
client1.expect(prompt)
client2.send('SET allow_experimental_live_view = 1')
client2.expect(prompt)
client1.send('DROP TABLE IF EXISTS test.lv')
client1.expect(prompt)
client1.send('DROP TABLE IF EXISTS test.mt')
client1.expect(prompt)
client1.send('DROP DICTIONARY IF EXITS test.dict')
client1.expect(prompt)
client1.send("CREATE TABLE test.mt (a Int32, b Int32) Engine=MergeTree order by tuple()")
client1.expect(prompt)
client1.send("CREATE DICTIONARY test.dict(a Int32, b Int32) PRIMARY KEY a LAYOUT(FLAT()) " + \
"SOURCE(CLICKHOUSE(db 'test' table 'mt')) LIFETIME(1)")
client1.expect(prompt)
client1.send("CREATE LIVE VIEW test.lv WITH REFRESH 1 AS SELECT * FROM test.dict")
client1.expect(prompt)
client2.send("INSERT INTO test.mt VALUES (1,2)")
client2.expect(prompt)
client1.send('WATCH test.lv FORMAT JSONEachRow')
client1.expect(r'"_version":"1"')
client2.send("INSERT INTO test.mt VALUES (2,2)")
client2.expect(prompt)
client1.expect(r'"_version":"2"')
client2.send("INSERT INTO test.mt VALUES (3,2)")
client2.expect(prompt)
client1.expect(r'"_version":"3"')
# send Ctrl-C
client1.send('\x03', eol='')
match = client1.expect('(%s)|([#\$] )' % prompt)
if match.groups()[1]:
client1.send(client1.command)
client1.expect(prompt)
client1.send('DROP TABLE IF EXISTS test.lv')
client1.expect(prompt)
client1.send('DROP DICTIONARY IF EXISTS test.dict')
client1.expect(prompt)
client1.send('DROP TABLE IF EXISTS test.mt')
client1.expect(prompt)

View File

@ -0,0 +1,2 @@
0
0

View File

@ -0,0 +1,3 @@
SELECT positionCaseInsensitiveUTF8('иголка.ру', 'иголка.Ñ€Ñ\0') AS res;
SELECT positionCaseInsensitiveUTF8('иголка.ру', randomString(rand() % 100)) FROM system.numbers; -- { serverError 2 }
SELECT sum(ignore(positionCaseInsensitiveUTF8('иголка.ру', randomString(rand() % 2)))) FROM numbers(1000000);

View File

@ -0,0 +1 @@
\N

View File

@ -0,0 +1,2 @@
SELECT DISTINCT result FROM (SELECT toStartOfFifteenMinutes(toDateTime(toStartOfFifteenMinutes(toDateTime(1000.0001220703125) + (number * 65536))) + (number * 9223372036854775807)) AS result FROM system.numbers LIMIT 1048576) ORDER BY result DESC NULLS FIRST FORMAT Null;
SELECT round(round(round(round(round(100)), round(round(round(round(NULL), round(65535)), toTypeName(now() + 9223372036854775807) LIKE 'DateTime%DateTime%DateTime%DateTime%', round(-2)), 255), round(NULL))));

View File

@ -0,0 +1 @@
0 0 2

View File

@ -0,0 +1 @@
SELECT bitAnd(number, toUInt64(pow(257, 20) - 1048576)) AS k, argMaxIf(k, if((number % 255) = 256, toInt256(65535), number), number > 42), uniq(number) AS u FROM numbers(2) GROUP BY toInt256(-2, NULL), k;

View File

@ -0,0 +1,23 @@
#!/usr/bin/env bash
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# We throw our own exception from operator new.
# In previous versions of Hyperscan it triggered debug assertion as it only expected std::bad_alloc.
M=1000000
while true
do
$CLICKHOUSE_CLIENT --allow_hyperscan 1 --max_memory_usage $M --format Null --query "
SELECT [1, 2, 3, 11] = arraySort(multiMatchAllIndices('фабрикант', ['', 'рикан', 'а', 'f[a${RANDOM}e]b[ei]rl', 'ф[иа${RANDOM}эе]б[еэи][рпл]', 'афиукд', 'a[f${RANDOM}t],th', '^ф[аие${RANDOM}э]?б?[еэи]?$', 'бе${RANDOM}рлик', 'fa${RANDOM}b', 'фа[беьв]+е?[рл${RANDOM}ко]']))
" 2>&1 | grep -q 'Memory limit' || break;
M=$((M + 100000))
done
echo 'Ok'

View File

@ -0,0 +1 @@
SELECT arrayResize([1, 2, 3], -9223372036854775808); -- { serverError 128 }

View File

@ -26,7 +26,7 @@ class Node(object):
def repr(self):
return f"Node(name='{self.name}')"
def restart(self, timeout=300, safe=True):
def restart(self, timeout=300, retries=5):
"""Restart node.
"""
with self.cluster.lock:
@ -35,15 +35,20 @@ class Node(object):
shell = self.cluster._bash.pop(key)
shell.__exit__(None, None, None)
self.cluster.command(None, f'{self.cluster.docker_compose} restart {self.name}', timeout=timeout)
for retry in range(retries):
r = self.cluster.command(None, f'{self.cluster.docker_compose} restart {self.name}', timeout=timeout)
if r.exitcode == 0:
break
def start(self, timeout=300, safe=True):
def start(self, timeout=300, retries=5):
"""Start node.
"""
self.cluster.command(None, f'{self.cluster.docker_compose} start {self.name}', timeout=timeout)
for retry in range(retries):
r = self.cluster.command(None, f'{self.cluster.docker_compose} start {self.name}', timeout=timeout)
if r.exitcode == 0:
break
def stop(self, timeout=300, safe=True):
def stop(self, timeout=300, retries=5):
"""Stop node.
"""
with self.cluster.lock:
@ -52,7 +57,10 @@ class Node(object):
shell = self.cluster._bash.pop(key)
shell.__exit__(None, None, None)
self.cluster.command(None, f'{self.cluster.docker_compose} stop {self.name}', timeout=timeout)
for retry in range(retries):
r = self.cluster.command(None, f'{self.cluster.docker_compose} stop {self.name}', timeout=timeout)
if r.exitcode == 0:
break
def command(self, *args, **kwargs):
return self.cluster.command(self.name, *args, **kwargs)
@ -71,7 +79,7 @@ class ClickHouseNode(Node):
continue
assert False, "container is not healthy"
def stop(self, timeout=300, safe=True):
def stop(self, timeout=300, safe=True, retries=5):
"""Stop node.
"""
if safe:
@ -89,17 +97,23 @@ class ClickHouseNode(Node):
shell = self.cluster._bash.pop(key)
shell.__exit__(None, None, None)
self.cluster.command(None, f'{self.cluster.docker_compose} stop {self.name}', timeout=timeout)
for retry in range(retries):
r = self.cluster.command(None, f'{self.cluster.docker_compose} stop {self.name}', timeout=timeout)
if r.exitcode == 0:
break
def start(self, timeout=300, wait_healthy=True):
def start(self, timeout=300, wait_healthy=True, retries=5):
"""Start node.
"""
self.cluster.command(None, f'{self.cluster.docker_compose} start {self.name}', timeout=timeout)
for retry in range(retries):
r = self.cluster.command(None, f'{self.cluster.docker_compose} start {self.name}', timeout=timeout)
if r.exitcode == 0:
break
if wait_healthy:
self.wait_healthy(timeout)
def restart(self, timeout=300, safe=True, wait_healthy=True):
def restart(self, timeout=300, safe=True, wait_healthy=True, retries=5):
"""Restart node.
"""
if safe:
@ -117,7 +131,10 @@ class ClickHouseNode(Node):
shell = self.cluster._bash.pop(key)
shell.__exit__(None, None, None)
self.cluster.command(None, f'{self.cluster.docker_compose} restart {self.name}', timeout=timeout)
for retry in range(retries):
r = self.cluster.command(None, f'{self.cluster.docker_compose} restart {self.name}', timeout=timeout)
if r.exitcode == 0:
break
if wait_healthy:
self.wait_healthy(timeout)

View File

@ -270,7 +270,7 @@ def ldap_authenticated_users(*users, config_d_dir="/etc/clickhouse-server/users.
config = create_ldap_users_config_content(*users, config_d_dir=config_d_dir, config_file=config_file)
return add_config(config, restart=restart)
def invalid_server_config(servers, message=None, tail=13, timeout=60):
def invalid_server_config(servers, message=None, tail=30, timeout=60):
"""Check that ClickHouse errors when trying to load invalid LDAP servers configuration file.
"""
node = current().context.node
@ -299,7 +299,7 @@ def invalid_server_config(servers, message=None, tail=13, timeout=60):
with By("removing the config file", description=config.path):
node.command(f"rm -rf {config.path}", exitcode=0)
def invalid_user_config(servers, config, message=None, tail=13, timeout=60):
def invalid_user_config(servers, config, message=None, tail=30, timeout=60):
"""Check that ClickHouse errors when trying to load invalid LDAP users configuration file.
"""
node = current().context.node

View File

@ -245,7 +245,7 @@ def invalid_verification_cooldown_value(self, invalid_value, timeout=20):
}}
with When("I try to use this configuration then it should not work"):
invalid_server_config(servers, message=error_message, tail=17, timeout=timeout)
invalid_server_config(servers, message=error_message, tail=30, timeout=timeout)
@TestScenario
@Requirements(

View File

@ -39,7 +39,7 @@ def empty_server_name(self, timeout=20):
"message": "DB::Exception: user1: Authentication failed: password is incorrect or there is no user with such name"
}]
config = create_ldap_users_config_content(*users)
invalid_user_config(servers, config, message=message, tail=15, timeout=timeout)
invalid_user_config(servers, config, message=message, tail=30, timeout=timeout)
@TestScenario
@Requirements(
@ -147,7 +147,7 @@ def ldap_and_password(self):
error_message = "DB::Exception: More than one field of 'password'"
with Then("I expect an error when I try to load the configuration file", description=error_message):
invalid_user_config(servers, new_config, message=error_message, tail=16)
invalid_user_config(servers, new_config, message=error_message, tail=30)
@TestFeature
@Name("user config")

View File

@ -133,7 +133,7 @@ def create_entries_ldap_external_user_directory_config_content(entries, config_d
return Config(content, path, name, uid, "config.xml")
def invalid_ldap_external_user_directory_config(server, roles, message, tail=20, timeout=60, config=None):
def invalid_ldap_external_user_directory_config(server, roles, message, tail=30, timeout=60, config=None):
"""Check that ClickHouse errors when trying to load invalid LDAP external user directory
configuration file.
"""

View File

@ -41,7 +41,7 @@ def invalid_host(self):
RQ_SRS_009_LDAP_ExternalUserDirectory_Configuration_Server_Invalid("1.0"),
RQ_SRS_009_LDAP_ExternalUserDirectory_Configuration_Server_Host("1.0")
)
def empty_host(self, tail=20, timeout=60):
def empty_host(self, tail=30, timeout=60):
"""Check that server returns an error when LDAP server
host value is empty.
"""
@ -50,14 +50,14 @@ def empty_host(self, tail=20, timeout=60):
servers = {"foo": {"host": "", "port": "389", "enable_tls": "no"}}
invalid_server_config(servers, message=message, tail=16, timeout=timeout)
invalid_server_config(servers, message=message, tail=30, timeout=timeout)
@TestScenario
@Requirements(
RQ_SRS_009_LDAP_ExternalUserDirectory_Configuration_Server_Invalid("1.0"),
RQ_SRS_009_LDAP_ExternalUserDirectory_Configuration_Server_Host("1.0")
)
def missing_host(self, tail=20, timeout=60):
def missing_host(self, tail=30, timeout=60):
"""Check that server returns an error when LDAP server
host is missing.
"""
@ -148,7 +148,7 @@ def invalid_enable_tls_value(self, timeout=60):
servers = {"openldap1": {"host": "openldap1", "port": "389", "enable_tls": "foo",
"auth_dn_prefix": "cn=", "auth_dn_suffix": ",ou=users,dc=company,dc=com"
}}
invalid_server_config(servers, message=message, tail=18, timeout=timeout)
invalid_server_config(servers, message=message, tail=30, timeout=timeout)
@TestScenario
@Requirements(
@ -259,7 +259,7 @@ def invalid_verification_cooldown_value(self, invalid_value, timeout=20):
}}
with When("I try to use this configuration then it should not work"):
invalid_server_config(servers, message=error_message, tail=17, timeout=timeout)
invalid_server_config(servers, message=error_message, tail=30, timeout=timeout)
@TestScenario
@Requirements(

View File

@ -18,7 +18,7 @@ xfails = {
@Name("role mapping")
@ArgumentParser(argparser)
@Specifications(
QA_SRS014_ClickHouse_LDAP_Role_Mapping
SRS_014_ClickHouse_LDAP_Role_Mapping
)
@Requirements(
RQ_SRS_014_LDAP_RoleMapping("1.0")

View File

@ -0,0 +1,504 @@
# SRS-014 ClickHouse LDAP Role Mapping
# Software Requirements Specification
## Table of Contents
* 1 [Revision History](#revision-history)
* 2 [Introduction](#introduction)
* 3 [Terminology](#terminology)
* 3.1 [LDAP](#ldap)
* 4 [Requirements](#requirements)
* 4.1 [General](#general)
* 4.1.1 [RQ.SRS-014.LDAP.RoleMapping](#rqsrs-014ldaprolemapping)
* 4.1.2 [RQ.SRS-014.LDAP.RoleMapping.WithFixedRoles](#rqsrs-014ldaprolemappingwithfixedroles)
* 4.1.3 [RQ.SRS-014.LDAP.RoleMapping.Search](#rqsrs-014ldaprolemappingsearch)
* 4.2 [Mapped Role Names](#mapped-role-names)
* 4.2.1 [RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.WithUTF8Characters](#rqsrs-014ldaprolemappingmaprolenamewithutf8characters)
* 4.2.2 [RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.Long](#rqsrs-014ldaprolemappingmaprolenamelong)
* 4.2.3 [RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.WithSpecialXMLCharacters](#rqsrs-014ldaprolemappingmaprolenamewithspecialxmlcharacters)
* 4.2.4 [RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.WithSpecialRegexCharacters](#rqsrs-014ldaprolemappingmaprolenamewithspecialregexcharacters)
* 4.3 [Multiple Roles](#multiple-roles)
* 4.3.1 [RQ.SRS-014.LDAP.RoleMapping.Map.MultipleRoles](#rqsrs-014ldaprolemappingmapmultipleroles)
* 4.4 [LDAP Groups](#ldap-groups)
* 4.4.1 [RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.Removed](#rqsrs-014ldaprolemappingldapgroupremoved)
* 4.4.2 [RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.RemovedAndAdded.Parallel](#rqsrs-014ldaprolemappingldapgroupremovedandaddedparallel)
* 4.4.3 [RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.UserRemoved](#rqsrs-014ldaprolemappingldapgroupuserremoved)
* 4.4.4 [RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.UserRemovedAndAdded.Parallel](#rqsrs-014ldaprolemappingldapgroupuserremovedandaddedparallel)
* 4.5 [RBAC Roles](#rbac-roles)
* 4.5.1 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.NotPresent](#rqsrs-014ldaprolemappingrbacrolenotpresent)
* 4.5.2 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.Added](#rqsrs-014ldaprolemappingrbacroleadded)
* 4.5.3 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.Removed](#rqsrs-014ldaprolemappingrbacroleremoved)
* 4.5.4 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.Readded](#rqsrs-014ldaprolemappingrbacrolereadded)
* 4.5.5 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.RemovedAndAdded.Parallel](#rqsrs-014ldaprolemappingrbacroleremovedandaddedparallel)
* 4.5.6 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.New](#rqsrs-014ldaprolemappingrbacrolenew)
* 4.5.7 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.NewPrivilege](#rqsrs-014ldaprolemappingrbacrolenewprivilege)
* 4.5.8 [RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.RemovedPrivilege](#rqsrs-014ldaprolemappingrbacroleremovedprivilege)
* 4.6 [Authentication](#authentication)
* 4.6.1 [RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel](#rqsrs-014ldaprolemappingauthenticationparallel)
* 4.6.2 [RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.ValidAndInvalid](#rqsrs-014ldaprolemappingauthenticationparallelvalidandinvalid)
* 4.6.3 [RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.MultipleServers](#rqsrs-014ldaprolemappingauthenticationparallelmultipleservers)
* 4.6.4 [RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.LocalOnly](#rqsrs-014ldaprolemappingauthenticationparallellocalonly)
* 4.6.5 [RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.LocalAndMultipleLDAP](#rqsrs-014ldaprolemappingauthenticationparallellocalandmultipleldap)
* 4.6.6 [RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.SameUser](#rqsrs-014ldaprolemappingauthenticationparallelsameuser)
* 4.7 [Server Configuration](#server-configuration)
* 4.7.1 [BindDN Parameter](#binddn-parameter)
* 4.7.1.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.Server.BindDN](#rqsrs-014ldaprolemappingconfigurationserverbinddn)
* 4.7.1.2 [RQ.SRS-014.LDAP.RoleMapping.Configuration.Server.BindDN.ConflictWith.AuthDN](#rqsrs-014ldaprolemappingconfigurationserverbinddnconflictwithauthdn)
* 4.8 [External User Directory Configuration](#external-user-directory-configuration)
* 4.8.1 [Syntax](#syntax)
* 4.8.1.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Syntax](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingsyntax)
* 4.8.2 [Special Characters Escaping](#special-characters-escaping)
* 4.8.2.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.SpecialCharactersEscaping](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingspecialcharactersescaping)
* 4.8.3 [Multiple Sections](#multiple-sections)
* 4.8.3.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.MultipleSections](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingmultiplesections)
* 4.8.3.2 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.MultipleSections.IdenticalParameters](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingmultiplesectionsidenticalparameters)
* 4.8.4 [BaseDN Parameter](#basedn-parameter)
* 4.8.4.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.BaseDN](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingbasedn)
* 4.8.5 [Attribute Parameter](#attribute-parameter)
* 4.8.5.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Attribute](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingattribute)
* 4.8.6 [Scope Parameter](#scope-parameter)
* 4.8.6.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingscope)
* 4.8.6.2 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Base](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingscopevaluebase)
* 4.8.6.3 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.OneLevel](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingscopevalueonelevel)
* 4.8.6.4 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Children](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingscopevaluechildren)
* 4.8.6.5 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Subtree](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingscopevaluesubtree)
* 4.8.6.6 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Default](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingscopevaluedefault)
* 4.8.7 [Search Filter Parameter](#search-filter-parameter)
* 4.8.7.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.SearchFilter](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingsearchfilter)
* 4.8.8 [Prefix Parameter](#prefix-parameter)
* 4.8.8.1 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingprefix)
* 4.8.8.2 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.Default](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingprefixdefault)
* 4.8.8.3 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.WithUTF8Characters](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingprefixwithutf8characters)
* 4.8.8.4 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.WithSpecialXMLCharacters](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingprefixwithspecialxmlcharacters)
* 4.8.8.5 [RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.WithSpecialRegexCharacters](#rqsrs-014ldaprolemappingconfigurationuserdirectoryrolemappingprefixwithspecialregexcharacters)
* 5 [References](#references)
## Revision History
This document is stored in an electronic form using [Git] source control management software
hosted in a [GitHub Repository].
All the updates are tracked using the [Revision History].
## Introduction
The [SRS-007 ClickHouse Authentication of Users via LDAP] added support for authenticating
users using an [LDAP] server and the [SRS-009 ClickHouse LDAP External User Directory] added
support for authenticating users using an [LDAP] external user directory.
This requirements specification adds additional functionality for mapping [LDAP] groups to
the corresponding [ClickHouse] [RBAC] roles when [LDAP] external user directory is configured.
This functionality will enable easier access management for [LDAP] authenticated users
as the privileges granted by the roles can be granted or revoked by granting or revoking
a corresponding [LDAP] group to one or more [LDAP] users.
For the use case when only [LDAP] user authentication is used, the roles can be
managed using [RBAC] in the same way as for non-[LDAP] authenticated users.
## Terminology
### LDAP
* Lightweight Directory Access Protocol
## Requirements
### General
#### RQ.SRS-014.LDAP.RoleMapping
version: 1.0
[ClickHouse] SHALL support mapping of [LDAP] groups to [RBAC] roles
for users authenticated using [LDAP] external user directory.
#### RQ.SRS-014.LDAP.RoleMapping.WithFixedRoles
version: 1.0
[ClickHouse] SHALL support mapping of [LDAP] groups to [RBAC] roles
for users authenticated using [LDAP] external user directory when
one or more roles are specified in the `<roles>` section.
#### RQ.SRS-014.LDAP.RoleMapping.Search
version: 1.0
[ClickHouse] SHALL perform search on the [LDAP] server and map the results to [RBAC] role names
when authenticating users using the [LDAP] external user directory if the `<role_mapping>` section is configured
as part of the [LDAP] external user directory. The matched roles SHALL be assigned to the user.
### Mapped Role Names
#### RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.WithUTF8Characters
version: 1.0
[ClickHouse] SHALL support mapping [LDAP] search results for users authenticated using [LDAP] external user directory
to an [RBAC] role that contains UTF-8 characters.
#### RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.Long
version: 1.0
[ClickHouse] SHALL support mapping [LDAP] search results for users authenticated using [LDAP] external user directory
to an [RBAC] role that has a name with more than 128 characters.
#### RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.WithSpecialXMLCharacters
version: 1.0
[ClickHouse] SHALL support mapping [LDAP] search results for users authenticated using [LDAP] external user directory
to an [RBAC] role that has a name that contains special characters that need to be escaped in XML.
#### RQ.SRS-014.LDAP.RoleMapping.Map.Role.Name.WithSpecialRegexCharacters
version: 1.0
[ClickHouse] SHALL support mapping [LDAP] search results for users authenticated using [LDAP] external user directory
to an [RBAC] role that has a name that contains special characters that need to be escaped in regex.
### Multiple Roles
#### RQ.SRS-014.LDAP.RoleMapping.Map.MultipleRoles
version: 1.0
[ClickHouse] SHALL support mapping one or more [LDAP] search results for users authenticated using
[LDAP] external user directory to one or more [RBAC] role.
### LDAP Groups
#### RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.Removed
version: 1.0
[ClickHouse] SHALL not assign [RBAC] role(s) for any users authenticated using [LDAP] external user directory
if the corresponding [LDAP] group(s) that map those role(s) are removed. Any users that have active sessions SHALL still
have privileges provided by the role(s) until the next time they are authenticated.
#### RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.RemovedAndAdded.Parallel
version: 1.0
[ClickHouse] SHALL support authenticating users using [LDAP] external user directory
when [LDAP] groups are removed and added
at the same time as [LDAP] user authentications are performed in parallel.
#### RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.UserRemoved
version: 1.0
[ClickHouse] SHALL not assign [RBAC] role(s) for the user authenticated using [LDAP] external user directory
if the user has been removed from the corresponding [LDAP] group(s) that map those role(s).
Any active user sessions SHALL have privileges provided by the role(s) until the next time the user is authenticated.
#### RQ.SRS-014.LDAP.RoleMapping.LDAP.Group.UserRemovedAndAdded.Parallel
version: 1.0
[ClickHouse] SHALL support authenticating users using [LDAP] external user directory
when [LDAP] users are added and removed from [LDAP] groups used to map to [RBAC] roles
at the same time as [LDAP] user authentications are performed in parallel.
### RBAC Roles
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.NotPresent
version: 1.0
[ClickHouse] SHALL not reject authentication attempt using [LDAP] external user directory if any of the roles that are
are mapped from [LDAP] but are not present locally.
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.Added
version: 1.0
[ClickHouse] SHALL add the privileges provided by the [LDAP] mapped role when the
role is not present during user authentication using [LDAP] external user directory
as soon as the role is added.
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.Removed
version: 1.0
[ClickHouse] SHALL remove the privileges provided by the role from all the
users authenticated using [LDAP] external user directory if the [RBAC] role that was mapped
as a result of [LDAP] search is removed.
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.Readded
version: 1.0
[ClickHouse] SHALL reassign the [RBAC] role and add all the privileges provided by the role
when it is re-added after removal for all [LDAP] users authenticated using external user directory
for any role that was mapped as a result of [LDAP] search.
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.RemovedAndAdded.Parallel
version: 1.0
[ClickHouse] SHALL support authenticating users using [LDAP] external user directory
when [RBAC] roles that are mapped by [LDAP] groups
are added and removed at the same time as [LDAP] user authentications are performed in parallel.
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.New
version: 1.0
[ClickHouse] SHALL not allow any new roles to be assigned to any
users authenticated using [LDAP] external user directory unless the role is specified
in the configuration of the external user directory or was mapped as a result of [LDAP] search.
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.NewPrivilege
version: 1.0
[ClickHouse] SHALL add new privilege to all the users authenticated using [LDAP] external user directory
when new privilege is added to one of the roles that were mapped as a result of [LDAP] search.
#### RQ.SRS-014.LDAP.RoleMapping.RBAC.Role.RemovedPrivilege
version: 1.0
[ClickHouse] SHALL remove privilege from all the users authenticated using [LDAP] external user directory
when the privilege that was provided by the mapped role is removed from all the roles
that were mapped as a result of [LDAP] search.
### Authentication
#### RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel
version: 1.0
[ClickHouse] SHALL support parallel authentication of users using [LDAP] server
when using [LDAP] external user directory that has role mapping enabled.
#### RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.ValidAndInvalid
version: 1.0
[ClickHouse] SHALL support authentication of valid users and
prohibit authentication of invalid users using [LDAP] server
in parallel without having invalid attempts affecting valid authentications
when using [LDAP] external user directory that has role mapping enabled.
#### RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.MultipleServers
version: 1.0
[ClickHouse] SHALL support parallel authentication of external [LDAP] users
authenticated using multiple [LDAP] external user directories that have
role mapping enabled.
#### RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.LocalOnly
version: 1.0
[ClickHouse] SHALL support parallel authentication of users defined only locally
when one or more [LDAP] external user directories with role mapping
are specified in the configuration file.
#### RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.LocalAndMultipleLDAP
version: 1.0
[ClickHouse] SHALL support parallel authentication of local and external [LDAP] users
authenticated using multiple [LDAP] external user directories with role mapping enabled.
#### RQ.SRS-014.LDAP.RoleMapping.Authentication.Parallel.SameUser
version: 1.0
[ClickHouse] SHALL support parallel authentication of the same external [LDAP] user
authenticated using the same [LDAP] external user directory with role mapping enabled.
### Server Configuration
#### BindDN Parameter
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.Server.BindDN
version: 1.0
[ClickHouse] SHALL support the `<bind_dn>` parameter in the `<ldap_servers><server_name>` section
of the `config.xml` that SHALL be used to construct the `DN` to bind to.
The resulting `DN` SHALL be constructed by replacing all `{user_name}` substrings of the template
with the actual user name during each authentication attempt.
For example,
```xml
<yandex>
<ldap_servers>
<my_ldap_server>
<!-- ... -->
<bind_dn>uid={user_name},ou=users,dc=example,dc=com</bind_dn>
<!-- ... -->
</my_ldap_server>
</ldap_servers>
</yandex>
```
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.Server.BindDN.ConflictWith.AuthDN
version: 1.0
[ClickHouse] SHALL return an error if both `<bind_dn>` and `<auth_dn_prefix>` or `<auth_dn_suffix>` parameters
are specified as part of [LDAP] server description in the `<ldap_servers>` section of the `config.xml`.
### External User Directory Configuration
#### Syntax
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Syntax
version: 1.0
[ClickHouse] SHALL support the `role_mapping` sub-section in the `<user_directories><ldap>` section
of the `config.xml`.
For example,
```xml
<yandex>
<user_directories>
<ldap>
<!-- ... -->
<role_mapping>
<base_dn>ou=groups,dc=example,dc=com</base_dn>
<attribute>cn</attribute>
<scope>subtree</scope>
<search_filter>(&amp;(objectClass=groupOfNames)(member={bind_dn}))</search_filter>
<prefix>clickhouse_</prefix>
</role_mapping>
</ldap>
</user_directories>
</yandex>
```
#### Special Characters Escaping
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.SpecialCharactersEscaping
version: 1.0
[ClickHouse] SHALL support properly escaped special XML characters that can be present
as part of the values for different configuration parameters inside the
`<user_directories><ldap><role_mapping>` section of the `config.xml` such as
* `<search_filter>` parameter
* `<prefix>` parameter
#### Multiple Sections
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.MultipleSections
version: 1.0
[ClickHouse] SHALL support multiple `<role_mapping>` sections defined inside the same `<user_directories><ldap>` section
of the `config.xml` and all of the `<role_mapping>` sections SHALL be applied.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.MultipleSections.IdenticalParameters
version: 1.0
[ClickHouse] SHALL not duplicate mapped roles when multiple `<role_mapping>` sections
with identical parameters are defined inside the `<user_directories><ldap>` section
of the `config.xml`.
#### BaseDN Parameter
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.BaseDN
version: 1.0
[ClickHouse] SHALL support the `<base_dn>` parameter in the `<user_directories><ldap><role_mapping>` section
of the `config.xml` that SHALL specify the template to be used to construct the base `DN` for the [LDAP] search.
The resulting `DN` SHALL be constructed by replacing all the `{user_name}` and `{bind_dn}` substrings of
the template with the actual user name and bind `DN` during each [LDAP] search.
#### Attribute Parameter
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Attribute
version: 1.0
[ClickHouse] SHALL support the `<attribute>` parameter in the `<user_directories><ldap><role_mapping>` section of
the `config.xml` that SHALL specify the name of the attribute whose values SHALL be returned by the [LDAP] search.
#### Scope Parameter
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope
version: 1.0
[ClickHouse] SHALL support the `<scope>` parameter in the `<user_directories><ldap><role_mapping>` section of
the `config.xml` that SHALL define the scope of the LDAP search as defined
by the https://ldapwiki.com/wiki/LDAP%20Search%20Scopes.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Base
version: 1.0
[ClickHouse] SHALL support the `base` value for the the `<scope>` parameter in the
`<user_directories><ldap><role_mapping>` section of the `config.xml` that SHALL
limit the scope as specified by the https://ldapwiki.com/wiki/BaseObject.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.OneLevel
version: 1.0
[ClickHouse] SHALL support the `one_level` value for the the `<scope>` parameter in the
`<user_directories><ldap><role_mapping>` section of the `config.xml` that SHALL
limit the scope as specified by the https://ldapwiki.com/wiki/SingleLevel.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Children
version: 1.0
[ClickHouse] SHALL support the `children` value for the the `<scope>` parameter in the
`<user_directories><ldap><role_mapping>` section of the `config.xml` that SHALL
limit the scope as specified by the https://ldapwiki.com/wiki/SubordinateSubtree.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Subtree
version: 1.0
[ClickHouse] SHALL support the `children` value for the the `<scope>` parameter in the
`<user_directories><ldap><role_mapping>` section of the `config.xml` that SHALL
limit the scope as specified by the https://ldapwiki.com/wiki/WholeSubtree.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Scope.Value.Default
version: 1.0
[ClickHouse] SHALL support the `subtree` as the default value for the the `<scope>` parameter in the
`<user_directories><ldap><role_mapping>` section of the `config.xml` when the `<scope>` parameter is not specified.
#### Search Filter Parameter
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.SearchFilter
version: 1.0
[ClickHouse] SHALL support the `<search_filter>` parameter in the `<user_directories><ldap><role_mapping>`
section of the `config.xml` that SHALL specify the template used to construct
the [LDAP filter](https://ldap.com/ldap-filters/) for the search.
The resulting filter SHALL be constructed by replacing all `{user_name}`, `{bind_dn}`, and `{base_dn}` substrings
of the template with the actual user name, bind `DN`, and base `DN` during each the [LDAP] search.
#### Prefix Parameter
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix
version: 1.0
[ClickHouse] SHALL support the `<prefix>` parameter in the `<user directories><ldap><role_mapping>`
section of the `config.xml` that SHALL be expected to be in front of each string in
the original list of strings returned by the [LDAP] search.
Prefix SHALL be removed from the original strings and resulting strings SHALL be treated as [RBAC] role names.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.Default
version: 1.0
[ClickHouse] SHALL support empty string as the default value of the `<prefix>` parameter in
the `<user directories><ldap><role_mapping>` section of the `config.xml`.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.WithUTF8Characters
version: 1.0
[ClickHouse] SHALL support UTF8 characters as the value of the `<prefix>` parameter in
the `<user directories><ldap><role_mapping>` section of the `config.xml`.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.WithSpecialXMLCharacters
version: 1.0
[ClickHouse] SHALL support XML special characters as the value of the `<prefix>` parameter in
the `<user directories><ldap><role_mapping>` section of the `config.xml`.
##### RQ.SRS-014.LDAP.RoleMapping.Configuration.UserDirectory.RoleMapping.Prefix.WithSpecialRegexCharacters
version: 1.0
[ClickHouse] SHALL support regex special characters as the value of the `<prefix>` parameter in
the `<user directories><ldap><role_mapping>` section of the `config.xml`.
## References
* **Access Control and Account Management**: https://clickhouse.tech/docs/en/operations/access-rights/
* **LDAP**: https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol
* **ClickHouse:** https://clickhouse.tech
* **GitHub Repository**: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
* **Revision History**: https://github.com/ClickHouse/ClickHouse/commits/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
* **Git:** https://git-scm.com/
[RBAC]: https://clickhouse.tech/docs/en/operations/access-rights/
[SRS]: #srs
[Access Control and Account Management]: https://clickhouse.tech/docs/en/operations/access-rights/
[SRS-009 ClickHouse LDAP External User Directory]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/external_user_directory/requirements/requirements.md
[SRS-007 ClickHouse Authentication of Users via LDAP]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/authentication/requirements/requirements.md
[LDAP]: https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol
[ClickHouse]: https://clickhouse.tech
[GitHub Repository]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
[Revision History]: https://github.com/ClickHouse/ClickHouse/commits/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
[Git]: https://git-scm.com/
[GitHub]: https://github.com

View File

@ -1,6 +1,6 @@
# These requirements were auto generated
# from software requirements specification (SRS)
# document by TestFlows v1.6.210101.1235930.
# document by TestFlows v1.6.210129.1222545.
# Do not edit by hand but re-generate instead
# using 'tfs requirements generate' command.
from testflows.core import Specification
@ -814,15 +814,15 @@ RQ_SRS_014_LDAP_RoleMapping_Configuration_UserDirectory_RoleMapping_Prefix_WithS
level=4,
num='4.8.8.5')
QA_SRS014_ClickHouse_LDAP_Role_Mapping = Specification(
name='QA-SRS014 ClickHouse LDAP Role Mapping',
SRS_014_ClickHouse_LDAP_Role_Mapping = Specification(
name='SRS-014 ClickHouse LDAP Role Mapping',
description=None,
author='vzakaznikov',
date='December 4, 2020',
status='-',
approved_by='-',
approved_date='-',
approved_version='-',
author=None,
date=None,
status=None,
approved_by=None,
approved_date=None,
approved_version=None,
version=None,
group=None,
type=None,
@ -950,27 +950,9 @@ QA_SRS014_ClickHouse_LDAP_Role_Mapping = Specification(
RQ_SRS_014_LDAP_RoleMapping_Configuration_UserDirectory_RoleMapping_Prefix_WithSpecialRegexCharacters,
),
content='''
# QA-SRS014 ClickHouse LDAP Role Mapping
# SRS-014 ClickHouse LDAP Role Mapping
# Software Requirements Specification
(c) 2020 Altinity LTD. All Rights Reserved.
**Document status:** Confidential
**Author:** vzakaznikov
**Date:** December 4, 2020
## Approval
**Status:** -
**Version:** -
**Approved by:** -
**Date:** -
## Table of Contents
* 1 [Revision History](#revision-history)
@ -1046,13 +1028,13 @@ QA_SRS014_ClickHouse_LDAP_Role_Mapping = Specification(
## Revision History
This document is stored in an electronic form using [Git] source control management software
hosted in a [GitLab Repository].
hosted in a [GitHub Repository].
All the updates are tracked using the [Revision History].
## Introduction
The [QA-SRS007 ClickHouse Authentication of Users via LDAP] added support for authenticating
users using an [LDAP] server and the [QA-SRS009 ClickHouse LDAP External User Directory] added
The [SRS-007 ClickHouse Authentication of Users via LDAP] added support for authenticating
users using an [LDAP] server and the [SRS-009 ClickHouse LDAP External User Directory] added
support for authenticating users using an [LDAP] external user directory.
This requirements specification adds additional functionality for mapping [LDAP] groups to
@ -1457,19 +1439,19 @@ the `<user directories><ldap><role_mapping>` section of the `config.xml`.
* **Access Control and Account Management**: https://clickhouse.tech/docs/en/operations/access-rights/
* **LDAP**: https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol
* **ClickHouse:** https://clickhouse.tech
* **GitLab Repository**: https://gitlab.com/altinity-qa/documents/qa-srs014-clickhouse-ldap-role-mapping/-/blob/master/QA_SRS014_ClickHouse_LDAP_Role_Mapping.md
* **Revision History**: https://gitlab.com/altinity-qa/documents/qa-srs014-clickhouse-ldap-role-mapping/-/commits/master/QA_SRS014_ClickHouse_LDAP_Role_Mapping.md
* **GitHub Repository**: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
* **Revision History**: https://github.com/ClickHouse/ClickHouse/commits/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
* **Git:** https://git-scm.com/
[RBAC]: https://clickhouse.tech/docs/en/operations/access-rights/
[SRS]: #srs
[Access Control and Account Management]: https://clickhouse.tech/docs/en/operations/access-rights/
[QA-SRS009 ClickHouse LDAP External User Directory]: https://gitlab.com/altinity-qa/documents/qa-srs009-clickhouse-ldap-external-user-directory/-/blob/master/QA_SRS009_ClickHouse_LDAP_External_User_Directory.md
[QA-SRS007 ClickHouse Authentication of Users via LDAP]: https://gitlab.com/altinity-qa/documents/qa-srs007-clickhouse-athentication-of-users-via-ldap/-/blob/master/QA_SRS007_ClickHouse_Authentication_of_Users_via_LDAP.md
[SRS-009 ClickHouse LDAP External User Directory]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/external_user_directory/requirements/requirements.md
[SRS-007 ClickHouse Authentication of Users via LDAP]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/authentication/requirements/requirements.md
[LDAP]: https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol
[ClickHouse]: https://clickhouse.tech
[GitLab Repository]: https://gitlab.com/altinity-qa/documents/qa-srs014-clickhouse-ldap-role-mapping/-/blob/master/QA_SRS014_ClickHouse_LDAP_Role_Mapping.md
[Revision History]: https://gitlab.com/altinity-qa/documents/qa-srs014-clickhouse-ldap-role-mapping/-/commits/master/QA_SRS014_ClickHouse_LDAP_Role_Mapping.md
[GitHub Repository]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
[Revision History]: https://github.com/ClickHouse/ClickHouse/commits/master/tests/testflows/ldap/role_mapping/requirements/requirements.md
[Git]: https://git-scm.com/
[GitLab]: https://gitlab.com
[GitHub]: https://github.com
''')

View File

@ -65,7 +65,7 @@ def bind_dn_conflict_with_auth_dn(self, timeout=60):
}
}
invalid_server_config(servers, message=message, tail=18, timeout=timeout)
invalid_server_config(servers, message=message, tail=30, timeout=timeout)
@TestFeature
@ -75,4 +75,4 @@ def feature(self, node="clickhouse1"):
"""
self.context.node = self.context.cluster.node(node)
for scenario in loads(current_module(), Scenario):
scenario()
scenario()