Merge remote-tracking branch 'origin/master' into pr-local-plan

This commit is contained in:
Igor Nikonov 2024-06-17 12:25:13 +00:00
commit 81668cc290
60 changed files with 501 additions and 117 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

@ -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

@ -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

@ -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

@ -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

@ -1871,7 +1871,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const
if (table.get()->isView() && table->as<StorageView>() && table->as<StorageView>()->isParameterizedView())
{
auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone();
NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression);
NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression, getQueryContext());
StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values);
ASTCreateQuery create;
@ -2085,7 +2085,7 @@ StoragePtr Context::buildParametrizedViewStorage(const ASTPtr & table_expression
return nullptr;
auto query = original_view->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone();
NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression);
NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression, getQueryContext());
StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values);
ASTCreateQuery create;

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,9 +210,8 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr
auto total = disk->getTotalSpace();
/// Some disks don't support information about the space.
if (!total)
continue;
if (total)
{
auto available = disk->getAvailableSpace();
auto unreserved = disk->getUnreservedSpace();
@ -232,6 +231,34 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr
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

@ -7,6 +7,7 @@
#include <Common/FieldVisitorToString.h>
#include <Parsers/ASTHelpers.h>
#include <Common/assert_cast.h>
#include <Interpreters/evaluateConstantExpression.h>
namespace DB
@ -20,8 +21,9 @@ namespace ErrorCodes
class FunctionParameterValuesVisitor
{
public:
explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_)
explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_, ContextPtr context_)
: parameter_values(parameter_values_)
, context(context_)
{
}
@ -35,6 +37,7 @@ public:
private:
NameToNameMap & parameter_values;
ContextPtr context;
void visitFunction(const ASTFunction & parameter_function)
{
@ -64,15 +67,20 @@ private:
parameter_values[identifier->name()] = convertFieldToString(cast_literal->value);
}
}
else
{
ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(expression_list->children[1], context);
parameter_values[identifier->name()] = convertFieldToString(res->as<ASTLiteral>()->value);
}
}
}
}
};
NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast)
NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast, ContextPtr context)
{
NameToNameMap parameter_values;
FunctionParameterValuesVisitor(parameter_values).visit(ast);
FunctionParameterValuesVisitor(parameter_values, context).visit(ast);
return parameter_values;
}

View File

@ -2,12 +2,13 @@
#include <Core/Names.h>
#include <Parsers/IAST_fwd.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{
/// Find parameters in a query parameter values and collect them into map.
NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast);
NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast, ContextPtr context);
}

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

@ -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

@ -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

@ -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

@ -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 2024-04-01 01:00:00

View File

@ -0,0 +1,14 @@
drop table if exists table_pv;
create table table_pv (id Int32, timestamp_field DateTime) engine = Memory();
insert into table_pv values(1, '2024-03-01 00:00:00');
insert into table_pv values (2, '2024-04-01 01:00:00');
create view pv as select * from table_pv where timestamp_field > {timestamp_param:DateTime};
select * from pv (timestamp_param=toDateTime('2024-04-01 00:00:01'));
select * from pv (timestamp_param=toDateTime('2024-040')); -- { serverError CANNOT_PARSE_DATETIME }
drop table table_pv;

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

@ -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