mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge branch 'master' into intdiv-ubsan
This commit is contained in:
commit
1c0c3b3aab
2
contrib/hyperscan
vendored
2
contrib/hyperscan
vendored
@ -1 +1 @@
|
||||
Subproject commit 3907fd00ee8b2538739768fa9533f8635a276531
|
||||
Subproject commit e9f08df0213fc637aac0a5bbde9beeaeba2fe9fa
|
2
contrib/poco
vendored
2
contrib/poco
vendored
@ -1 +1 @@
|
||||
Subproject commit 2c32e17c7dfee1f8bf24227b697cdef5fddf0823
|
||||
Subproject commit e11f3c971570cf6a31006cd21cadf41a259c360a
|
@ -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
|
||||
|
@ -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
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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) \
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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];
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
41
tests/queries/0_stateless/00962_live_view_periodic_refresh.py
Executable file
41
tests/queries/0_stateless/00962_live_view_periodic_refresh.py
Executable 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)
|
||||
|
52
tests/queries/0_stateless/00962_live_view_periodic_refresh_and_timeout.py
Executable file
52
tests/queries/0_stateless/00962_live_view_periodic_refresh_and_timeout.py
Executable 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)
|
68
tests/queries/0_stateless/00962_live_view_periodic_refresh_dictionary.py
Executable file
68
tests/queries/0_stateless/00962_live_view_periodic_refresh_dictionary.py
Executable 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)
|
||||
|
||||
|
||||
|
2
tests/queries/0_stateless/01674_unicode_asan.reference
Normal file
2
tests/queries/0_stateless/01674_unicode_asan.reference
Normal file
@ -0,0 +1,2 @@
|
||||
0
|
||||
0
|
3
tests/queries/0_stateless/01674_unicode_asan.sql
Normal file
3
tests/queries/0_stateless/01674_unicode_asan.sql
Normal 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);
|
@ -0,0 +1 @@
|
||||
\N
|
2
tests/queries/0_stateless/01680_date_time_add_ubsan.sql
Normal file
2
tests/queries/0_stateless/01680_date_time_add_ubsan.sql
Normal 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))));
|
@ -0,0 +1 @@
|
||||
0 0 2
|
1
tests/queries/0_stateless/01681_arg_min_max_if_fix.sql
Normal file
1
tests/queries/0_stateless/01681_arg_min_max_if_fix.sql
Normal 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;
|
@ -0,0 +1 @@
|
||||
Ok
|
23
tests/queries/0_stateless/01681_hyperscan_debug_assertion.sh
Executable file
23
tests/queries/0_stateless/01681_hyperscan_debug_assertion.sh
Executable 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'
|
1
tests/queries/0_stateless/01682_gather_utils_ubsan.sql
Normal file
1
tests/queries/0_stateless/01682_gather_utils_ubsan.sql
Normal file
@ -0,0 +1 @@
|
||||
SELECT arrayResize([1, 2, 3], -9223372036854775808); -- { serverError 128 }
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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(
|
||||
|
@ -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")
|
||||
|
@ -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.
|
||||
"""
|
||||
|
@ -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(
|
||||
|
@ -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")
|
||||
|
504
tests/testflows/ldap/role_mapping/requirements/requirements.md
Normal file
504
tests/testflows/ldap/role_mapping/requirements/requirements.md
Normal 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>(&(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
|
@ -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
|
||||
''')
|
||||
|
@ -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()
|
||||
|
Loading…
Reference in New Issue
Block a user