Merge branch 'master' of github.com:ClickHouse/ClickHouse into davenger-patch-1

This commit is contained in:
Alexander Gololobov 2024-06-17 17:13:40 +02:00
commit 18a5d8acdc
79 changed files with 790 additions and 181 deletions

View File

@ -32,7 +32,7 @@ constexpr void static_for(F && f)
template <is_enum T>
struct fmt::formatter<T> : fmt::formatter<std::string_view>
{
constexpr auto format(T value, auto& format_context)
constexpr auto format(T value, auto& format_context) const
{
return formatter<string_view>::format(magic_enum::enum_name(value), format_context);
}

View File

@ -12,6 +12,8 @@
#include <base/types.h>
#include <base/unaligned.h>
#include <base/simd.h>
#include <fmt/core.h>
#include <fmt/ostream.h>
#include <city.h>
@ -376,3 +378,5 @@ namespace PackedZeroTraits
std::ostream & operator<<(std::ostream & os, const StringRef & str);
template<> struct fmt::formatter<StringRef> : fmt::ostream_formatter {};

View File

@ -62,7 +62,7 @@ struct fmt::formatter<wide::integer<Bits, Signed>>
}
template <typename FormatContext>
auto format(const wide::integer<Bits, Signed> & value, FormatContext & ctx)
auto format(const wide::integer<Bits, Signed> & value, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", to_string(value));
}

2
contrib/fmtlib vendored

@ -1 +1 @@
Subproject commit b6f4ceaed0a0a24ccf575fab6c56dd50ccf6f1a9
Subproject commit a33701196adfad74917046096bf5a2aa0ab0bb50

View File

@ -13,7 +13,6 @@ set (SRCS
${FMT_SOURCE_DIR}/include/fmt/core.h
${FMT_SOURCE_DIR}/include/fmt/format.h
${FMT_SOURCE_DIR}/include/fmt/format-inl.h
${FMT_SOURCE_DIR}/include/fmt/locale.h
${FMT_SOURCE_DIR}/include/fmt/os.h
${FMT_SOURCE_DIR}/include/fmt/ostream.h
${FMT_SOURCE_DIR}/include/fmt/printf.h

View File

@ -0,0 +1,40 @@
---
sidebar_position: 1
sidebar_label: 2024
---
# 2024 Changelog
### ClickHouse release v23.8.15.35-lts (060ff8e813a) FIXME as compared to v23.8.14.6-lts (967e51c1d6b)
#### Build/Testing/Packaging Improvement
* Backported in [#63621](https://github.com/ClickHouse/ClickHouse/issues/63621): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Backported in [#65153](https://github.com/ClickHouse/ClickHouse/issues/65153): Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### Bug Fix (user-visible misbehavior in an official stable release)
* Backported in [#64422](https://github.com/ClickHouse/ClickHouse/issues/64422): Fixes [#59989](https://github.com/ClickHouse/ClickHouse/issues/59989): runs init scripts when force-enabled or when no database exists, rather than the inverse. [#59991](https://github.com/ClickHouse/ClickHouse/pull/59991) ([jktng](https://github.com/jktng)).
* Backported in [#64016](https://github.com/ClickHouse/ClickHouse/issues/64016): Fix "Invalid storage definition in metadata file" for parameterized views. [#60708](https://github.com/ClickHouse/ClickHouse/pull/60708) ([Azat Khuzhin](https://github.com/azat)).
* Backported in [#63456](https://github.com/ClickHouse/ClickHouse/issues/63456): Fix the issue where the function `addDays` (and similar functions) reports an error when the first parameter is `DateTime64`. [#61561](https://github.com/ClickHouse/ClickHouse/pull/61561) ([Shuai li](https://github.com/loneylee)).
* Backported in [#63289](https://github.com/ClickHouse/ClickHouse/issues/63289): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)).
* Backported in [#63512](https://github.com/ClickHouse/ClickHouse/issues/63512): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)).
* Backported in [#63902](https://github.com/ClickHouse/ClickHouse/issues/63902): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)).
* Backported in [#64104](https://github.com/ClickHouse/ClickHouse/issues/64104): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)).
* Backported in [#64265](https://github.com/ClickHouse/ClickHouse/issues/64265): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)).
* Backported in [#64867](https://github.com/ClickHouse/ClickHouse/issues/64867): Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)).
#### NO CL CATEGORY
* Backported in [#63704](https://github.com/ClickHouse/ClickHouse/issues/63704):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)).
#### NO CL ENTRY
* NO CL ENTRY: 'Installation test has wrong check_state'. [#63994](https://github.com/ClickHouse/ClickHouse/pull/63994) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
#### NOT FOR CHANGELOG / INSIGNIFICANT
* Backported in [#63343](https://github.com/ClickHouse/ClickHouse/issues/63343): The commit url has different pattern. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Backported in [#63965](https://github.com/ClickHouse/ClickHouse/issues/63965): fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)).
* Backported in [#64043](https://github.com/ClickHouse/ClickHouse/issues/64043): Do not create new release in release branch automatically. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).
* Pin requests to fix the integration tests. [#65183](https://github.com/ClickHouse/ClickHouse/pull/65183) ([Mikhail f. Shiryaev](https://github.com/Felixoid)).

View File

@ -508,7 +508,7 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`:
- `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request.
- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`.
- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `http_response_headers`, `response_content`, `query`, `query_param_name`.
`type` currently supports three types: [predefined_query_handler](#predefined_query_handler), [dynamic_query_handler](#dynamic_query_handler), [static](#static).
- `query` — use with `predefined_query_handler` type, executes query when the handler is called.
@ -519,6 +519,8 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`:
- `content_type` — use with any type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `http_response_headers` — use with any type, response headers map. Could be used to set content type as well.
- `response_content` — use with `static` type, response content sent to client, when using the prefix file:// or config://, find the content from the file or configuration sends to client.
Next are the configuration methods for different `type`.
@ -616,6 +618,33 @@ Return a message.
<type>static</type>
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
<defaults/>
</http_handlers>
```
`http_response_headers` could be used to set content type instead of `content_type`.
``` xml
<http_handlers>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/hi</url>
<handler>
<type>static</type>
<status>402</status>
<http_response_headers>
<Content-Type>text/html; charset=UTF-8</Content-Type>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
@ -696,6 +725,9 @@ Find the content from the file send to client.
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file:///absolute_path_file.html</response_content>
</handler>
</rule>
@ -706,6 +738,9 @@ Find the content from the file send to client.
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file://./relative_path_file.html</response_content>
</handler>
</rule>

View File

@ -639,6 +639,10 @@ An internal metric of the low-level memory allocator (jemalloc). See https://jem
An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html
### jemalloc.prof.active
An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html
**See Also**
- [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring.

View File

@ -7,33 +7,43 @@ sidebar_label: Float32, Float64
# Float32, Float64
:::note
If you need accurate calculations, in particular if you work with financial or business data requiring a high precision you should consider using Decimal instead. Floats might lead to inaccurate results as illustrated below:
If you need accurate calculations, in particular if you work with financial or business data requiring a high precision, you should consider using [Decimal](../data-types/decimal.md) instead.
```
[Floating Point Numbers](https://en.wikipedia.org/wiki/IEEE_754) might lead to inaccurate results as illustrated below:
```sql
CREATE TABLE IF NOT EXISTS float_vs_decimal
(
my_float Float64,
my_decimal Decimal64(3)
)Engine=MergeTree ORDER BY tuple()
INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal
)
Engine=MergeTree
ORDER BY tuple();
# Generate 1 000 000 random numbers with 2 decimal places and store them as a float and as a decimal
INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000;
```
```
SELECT sum(my_float), sum(my_decimal) FROM float_vs_decimal;
> 500279.56300000014 500279.563
┌──────sum(my_float)─┬─sum(my_decimal)─┐
│ 499693.60500000004 │ 499693.605 │
└────────────────────┴─────────────────┘
SELECT sumKahan(my_float), sumKahan(my_decimal) FROM float_vs_decimal;
> 500279.563 500279.563
┌─sumKahan(my_float)─┬─sumKahan(my_decimal)─┐
│ 499693.605 │ 499693.605 │
└────────────────────┴──────────────────────┘
```
:::
[Floating point numbers](https://en.wikipedia.org/wiki/IEEE_754).
Types are equivalent to types of C:
The equivalent types in ClickHouse and in C are given below:
- `Float32``float`.
- `Float64``double`.
Aliases:
Float types in ClickHouse have the following aliases:
- `Float32``FLOAT`, `REAL`, `SINGLE`.
- `Float64``DOUBLE`, `DOUBLE PRECISION`.

View File

@ -414,6 +414,8 @@ $ curl -v 'http://localhost:8123/predefined_query'
- `content_type` — используется со всеми типами, возвращает [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `http_response_headers` — используется со всеми типами чтобы добавить кастомные хедеры в ответ. Может использоваться в том числе для задания хедера `Content-Type` вместо `content_type`.
- `response_content` — используется с типом`static`, содержимое ответа, отправленное клиенту, при использовании префикса file:// or config://, находит содержимое из файла или конфигурации, отправленного клиенту.
Далее приведены методы настройки для различных типов.
@ -509,6 +511,33 @@ max_final_threads 2
<type>static</type>
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
<defaults/>
</http_handlers>
```
`http_response_headers` так же может использоваться для определения `Content-Type` вместо `content_type`.
``` xml
<http_handlers>
<rule>
<methods>GET</methods>
<headers><XXX>xxx</XXX></headers>
<url>/hi</url>
<handler>
<type>static</type>
<status>402</status>
<http_response_headers>
<Content-Type>text/html; charset=UTF-8</Content-Type>
<Content-Language>en</Content-Language>
<X-My-Custom-Header>43</X-My-Custom-Header>
</http_response_headers>
<response_content>Say Hi!</response_content>
</handler>
</rule>
@ -589,6 +618,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file:///absolute_path_file.html</response_content>
</handler>
</rule>
@ -599,6 +631,9 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler'
<handler>
<type>static</type>
<content_type>text/html; charset=UTF-8</content_type>
<http_response_headers>
<ETag>737060cd8c284d8af7ad3082f209582d</ETag>
</http_response_headers>
<response_content>file://./relative_path_file.html</response_content>
</handler>
</rule>

View File

@ -31,6 +31,7 @@ namespace DB
{
namespace ErrorCodes
{
extern const int AUTHENTICATION_FAILED;
extern const int SUPPORT_IS_DISABLED;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
@ -90,8 +91,10 @@ bool AuthenticationData::Util::checkPasswordBcrypt(std::string_view password [[m
{
#if USE_BCRYPT
int ret = bcrypt_checkpw(password.data(), reinterpret_cast<const char *>(password_bcrypt.data()));
/// Before 24.6 we didn't validate hashes on creation, so it could be that the stored hash is invalid
/// and it could not be decoded by the library
if (ret == -1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "BCrypt library failed: bcrypt_checkpw returned {}", ret);
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Internal failure decoding Bcrypt hash");
return (ret == 0);
#else
throw Exception(
@ -230,6 +233,17 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Password hash for the 'BCRYPT_PASSWORD' authentication type has length {} "
"but must be 59 or 60 bytes.", hash.size());
auto resized = hash;
resized.resize(64);
#if USE_BCRYPT
/// Verify that it is a valid hash
int ret = bcrypt_checkpw("", reinterpret_cast<const char *>(resized.data()));
if (ret == -1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Could not decode the provided hash with 'bcrypt_hash'");
#endif
password_hash = hash;
password_hash.resize(64);
return;

View File

@ -406,7 +406,7 @@ struct fmt::formatter<DB::Identifier>
}
template <typename FormatContext>
auto format(const DB::Identifier & identifier, FormatContext & ctx)
auto format(const DB::Identifier & identifier, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", identifier.getFullName());
}
@ -428,7 +428,7 @@ struct fmt::formatter<DB::IdentifierView>
}
template <typename FormatContext>
auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx)
auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", identifier_view.getFullName());
}

View File

@ -112,7 +112,7 @@ struct fmt::formatter<DB::TestHint::ErrorVector>
}
template <typename FormatContext>
auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx)
auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) const
{
if (ErrorVector.empty())
return fmt::format_to(ctx.out(), "{}", 0);

View File

@ -415,6 +415,15 @@ Value saveAllArenasMetric(AsynchronousMetricValues & values,
fmt::format("jemalloc.arenas.all.{}", metric_name));
}
template<typename Value>
Value saveJemallocProf(AsynchronousMetricValues & values,
const std::string & metric_name)
{
return saveJemallocMetricImpl<Value>(values,
fmt::format("prof.{}", metric_name),
fmt::format("jemalloc.prof.{}", metric_name));
}
}
#endif
@ -607,6 +616,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update)
saveJemallocMetric<size_t>(new_values, "background_thread.num_threads");
saveJemallocMetric<uint64_t>(new_values, "background_thread.num_runs");
saveJemallocMetric<uint64_t>(new_values, "background_thread.run_intervals");
saveJemallocProf<size_t>(new_values, "active");
saveAllArenasMetric<size_t>(new_values, "pactive");
[[maybe_unused]] size_t je_malloc_pdirty = saveAllArenasMetric<size_t>(new_values, "pdirty");
[[maybe_unused]] size_t je_malloc_pmuzzy = saveAllArenasMetric<size_t>(new_values, "pmuzzy");

View File

@ -85,9 +85,18 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
/// Write information about current server instance to the file.
WriteBufferFromFileDescriptor out(fd, 1024);
fill(out);
/// Finalize here to avoid throwing exceptions in destructor.
out.finalize();
try
{
fill(out);
/// Finalize here to avoid throwing exceptions in destructor.
out.finalize();
}
catch (...)
{
/// Finalize in case of exception to avoid throwing exceptions in destructor
out.finalize();
throw;
}
}
catch (...)
{

View File

@ -41,21 +41,9 @@ Throttler::Throttler(size_t max_speed_, size_t limit_, const char * limit_exceed
UInt64 Throttler::add(size_t amount)
{
// Values obtained under lock to be checked after release
size_t count_value;
double tokens_value;
{
std::lock_guard lock(mutex);
auto now = clock_gettime_ns_adjusted(prev_ns);
if (max_speed)
{
double delta_seconds = prev_ns ? static_cast<double>(now - prev_ns) / NS : 0;
tokens = std::min<double>(tokens + max_speed * delta_seconds - amount, max_burst);
}
count += amount;
count_value = count;
tokens_value = tokens;
prev_ns = now;
}
size_t count_value = 0;
double tokens_value = 0.0;
addImpl(amount, count_value, tokens_value);
if (limit && count_value > limit)
throw Exception::createDeprecated(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED);
@ -77,6 +65,21 @@ UInt64 Throttler::add(size_t amount)
return static_cast<UInt64>(sleep_time_ns);
}
void Throttler::addImpl(size_t amount, size_t & count_value, double & tokens_value)
{
std::lock_guard lock(mutex);
auto now = clock_gettime_ns_adjusted(prev_ns);
if (max_speed)
{
double delta_seconds = prev_ns ? static_cast<double>(now - prev_ns) / NS : 0;
tokens = std::min<double>(tokens + max_speed * delta_seconds - amount, max_burst);
}
count += amount;
count_value = count;
tokens_value = tokens;
prev_ns = now;
}
void Throttler::reset()
{
std::lock_guard lock(mutex);
@ -98,4 +101,14 @@ bool Throttler::isThrottling() const
return false;
}
Int64 Throttler::getAvailable()
{
// To update bucket state and receive current number of token in a thread-safe way
size_t count_value = 0;
double tokens_value = 0.0;
addImpl(0, count_value, tokens_value);
return static_cast<Int64>(tokens_value);
}
}

View File

@ -57,7 +57,13 @@ public:
/// Is throttler already accumulated some sleep time and throttling.
bool isThrottling() const;
Int64 getAvailable();
UInt64 getMaxSpeed() const { return static_cast<UInt64>(max_speed); }
UInt64 getMaxBurst() const { return static_cast<UInt64>(max_burst); }
private:
void addImpl(size_t amount, size_t & count_value, double & tokens_value);
size_t count{0};
const size_t max_speed{0}; /// in tokens per second.
const size_t max_burst{0}; /// in tokens.

View File

@ -108,7 +108,7 @@ struct fmt::formatter<DB::TransactionID>
}
template<typename FormatContext>
auto format(const DB::TransactionID & tid, FormatContext & context)
auto format(const DB::TransactionID & tid, FormatContext & context) const
{
return fmt::format_to(context.out(), "({}, {}, {})", tid.start_csn, tid.local_tid, tid.host_id);
}

View File

@ -647,7 +647,7 @@ public:
template <> struct fmt::formatter<Coordination::Error> : fmt::formatter<std::string_view>
{
constexpr auto format(Coordination::Error code, auto & ctx)
constexpr auto format(Coordination::Error code, auto & ctx) const
{
return formatter<string_view>::format(Coordination::errorMessage(code), ctx);
}

View File

@ -49,7 +49,7 @@ struct fmt::formatter<ReadableSize>
}
template <typename FormatContext>
auto format(const ReadableSize & size, FormatContext & ctx)
auto format(const ReadableSize & size, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", formatReadableSizeWithBinarySuffix(size.value));
}

View File

@ -57,7 +57,7 @@ using ClusterUpdateActions = std::vector<ClusterUpdateAction>;
template <>
struct fmt::formatter<DB::RaftServerConfig> : fmt::formatter<string_view>
{
constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx)
constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) const
{
return fmt::format_to(
ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority);
@ -67,7 +67,7 @@ struct fmt::formatter<DB::RaftServerConfig> : fmt::formatter<string_view>
template <>
struct fmt::formatter<DB::ClusterUpdateAction> : fmt::formatter<string_view>
{
constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx)
constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) const
{
if (const auto * add = std::get_if<DB::AddRaftServer>(&action))
return fmt::format_to(ctx.out(), "(Add server {})", add->id);

View File

@ -1038,7 +1038,7 @@ struct fmt::formatter<DB::Field>
}
template <typename FormatContext>
auto format(const DB::Field & x, FormatContext & ctx)
auto format(const DB::Field & x, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", toString(x));
}

View File

@ -125,7 +125,7 @@ namespace fmt
}
template <typename FormatContext>
auto format(const DB::QualifiedTableName & name, FormatContext & ctx)
auto format(const DB::QualifiedTableName & name, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}.{}", DB::backQuoteIfNeed(name.database), DB::backQuoteIfNeed(name.table));
}

View File

@ -629,7 +629,7 @@ struct fmt::formatter<DB::DataTypePtr>
}
template <typename FormatContext>
auto format(const DB::DataTypePtr & type, FormatContext & ctx)
auto format(const DB::DataTypePtr & type, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", type->getName());
}

View File

@ -219,6 +219,9 @@ public:
return client_configuration.for_disk_s3;
}
ThrottlerPtr getPutRequestThrottler() const { return client_configuration.put_request_throttler; }
ThrottlerPtr getGetRequestThrottler() const { return client_configuration.get_request_throttler; }
private:
friend struct ::MockS3::Client;

View File

@ -77,7 +77,15 @@ WriteBufferFromFile::~WriteBufferFromFile()
if (fd < 0)
return;
finalize();
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
int err = ::close(fd);
/// Everything except for EBADF should be ignored in dtor, since all of
/// others (EINTR/EIO/ENOSPC/EDQUOT) could be possible during writing to

View File

@ -105,7 +105,14 @@ WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor(
WriteBufferFromFileDescriptor::~WriteBufferFromFileDescriptor()
{
finalize();
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void WriteBufferFromFileDescriptor::finalizeImpl()

View File

@ -1420,7 +1420,7 @@ struct fmt::formatter<DB::UUID>
}
template<typename FormatContext>
auto format(const DB::UUID & uuid, FormatContext & context)
auto format(const DB::UUID & uuid, FormatContext & context) const
{
return fmt::format_to(context.out(), "{}", toString(uuid));
}

View File

@ -450,8 +450,8 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
if (!column.statistics.empty())
{
column_declaration->stat_type = column.statistics.getAST();
column_declaration->children.push_back(column_declaration->stat_type);
column_declaration->statistics_desc = column.statistics.getAST();
column_declaration->children.push_back(column_declaration->statistics_desc);
}
if (column.ttl)
@ -676,12 +676,11 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
}
column.statistics.column_name = column.name; /// We assign column name here for better exception error message.
if (col_decl.stat_type)
if (col_decl.statistics_desc)
{
if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistics is now disabled. Turn on allow_experimental_statistics");
column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl);
column.statistics.data_type = column.type;
column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl, column.type);
}
if (col_decl.ttl)
@ -1089,11 +1088,14 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
String current_database = getContext()->getCurrentDatabase();
auto database_name = create.database ? create.getDatabase() : current_database;
bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery();
auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup);
if (!create.sql_security && create.supportSQLSecurity() && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query)
create.sql_security = std::make_shared<ASTSQLSecurity>();
if (create.sql_security)
processSQLSecurityOption(getContext(), create.sql_security->as<ASTSQLSecurity &>(), create.attach, create.is_materialized_view);
processSQLSecurityOption(getContext(), create.sql_security->as<ASTSQLSecurity &>(), create.is_materialized_view, /* skip_check_permissions= */ mode >= LoadingStrictnessLevel::SECONDARY_CREATE);
DDLGuardPtr ddl_guard;
@ -1220,9 +1222,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (!UserDefinedSQLFunctionFactory::instance().empty())
UserDefinedSQLFunctionVisitor::visit(query_ptr);
bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery();
auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup);
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode);
@ -1887,7 +1886,7 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr
}
}
void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view)
void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view, bool skip_check_permissions)
{
/// If no SQL security is specified, apply default from default_*_view_sql_security setting.
if (!sql_security.type)
@ -1928,7 +1927,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ
}
/// Checks the permissions for the specified definer user.
if (sql_security.definer && !sql_security.is_definer_current_user && !is_attach)
if (sql_security.definer && !sql_security.is_definer_current_user && !skip_check_permissions)
{
const auto definer_name = sql_security.definer->toString();
@ -1938,7 +1937,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ
context_->checkAccess(AccessType::SET_DEFINER, definer_name);
}
if (sql_security.type == SQLSecurityType::NONE && !is_attach)
if (sql_security.type == SQLSecurityType::NONE && !skip_check_permissions)
context_->checkAccess(AccessType::ALLOW_SQL_SECURITY_NONE);
}

View File

@ -82,7 +82,7 @@ public:
void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override;
/// Check access right, validate definer statement and replace `CURRENT USER` with actual name.
static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach = false, bool is_materialized_view = false);
static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view = false, bool skip_check_permissions = false);
private:
struct TableProperties

View File

@ -210,27 +210,54 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr
auto total = disk->getTotalSpace();
/// Some disks don't support information about the space.
if (!total)
continue;
auto available = disk->getAvailableSpace();
auto unreserved = disk->getUnreservedSpace();
new_values[fmt::format("DiskTotal_{}", name)] = { *total,
"The total size in bytes of the disk (virtual filesystem). Remote filesystems may not provide this information." };
if (available)
if (total)
{
new_values[fmt::format("DiskUsed_{}", name)] = { *total - *available,
"Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." };
auto available = disk->getAvailableSpace();
auto unreserved = disk->getUnreservedSpace();
new_values[fmt::format("DiskAvailable_{}", name)] = { *available,
"Available bytes on the disk (virtual filesystem). Remote filesystems may not provide this information." };
new_values[fmt::format("DiskTotal_{}", name)] = { *total,
"The total size in bytes of the disk (virtual filesystem). Remote filesystems may not provide this information." };
if (available)
{
new_values[fmt::format("DiskUsed_{}", name)] = { *total - *available,
"Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." };
new_values[fmt::format("DiskAvailable_{}", name)] = { *available,
"Available bytes on the disk (virtual filesystem). Remote filesystems may not provide this information." };
}
if (unreserved)
new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved,
"Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." };
}
if (unreserved)
new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved,
"Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." };
#if USE_AWS_S3
try
{
if (auto s3_client = disk->getS3StorageClient())
{
if (auto put_throttler = s3_client->getPutRequestThrottler())
{
new_values[fmt::format("DiskPutObjectThrottlerRPS_{}", name)] = { put_throttler->getMaxSpeed(),
"PutObject Request throttling limit on the disk in requests per second (virtual filesystem). Local filesystems may not provide this information." };
new_values[fmt::format("DiskPutObjectThrottlerAvailable_{}", name)] = { put_throttler->getAvailable(),
"Number of PutObject requests that can be currently issued without hitting throttling limit on the disk (virtual filesystem). Local filesystems may not provide this information." };
}
if (auto get_throttler = s3_client->getGetRequestThrottler())
{
new_values[fmt::format("DiskGetObjectThrottlerRPS_{}", name)] = { get_throttler->getMaxSpeed(),
"GetObject Request throttling limit on the disk in requests per second (virtual filesystem). Local filesystems may not provide this information." };
new_values[fmt::format("DiskGetObjectThrottlerAvailable_{}", name)] = { get_throttler->getAvailable(),
"Number of GetObject requests that can be currently issued without hitting throttling limit on the disk (virtual filesystem). Local filesystems may not provide this information." };
}
}
}
catch (...) // NOLINT(bugprone-empty-catch)
{
// Skip disk that do not have s3 throttlers
}
#endif
}
}

View File

@ -136,7 +136,7 @@ namespace fmt
}
template <typename FormatContext>
auto format(const DB::StorageID & storage_id, FormatContext & ctx)
auto format(const DB::StorageID & storage_id, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", storage_id.getNameForLogs());
}

View File

@ -39,10 +39,10 @@ ASTPtr ASTColumnDeclaration::clone() const
res->children.push_back(res->codec);
}
if (stat_type)
if (statistics_desc)
{
res->stat_type = stat_type->clone();
res->children.push_back(res->stat_type);
res->statistics_desc = statistics_desc->clone();
res->children.push_back(res->statistics_desc);
}
if (ttl)
@ -111,10 +111,10 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo
codec->formatImpl(format_settings, state, frame);
}
if (stat_type)
if (statistics_desc)
{
format_settings.ostr << ' ';
stat_type->formatImpl(format_settings, state, frame);
statistics_desc->formatImpl(format_settings, state, frame);
}
if (ttl)

View File

@ -19,7 +19,7 @@ public:
bool ephemeral_default = false;
ASTPtr comment;
ASTPtr codec;
ASTPtr stat_type;
ASTPtr statistics_desc;
ASTPtr ttl;
ASTPtr collation;
ASTPtr settings;

View File

@ -193,7 +193,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ASTPtr default_expression;
ASTPtr comment_expression;
ASTPtr codec_expression;
ASTPtr stat_type_expression;
ASTPtr statistics_desc_expression;
ASTPtr ttl_expression;
ASTPtr collation_expression;
ASTPtr settings;
@ -325,7 +325,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
if (s_stat.ignore(pos, expected))
{
if (!stat_type_parser.parse(pos, stat_type_expression, expected))
if (!stat_type_parser.parse(pos, statistics_desc_expression, expected))
return false;
}
@ -398,10 +398,10 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
column_declaration->children.push_back(std::move(settings));
}
if (stat_type_expression)
if (statistics_desc_expression)
{
column_declaration->stat_type = stat_type_expression;
column_declaration->children.push_back(std::move(stat_type_expression));
column_declaration->statistics_desc = statistics_desc_expression;
column_declaration->children.push_back(std::move(statistics_desc_expression));
}
if (ttl_expression)

View File

@ -40,7 +40,7 @@ struct fmt::formatter<DB::ASTPtr>
}
template<typename FormatContext>
auto format(const DB::ASTPtr & ast, FormatContext & context)
auto format(const DB::ASTPtr & ast, FormatContext & context) const
{
return fmt::format_to(context.out(), "{}", DB::serializeAST(*ast));
}

View File

@ -1,14 +1,15 @@
#include <Processors/QueryPlan/ReadFromLoopStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Storages/IStorage.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <QueryPipeline/QueryPipeline.h>
#include <Interpreters/Context.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/ISource.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <QueryPipeline/QueryPlanResourceHolder.h>
#include <Processors/ISource.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromLoopStep.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <QueryPipeline/QueryPipeline.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <QueryPipeline/QueryPlanResourceHolder.h>
#include <Storages/IStorage.h>
namespace DB
{
@ -111,6 +112,13 @@ namespace DB
std::unique_ptr<PullingPipelineExecutor> executor;
};
static ContextPtr disableParallelReplicas(ContextPtr context)
{
auto modified_context = Context::createCopy(context);
modified_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
return modified_context;
}
ReadFromLoopStep::ReadFromLoopStep(
const Names & column_names_,
const SelectQueryInfo & query_info_,
@ -125,7 +133,7 @@ namespace DB
column_names_,
query_info_,
storage_snapshot_,
context_)
disableParallelReplicas(context_))
, column_names(column_names_)
, processed_stage(processed_stage_)
, inner_storage(std::move(inner_storage_))

View File

@ -37,7 +37,7 @@ struct fmt::formatter<DB::RowNumber>
}
template <typename FormatContext>
auto format(const DB::RowNumber & x, FormatContext & ctx)
auto format(const DB::RowNumber & x, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}:{}", x.block, x.row);
}

View File

@ -30,7 +30,6 @@
#include <Common/scope_guard_safe.h>
#include <Common/setThreadName.h>
#include <Common/typeid_cast.h>
#include <Common/re2.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatFactory.h>
@ -44,6 +43,7 @@
#include <Poco/Base64Decoder.h>
#include <Poco/Base64Encoder.h>
#include <Poco/Net/HTTPBasicCredentials.h>
#include <Poco/Net/HTTPMessage.h>
#include <Poco/Net/HTTPStream.h>
#include <Poco/MemoryStream.h>
#include <Poco/StreamCopier.h>
@ -53,7 +53,10 @@
#include <algorithm>
#include <chrono>
#include <memory>
#include <optional>
#include <sstream>
#include <unordered_map>
#include <utility>
#if USE_SSL
#include <Poco/Net/X509Certificate.h>
@ -338,11 +341,11 @@ void HTTPHandler::pushDelayedResults(Output & used_output)
}
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const std::optional<String> & content_type_override_)
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_)
: server(server_)
, log(getLogger(name))
, default_settings(server.context()->getSettingsRef())
, content_type_override(content_type_override_)
, http_response_headers_override(http_response_headers_override_)
{
server_display_name = server.config().getString("display_name", getFQDNOrHostName());
}
@ -670,8 +673,7 @@ void HTTPHandler::processQuery(
{
auto tmp_data = std::make_shared<TemporaryDataOnDisk>(server.context()->getTempDataOnDisk());
auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr
{
auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr {
return tmp_data->createRawStream();
};
@ -893,13 +895,14 @@ void HTTPHandler::processQuery(
customizeContext(request, context, *in_post_maybe_compressed);
in = has_external_data ? std::move(in_param) : std::make_unique<ConcatReadBuffer>(*in_param, *in_post_maybe_compressed);
applyHTTPResponseHeaders(response, http_response_headers_override);
auto set_query_result = [&response, this] (const QueryResultDetails & details)
{
response.add("X-ClickHouse-Query-Id", details.query_id);
if (content_type_override)
response.setContentType(*content_type_override);
else if (details.content_type)
if (!(http_response_headers_override && http_response_headers_override->contains(Poco::Net::HTTPMessage::CONTENT_TYPE))
&& details.content_type)
response.setContentType(*details.content_type);
if (details.format)
@ -1185,8 +1188,9 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
used_output.finalize();
}
DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & param_name_, const std::optional<String>& content_type_override_)
: HTTPHandler(server_, "DynamicQueryHandler", content_type_override_), param_name(param_name_)
DynamicQueryHandler::DynamicQueryHandler(
IServer & server_, const std::string & param_name_, const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "DynamicQueryHandler", http_response_headers_override_), param_name(param_name_)
{
}
@ -1247,8 +1251,8 @@ PredefinedQueryHandler::PredefinedQueryHandler(
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const std::optional<String> & content_type_override_)
: HTTPHandler(server_, "PredefinedQueryHandler", content_type_override_)
const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "PredefinedQueryHandler", http_response_headers_override_)
, receive_params(receive_params_)
, predefined_query(predefined_query_)
, url_regex(url_regex_)
@ -1340,14 +1344,10 @@ HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server,
{
auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query");
std::optional<String> content_type_override;
if (config.has(config_prefix + ".handler.content_type"))
content_type_override = config.getString(config_prefix + ".handler.content_type");
HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
auto creator = [&server, query_param_name, content_type_override] () -> std::unique_ptr<DynamicQueryHandler>
{
return std::make_unique<DynamicQueryHandler>(server, query_param_name, content_type_override);
};
auto creator = [&server, query_param_name, http_response_headers_override]() -> std::unique_ptr<DynamicQueryHandler>
{ return std::make_unique<DynamicQueryHandler>(server, query_param_name, http_response_headers_override); };
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(std::move(creator));
factory->addFiltersFromConfig(config, config_prefix);
@ -1402,9 +1402,7 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
headers_name_with_regex.emplace(std::make_pair(header_name, regex));
}
std::optional<String> content_type_override;
if (config.has(config_prefix + ".handler.content_type"))
content_type_override = config.getString(config_prefix + ".handler.content_type");
HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
std::shared_ptr<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>> factory;
@ -1424,12 +1422,12 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
predefined_query,
regex,
headers_name_with_regex,
content_type_override]
http_response_headers_override]
-> std::unique_ptr<PredefinedQueryHandler>
{
return std::make_unique<PredefinedQueryHandler>(
server, analyze_receive_params, predefined_query, regex,
headers_name_with_regex, content_type_override);
headers_name_with_regex, http_response_headers_override);
};
factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>>(std::move(creator));
factory->addFiltersFromConfig(config, config_prefix);
@ -1442,12 +1440,12 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
analyze_receive_params,
predefined_query,
headers_name_with_regex,
content_type_override]
http_response_headers_override]
-> std::unique_ptr<PredefinedQueryHandler>
{
return std::make_unique<PredefinedQueryHandler>(
server, analyze_receive_params, predefined_query, CompiledRegexPtr{},
headers_name_with_regex, content_type_override);
headers_name_with_regex, http_response_headers_override);
};
factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>>(std::move(creator));

View File

@ -1,5 +1,8 @@
#pragma once
#include <optional>
#include <string>
#include <unordered_map>
#include <Core/Names.h>
#include <Server/HTTP/HTMLForm.h>
#include <Server/HTTP/HTTPRequestHandler.h>
@ -10,6 +13,8 @@
#include <Compression/CompressedWriteBuffer.h>
#include <Common/re2.h>
#include "HTTPResponseHeaderWriter.h"
namespace CurrentMetrics
{
extern const Metric HTTPConnection;
@ -31,7 +36,7 @@ using CompiledRegexPtr = std::shared_ptr<const re2::RE2>;
class HTTPHandler : public HTTPRequestHandler
{
public:
HTTPHandler(IServer & server_, const std::string & name, const std::optional<String> & content_type_override_);
HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_);
~HTTPHandler() override;
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
@ -113,8 +118,8 @@ private:
/// See settings http_max_fields, http_max_field_name_size, http_max_field_value_size in HTMLForm.
const Settings & default_settings;
/// Overrides Content-Type provided by the format of the response.
std::optional<String> content_type_override;
/// Overrides for response headers.
HTTPResponseHeaderSetup http_response_headers_override;
// session is reset at the end of each request/response.
std::unique_ptr<Session> session;
@ -162,8 +167,12 @@ class DynamicQueryHandler : public HTTPHandler
{
private:
std::string param_name;
public:
explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query", const std::optional<String>& content_type_override_ = std::nullopt);
explicit DynamicQueryHandler(
IServer & server_,
const std::string & param_name_ = "query",
const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt);
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override;
@ -177,11 +186,15 @@ private:
std::string predefined_query;
CompiledRegexPtr url_regex;
std::unordered_map<String, CompiledRegexPtr> header_name_with_capture_regex;
public:
PredefinedQueryHandler(
IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_
, const CompiledRegexPtr & url_regex_, const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_
, const std::optional<std::string> & content_type_override_);
IServer & server_,
const NameSet & receive_params_,
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt);
void customizeContext(HTTPServerRequest & request, ContextMutablePtr context, ReadBuffer & body) override;

View File

@ -74,7 +74,8 @@ static auto createPingHandlerFactory(IServer & server)
auto creator = [&server]() -> std::unique_ptr<StaticRequestHandler>
{
constexpr auto ping_response_expression = "Ok.\n";
return std::make_unique<StaticRequestHandler>(server, ping_response_expression);
return std::make_unique<StaticRequestHandler>(
server, ping_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8"));
};
return std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(creator));
}
@ -214,7 +215,8 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS
auto root_creator = [&server]() -> std::unique_ptr<StaticRequestHandler>
{
constexpr auto root_response_expression = "config://http_server_default_response";
return std::make_unique<StaticRequestHandler>(server, root_response_expression);
return std::make_unique<StaticRequestHandler>(
server, root_response_expression, parseHTTPResponseHeaders("text/html; charset=UTF-8"));
};
auto root_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(root_creator));
root_handler->attachStrictPath("/");

View File

@ -0,0 +1,69 @@
#include "HTTPResponseHeaderWriter.h"
#include <unordered_map>
#include <utility>
#include <Poco/Net/HTTPMessage.h>
namespace DB
{
std::unordered_map<String, String>
baseParseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::unordered_map<String, String> http_response_headers_override;
String http_response_headers_key = config_prefix + ".handler.http_response_headers";
String http_response_headers_key_prefix = http_response_headers_key + ".";
if (config.has(http_response_headers_key))
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(http_response_headers_key, keys);
for (const auto & key : keys)
{
http_response_headers_override[key] = config.getString(http_response_headers_key_prefix + key);
}
}
if (config.has(config_prefix + ".handler.content_type"))
http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = config.getString(config_prefix + ".handler.content_type");
return http_response_headers_override;
}
HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
std::unordered_map<String, String> http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix);
if (http_response_headers_override.empty())
return {};
return std::move(http_response_headers_override);
}
std::unordered_map<String, String> parseHTTPResponseHeaders(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type)
{
std::unordered_map<String, String> http_response_headers_override = baseParseHTTPResponseHeaders(config, config_prefix);
if (!http_response_headers_override.contains(Poco::Net::HTTPMessage::CONTENT_TYPE))
http_response_headers_override[Poco::Net::HTTPMessage::CONTENT_TYPE] = default_content_type;
return http_response_headers_override;
}
std::unordered_map<String, String> parseHTTPResponseHeaders(const std::string & default_content_type)
{
return {{{Poco::Net::HTTPMessage::CONTENT_TYPE, default_content_type}}};
}
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup)
{
if (setup)
for (const auto & [header_name, header_value] : *setup)
response.set(header_name, header_value);
}
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map<String, String> & setup)
{
for (const auto & [header_name, header_value] : setup)
response.set(header_name, header_value);
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <optional>
#include <string>
#include <unordered_map>
#include <base/types.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
using HTTPResponseHeaderSetup = std::optional<std::unordered_map<String, String>>;
HTTPResponseHeaderSetup parseHTTPResponseHeaders(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
std::unordered_map<String, String> parseHTTPResponseHeaders(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_content_type);
std::unordered_map<String, String> parseHTTPResponseHeaders(const std::string & default_content_type);
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const HTTPResponseHeaderSetup & setup);
void applyHTTPResponseHeaders(Poco::Net::HTTPResponse & response, const std::unordered_map<String, String> & setup);
}

View File

@ -2,7 +2,7 @@
#include "IServer.h"
#include "HTTPHandlerFactory.h"
#include "HTTPHandlerRequestFilter.h"
#include "HTTPResponseHeaderWriter.h"
#include <IO/HTTPCommon.h>
#include <IO/ReadBufferFromFile.h>
@ -14,6 +14,7 @@
#include <Common/Exception.h>
#include <unordered_map>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
#include <Poco/Net/HTTPRequestHandlerFactory.h>
@ -94,7 +95,7 @@ void StaticRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServer
try
{
response.setContentType(content_type);
applyHTTPResponseHeaders(response, http_response_headers_override);
if (request.getVersion() == Poco::Net::HTTPServerRequest::HTTP_1_1)
response.setChunkedTransferEncoding(true);
@ -155,8 +156,9 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out)
writeString(response_expression, out);
}
StaticRequestHandler::StaticRequestHandler(IServer & server_, const String & expression, int status_, const String & content_type_)
: server(server_), status(status_), content_type(content_type_), response_expression(expression)
StaticRequestHandler::StaticRequestHandler(
IServer & server_, const String & expression, const std::unordered_map<String, String> & http_response_headers_override_, int status_)
: server(server_), status(status_), http_response_headers_override(http_response_headers_override_), response_expression(expression)
{
}
@ -166,12 +168,12 @@ HTTPRequestHandlerFactoryPtr createStaticHandlerFactory(IServer & server,
{
int status = config.getInt(config_prefix + ".handler.status", 200);
std::string response_content = config.getRawString(config_prefix + ".handler.response_content", "Ok.\n");
std::string response_content_type = config.getString(config_prefix + ".handler.content_type", "text/plain; charset=UTF-8");
auto creator = [&server, response_content, status, response_content_type]() -> std::unique_ptr<StaticRequestHandler>
{
return std::make_unique<StaticRequestHandler>(server, response_content, status, response_content_type);
};
std::unordered_map<String, String> http_response_headers_override
= parseHTTPResponseHeaders(config, config_prefix, "text/plain; charset=UTF-8");
auto creator = [&server, http_response_headers_override, response_content, status]() -> std::unique_ptr<StaticRequestHandler>
{ return std::make_unique<StaticRequestHandler>(server, response_content, http_response_headers_override, status); };
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<StaticRequestHandler>>(std::move(creator));

View File

@ -1,9 +1,9 @@
#pragma once
#include <unordered_map>
#include <Server/HTTP/HTTPRequestHandler.h>
#include <base/types.h>
namespace DB
{
@ -17,15 +17,16 @@ private:
IServer & server;
int status;
String content_type;
/// Overrides for response headers.
std::unordered_map<String, String> http_response_headers_override;
String response_expression;
public:
StaticRequestHandler(
IServer & server,
const String & expression,
int status_ = 200,
const String & content_type_ = "text/html; charset=UTF-8");
const std::unordered_map<String, String> & http_response_headers_override_,
int status_ = 200);
void writeResponse(WriteBuffer & out);

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBuffer.h>
@ -24,7 +25,6 @@
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
#include "Parsers/ASTSetQuery.h"
#include <Core/Defines.h>
#include <Compression/CompressionFactory.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -207,6 +207,13 @@ void ColumnDescription::readText(ReadBuffer & buf)
if (col_ast->settings)
settings = col_ast->settings->as<ASTSetQuery &>().changes;
if (col_ast->statistics_desc)
{
statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast, type);
/// every column has name `x` here, so we have to set the name manually.
statistics.column_name = name;
}
}
else
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description");

View File

@ -1,7 +1,4 @@
// Needs to go first because its partial specialization of fmt::formatter
// should be defined before any instantiation
#include <fmt/ostream.h>
#include <fmt/ranges.h>
#include <Storages/Kafka/KafkaConsumer.h>
#include <IO/ReadBufferFromMemory.h>

View File

@ -1,12 +1,14 @@
#pragma once
#include <boost/circular_buffer.hpp>
#include <fmt/ostream.h>
#include <Core/Names.h>
#include <base/types.h>
#include <IO/ReadBuffer.h>
#include <cppkafka/cppkafka.h>
#include <cppkafka/topic_partition.h>
#include <Common/CurrentMetrics.h>
namespace CurrentMetrics
@ -197,3 +199,6 @@ private:
};
}
template <> struct fmt::formatter<cppkafka::TopicPartition> : fmt::ostream_formatter {};
template <> struct fmt::formatter<cppkafka::Error> : fmt::ostream_formatter {};

View File

@ -69,7 +69,7 @@ struct fmt::formatter<DB::MarkRange>
}
template <typename FormatContext>
auto format(const DB::MarkRange & range, FormatContext & ctx)
auto format(const DB::MarkRange & range, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", fmt::format("({}, {})", range.begin, range.end));
}

View File

@ -112,7 +112,7 @@ struct fmt::formatter<DB::Part>
static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); }
template <typename FormatContext>
auto format(const DB::Part & part, FormatContext & ctx)
auto format(const DB::Part & part, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{} in replicas [{}]", part.description.describe(), fmt::join(part.replicas, ", "));
}

View File

@ -13,7 +13,7 @@ struct fmt::formatter<DB::RangesInDataPartDescription>
static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); }
template <typename FormatContext>
auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx)
auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx) const
{
return fmt::format_to(ctx.out(), "{}", range.describe());
}

View File

@ -158,7 +158,7 @@ struct fmt::formatter<DB::NamedCollectionValidateKey<T>>
}
template <typename FormatContext>
auto format(const DB::NamedCollectionValidateKey<T> & elem, FormatContext & context)
auto format(const DB::NamedCollectionValidateKey<T> & elem, FormatContext & context) const
{
return fmt::format_to(context.out(), "{}", elem.value);
}

View File

@ -132,11 +132,11 @@ void WriteBufferFromHDFS::sync()
}
void WriteBufferFromHDFS::finalizeImpl()
WriteBufferFromHDFS::~WriteBufferFromHDFS()
{
try
{
next();
finalize();
}
catch (...)
{
@ -144,11 +144,5 @@ void WriteBufferFromHDFS::finalizeImpl()
}
}
WriteBufferFromHDFS::~WriteBufferFromHDFS()
{
finalize();
}
}
#endif

View File

@ -38,8 +38,6 @@ public:
std::string getFileName() const override { return filename; }
private:
void finalizeImpl() override;
struct WriteBufferFromHDFSImpl;
std::unique_ptr<WriteBufferFromHDFSImpl> impl;
const std::string filename;

View File

@ -83,7 +83,6 @@ void StorageObjectStorageSink::finalize()
{
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
@ -91,6 +90,8 @@ void StorageObjectStorageSink::finalize()
release();
throw;
}
write_buf->finalize();
}
void StorageObjectStorageSink::release()

View File

@ -169,9 +169,9 @@ std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(co
return result;
}
ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column)
ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type)
{
const auto & stat_type_list_ast = column.stat_type->as<ASTFunction &>().arguments;
const auto & stat_type_list_ast = column.statistics_desc->as<ASTFunction &>().arguments;
if (stat_type_list_ast->children.empty())
throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column));
ColumnStatisticsDescription stats;
@ -185,7 +185,7 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c
throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type);
stats.types_to_desc.emplace(stat.type, std::move(stat));
}
stats.data_type = data_type;
return stats;
}

View File

@ -55,7 +55,7 @@ struct ColumnStatisticsDescription
ASTPtr getAST() const;
static std::vector<ColumnStatisticsDescription> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type);
using StatisticsTypeDescMap = std::map<StatisticsType, SingleStatisticsDescription>;
StatisticsTypeDescMap types_to_desc;

View File

@ -1823,7 +1823,6 @@ private:
{
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
@ -1831,12 +1830,14 @@ private:
release();
throw;
}
write_buf->finalize();
}
void release()
{
writer.reset();
write_buf->finalize();
write_buf.reset();
}
StorageMetadataPtr metadata_snapshot;

View File

@ -609,7 +609,6 @@ void StorageURLSink::finalize()
{
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
@ -617,12 +616,14 @@ void StorageURLSink::finalize()
release();
throw;
}
write_buf->finalize();
}
void StorageURLSink::release()
{
writer.reset();
write_buf->finalize();
write_buf.reset();
}
class PartitionedStorageURLSink : public PartitionedSink

View File

@ -550,7 +550,7 @@ class Release:
def _create_tag(
self, tag: str, commit: str, tag_message: str = ""
) -> Iterator[None]:
tag_message = tag_message or "Release {tag}"
tag_message = tag_message or f"Release {tag}"
# Create tag even in dry-run
self.run(f"git tag -a -m '{tag_message}' '{tag}' {commit}")
rollback_cmd = f"git tag -d '{tag}'"

View File

@ -0,0 +1,3 @@
<clickhouse>
<asynchronous_metrics_update_period_s>1</asynchronous_metrics_update_period_s>
</clickhouse>

View File

@ -0,0 +1,73 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1",
main_configs=["configs/asynchronous_metrics_update_period_s.xml"],
env_variables={"MALLOC_CONF": "background_thread:true,prof:true"},
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
# asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting
# asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and
# are available for querying during the test.
def test_asynchronous_metric_jemalloc_profile_active(started_cluster):
# default open
if node1.is_built_with_sanitizer():
pytest.skip("Disabled for sanitizers")
res_o = node1.query(
"SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;"
)
assert (
res_o
== """Row 1:
metric: jemalloc.prof.active
value: 1
description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html
"""
)
# disable
node1.query("SYSTEM JEMALLOC DISABLE PROFILE")
time.sleep(5)
res_t = node1.query(
"SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;"
)
assert (
res_t
== """Row 1:
metric: jemalloc.prof.active
value: 0
description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html
"""
)
# enable
node1.query("SYSTEM JEMALLOC ENABLE PROFILE")
time.sleep(5)
res_f = node1.query(
"SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;"
)
assert (
res_f
== """Row 1:
metric: jemalloc.prof.active
value: 1
description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html
"""
)

View File

@ -168,6 +168,32 @@ def test_restore_table(engine):
assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n"
def test_restore_materialized_view_with_definer():
instance.query("CREATE DATABASE test")
instance.query(
"CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s"
)
instance.query("CREATE USER u1")
instance.query("GRANT SELECT ON *.* TO u1")
instance.query("GRANT INSERT ON *.* TO u1")
instance.query(
"""
CREATE MATERIALIZED VIEW test.test_mv_1 (s String)
ENGINE = MergeTree ORDER BY s
DEFINER = u1 SQL SECURITY DEFINER
AS SELECT * FROM test.test_table
"""
)
backup_name = new_backup_name()
instance.query(f"BACKUP DATABASE test TO {backup_name}")
instance.query("DROP DATABASE test")
instance.query("DROP USER u1")
instance.query(f"RESTORE DATABASE test FROM {backup_name}")
@pytest.mark.parametrize(
"engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"]
)

View File

@ -88,6 +88,11 @@ def test_dynamic_query_handler():
"application/whatever; charset=cp1337"
== res_custom_ct.headers["content-type"]
)
assert "it works" == res_custom_ct.headers["X-Test-Http-Response-Headers-Works"]
assert (
"also works"
== res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"]
)
def test_predefined_query_handler():
@ -146,6 +151,10 @@ def test_predefined_query_handler():
)
assert b"max_final_threads\t1\nmax_threads\t1\n" == res2.content
assert "application/generic+one" == res2.headers["content-type"]
assert "it works" == res2.headers["X-Test-Http-Response-Headers-Works"]
assert (
"also works" == res2.headers["X-Test-Http-Response-Headers-Even-Multiple"]
)
cluster.instance.query(
"CREATE TABLE test_table (id UInt32, data String) Engine=TinyLog"
@ -212,6 +221,18 @@ def test_fixed_static_handler():
"test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"}
).content
)
assert (
"it works"
== cluster.instance.http_request(
"test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"}
).headers["X-Test-Http-Response-Headers-Works"]
)
assert (
"also works"
== cluster.instance.http_request(
"test_get_fixed_static_handler", method="GET", headers={"XXX": "xxx"}
).headers["X-Test-Http-Response-Headers-Even-Multiple"]
)
def test_config_static_handler():

View File

@ -18,6 +18,10 @@
<type>dynamic_query_handler</type>
<query_param_name>get_dynamic_handler_query</query_param_name>
<content_type>application/whatever; charset=cp1337</content_type>
<http_response_headers>
<X-Test-Http-Response-Headers-Works>it works</X-Test-Http-Response-Headers-Works>
<X-Test-Http-Response-Headers-Even-Multiple>also works</X-Test-Http-Response-Headers-Even-Multiple>
</http_response_headers>
</handler>
</rule>
</http_handlers>

View File

@ -19,6 +19,10 @@
<type>predefined_query_handler</type>
<query>SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String}</query>
<content_type>application/generic+one</content_type>
<http_response_headers>
<X-Test-Http-Response-Headers-Works>it works</X-Test-Http-Response-Headers-Works>
<X-Test-Http-Response-Headers-Even-Multiple>also works</X-Test-Http-Response-Headers-Even-Multiple>
</http_response_headers>
</handler>
</rule>
<rule>

View File

@ -12,6 +12,10 @@
<status>402</status>
<content_type>text/html; charset=UTF-8</content_type>
<response_content>Test get static handler and fix content</response_content>
<http_response_headers>
<X-Test-Http-Response-Headers-Works>it works</X-Test-Http-Response-Headers-Works>
<X-Test-Http-Response-Headers-Even-Multiple>also works</X-Test-Http-Response-Headers-Even-Multiple>
</http_response_headers>
</handler>
</rule>

View File

@ -6,7 +6,11 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", user_configs=["config/config.xml"], with_zookeeper=False
"node1", user_configs=["config/config.xml"], with_zookeeper=True
)
node2 = cluster.add_instance(
"node2", user_configs=["config/config.xml"], with_zookeeper=True
)
@ -122,3 +126,58 @@ def test_single_node_normal(started_cluster):
"""
)
run_test_single_node(started_cluster)
def test_replicated_table_ddl(started_cluster):
node1.query("DROP TABLE IF EXISTS test_stat")
node2.query("DROP TABLE IF EXISTS test_stat")
node1.query(
"""
CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest))
ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '1') ORDER BY a;
"""
)
node2.query(
"""
CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest))
ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '2') ORDER BY a;
"""
)
node1.query(
"ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq",
settings={"alter_sync": "2"},
)
node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync": "2"})
assert (
node2.query("SHOW CREATE TABLE test_stat")
== "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n"
)
node2.query("insert into test_stat values(1,2,3), (2,3,4)")
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True)
node1.query(
"ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"}
)
assert node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n"
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True)
node1.query(
"ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"}
)
node1.query(
"ALTER TABLE test_stat ADD STATISTICS b type tdigest",
settings={"alter_sync": "2"},
)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False)
node1.query(
"ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"}
)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True)

View File

@ -0,0 +1 @@
2

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -nq "
CREATE TABLE event_envoy
(
timestamp_interval DateTime CODEC(DoubleDelta),
region LowCardinality(String),
cluster LowCardinality(String)
)
ENGINE = MergeTree
ORDER BY (timestamp_interval)
SETTINGS index_granularity = 8192;
INSERT INTO event_envoy SELECT now() - number, 'us-east-1', 'ch_super_fast' FROM numbers_mt(1e5);
"
${CLICKHOUSE_CLIENT} -nq "
CREATE TABLE event_envoy_remote
(
timestamp_interval DateTime CODEC(DoubleDelta),
region LowCardinality(String),
cluster LowCardinality(String)
) AS remote('127.0.0.1', '${CLICKHOUSE_DATABASE}', event_envoy);
"
${CLICKHOUSE_CLIENT} -q "
CREATE TABLE global_event_envoy
(
timestamp_interval DateTime,
region LowCardinality(String),
cluster LowCardinality(String)
)
ENGINE = Merge('${CLICKHOUSE_DATABASE}', 'event_envoy.*');
"
${CLICKHOUSE_CLIENT} --prefer_localhost_replica 1 -q "
EXPLAIN indexes=1
SELECT timestamp_interval
FROM global_event_envoy
WHERE timestamp_interval <= now() - 54321 AND region = 'us-east-1'
" | grep -c 'Condition.*timestamp_interval'

View File

@ -0,0 +1,11 @@
-- Tags: no-parallel
DROP DATABASE IF EXISTS 03147_db;
CREATE DATABASE IF NOT EXISTS 03147_db;
CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n;
INSERT INTO 03147_db.t SELECT * FROM numbers(10);
USE 03147_db;
SET allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 100;
SELECT * FROM loop(03147_db.t) LIMIT 15 FORMAT Null;

View File

@ -0,0 +1,2 @@
123 123
456 456

View File

@ -0,0 +1,6 @@
-- This query succeeds only if it is correctly optimized.
SET allow_experimental_analyzer = 1;
SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (123, 456) AND t2.number = t1.number ORDER BY ALL;
-- Still TODO:
-- SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (SELECT 123 UNION ALL SELECT 456) AND t2.number = t1.number ORDER BY ALL;

View File

@ -0,0 +1,3 @@
-- Tags: no-fasttest
DROP USER IF EXISTS 03172_user_invalid_bcrypt_hash;
CREATE USER 03172_user_invalid_bcrypt_hash IDENTIFIED WITH bcrypt_hash BY '012345678901234567890123456789012345678901234567890123456789'; -- { serverError BAD_ARGUMENTS }

View File

@ -37,7 +37,7 @@ function wait_for_all_mutations()
echo "Timed out while waiting for mutation to execute!"
fi
sleep 0.1
sleep 0.3
done
}

View File

@ -41,6 +41,7 @@ v23.9.4.11-stable 2023-11-08
v23.9.3.12-stable 2023-10-31
v23.9.2.56-stable 2023-10-19
v23.9.1.1854-stable 2023-09-29
v23.8.15.35-lts 2024-06-14
v23.8.14.6-lts 2024-05-02
v23.8.13.25-lts 2024-04-26
v23.8.12.13-lts 2024-03-26

1 v24.5.3.5-stable 2024-06-13
41 v23.9.3.12-stable 2023-10-31
42 v23.9.2.56-stable 2023-10-19
43 v23.9.1.1854-stable 2023-09-29
44 v23.8.15.35-lts 2024-06-14
45 v23.8.14.6-lts 2024-05-02
46 v23.8.13.25-lts 2024-04-26
47 v23.8.12.13-lts 2024-03-26