From 586b1df3ee3f683e61c92e55d85e31f163eaa5af Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 6 Aug 2024 19:38:54 +0200 Subject: [PATCH 01/12] Add keeper error description to the message --- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2d826c6c2df..b85c7145a91 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -800,7 +800,8 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() { auto res = future.get(); if (res.error != Coordination::Error::ZOK && res.error != Coordination::Error::ZNODEEXISTS) - throw Coordination::Exception(res.error, "Failed to create new nodes {} at {}", res.path_created, zookeeper_path); + throw Coordination::Exception(res.error, "Failed to create new nodes {} at {} with error {}", + res.path_created, zookeeper_path, Coordination::errorMessage(res.error)); } } From 0882b810a753db4ed7643a2fed3f94fea785ae4d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 01:04:37 +0000 Subject: [PATCH 02/12] try to make bigger steps for execution of merges --- src/Storages/MergeTree/MergeTask.cpp | 54 ++++++++++++++-------- src/Storages/MergeTree/MergeTask.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + 3 files changed, 37 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5f178f08ec3..3203120c6eb 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -12,6 +12,7 @@ #include #include #include +#include "base/types.h" #include #include #include @@ -512,11 +513,20 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { - Block block; - if (!ctx->is_cancelled() && (global_ctx->merging_executor->pull(block))) - { - global_ctx->rows_written += block.rows(); + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + do + { + Block block; + + if (ctx->is_cancelled() || !global_ctx->merging_executor->pull(block)) + { + finalize(); + return false; + } + + global_ctx->rows_written += block.rows(); const_cast(*global_ctx->to).write(block); UInt64 result_rows = 0; @@ -536,11 +546,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() global_ctx->space_reservation->update(static_cast((1. - progress) * ctx->initial_reservation)); } + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } + /// Need execute again + return true; +} +void MergeTask::ExecuteAndFinalizeHorizontalPart::finalize() const +{ global_ctx->merging_executor.reset(); global_ctx->merged_pipeline.reset(); @@ -550,14 +563,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts with expired TTL"); - const auto data_settings = global_ctx->data->getSettings(); const size_t sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; - ctx->need_sync = needSyncPart(ctx->sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); - - return false; + ctx->need_sync = needSyncPart(ctx->sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *global_ctx->data->getSettings()); } - bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const { /// No need to execute this part if it is horizontal merge. @@ -734,17 +743,24 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const bool MergeTask::VerticalMergeStage::executeVerticalMergeForOneColumn() const { - Block block; - if (!global_ctx->merges_blocker->isCancelled() && !global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed) - && ctx->executor->pull(block)) + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + + do { + Block block; + + if (global_ctx->merges_blocker->isCancelled() + || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed) + || !ctx->executor->pull(block)) + return false; + ctx->column_elems_written += block.rows(); ctx->column_to->write(block); + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } - return false; + /// Need execute again + return true; } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 979c85482e5..24917a4cb0e 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -254,6 +254,7 @@ private: bool prepare(); bool executeImpl(); + void finalize() const; /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5ba1988cc5d..e56bb15f7b0 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,6 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ + M(Milliseconds, merge_preferred_step_execution_time_ms, 100, "Target time to execetion of one step of merge. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From 9f44cd85925b3d691afd71a9ecb9ccc9ec3bf57a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 14 Aug 2024 22:19:17 +0000 Subject: [PATCH 03/12] fix typo --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6c9a29c5bb9..149f9a3e80b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,7 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ - M(Milliseconds, merge_preferred_step_execution_time_ms, 100, "Target time to execetion of one step of merge. Can be exceeded if one step takes longer time", 0) \ + M(Milliseconds, merge_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From 75f951dae57c4de27cdba28792622a35597cf31c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 15 Aug 2024 17:59:31 +0000 Subject: [PATCH 04/12] limit task time for mutations --- src/Storages/MergeTree/MergeTask.cpp | 5 +- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 88 +++++++++++++--------- 3 files changed, 57 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ee15bc7f711..26cb821f33b 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -9,7 +9,6 @@ #include #include #include -#include "base/types.h" #include #include #include @@ -521,7 +520,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { Stopwatch watch(CLOCK_MONOTONIC_COARSE); - UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); do { @@ -751,7 +750,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const bool MergeTask::VerticalMergeStage::executeVerticalMergeForOneColumn() const { Stopwatch watch(CLOCK_MONOTONIC_COARSE); - UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); do { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 149f9a3e80b..de1f0f60cfc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,7 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ - M(Milliseconds, merge_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge. Can be exceeded if one step takes longer time", 0) \ + M(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 14c274d7f64..40648439887 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1257,6 +1257,8 @@ public: private: void prepare(); bool mutateOriginalPartAndPrepareProjections(); + void writeTempProjectionPart(size_t projection_idx, Chunk chunk); + void finalizeTempProjections(); bool iterateThroughAllProjections(); void constructTaskForProjectionPartsMerge(); void finalize(); @@ -1307,10 +1309,22 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { - Block cur_block; - Block projection_header; - if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); + + do { + Block cur_block; + Block projection_header; + + MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry); + + if (!ctx->mutating_executor->pull(cur_block)) + { + finalizeTempProjections(); + return false; + } + if (ctx->minmax_idx) ctx->minmax_idx->update(cur_block, MergeTreeData::getMinMaxColumnsNames(ctx->metadata_snapshot->getPartitionKey())); @@ -1322,46 +1336,56 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - const auto & projection = *ctx->projections_to_build[i]; + Chunk squashed_chunk; - ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); - Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); - - Chunk squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); - if (squashed_chunk) { - auto result = projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); - tmp_part.finalize(); - tmp_part.part->getDataPartStorage().commitTransaction(); - projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); + ProfileEventTimeIncrement projection_watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); + Block block_to_squash = ctx->projections_to_build[i]->calculate(cur_block, ctx->context); + + projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); + squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); } + + if (squashed_chunk) + writeTempProjectionPart(i, std::move(squashed_chunk)); } (*ctx->mutate_entry)->rows_written += cur_block.rows(); (*ctx->mutate_entry)->bytes_written_uncompressed += cur_block.bytes(); + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } + /// Need execute again + return true; +} +void PartMergerWriter::writeTempProjectionPart(size_t projection_idx, Chunk chunk) +{ + const auto & projection = *ctx->projections_to_build[projection_idx]; + const auto & projection_plan = projection_squashes[projection_idx]; + + auto result = projection_plan.getHeader().cloneWithColumns(chunk.detachColumns()); + + auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( + *ctx->data, + ctx->log, + result, + projection, + ctx->new_data_part.get(), + ++block_num); + + tmp_part.finalize(); + tmp_part.part->getDataPartStorage().commitTransaction(); + projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); +} + +void PartMergerWriter::finalizeTempProjections() +{ // Write the last block for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - const auto & projection = *ctx->projections_to_build[i]; - auto & projection_squash_plan = projection_squashes[i]; - auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + auto squashed_chunk = Squashing::squash(projection_squashes[i].flush()); if (squashed_chunk) - { - auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); - temp_part.finalize(); - temp_part.part->getDataPartStorage().commitTransaction(); - projection_parts[projection.name].emplace_back(std::move(temp_part.part)); - } + writeTempProjectionPart(i, std::move(squashed_chunk)); } projection_parts_iterator = std::make_move_iterator(projection_parts.begin()); @@ -1369,12 +1393,8 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() /// Maybe there are no projections ? if (projection_parts_iterator != std::make_move_iterator(projection_parts.end())) constructTaskForProjectionPartsMerge(); - - /// Let's move on to the next stage - return false; } - void PartMergerWriter::constructTaskForProjectionPartsMerge() { auto && [name, parts] = *projection_parts_iterator; From fb7afa779c7ea7d56c67da10a634133fe8416f97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 03:59:17 +0200 Subject: [PATCH 05/12] Support expressions with tuples like `expr().name` --- src/Parsers/ExpressionListParsers.cpp | 17 ++++++++++++++--- .../03224_tuple_element_identifier.reference | 4 ++++ .../03224_tuple_element_identifier.sql | 13 +++++++++++++ 3 files changed, 31 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03224_tuple_element_identifier.reference create mode 100644 tests/queries/0_stateless/03224_tuple_element_identifier.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index d38dc6d5f37..ad6b8e13ea6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2811,8 +2811,8 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po if (op.type == OperatorType::TupleElement) { ASTPtr tmp; - if (asterisk_parser.parse(pos, tmp, expected) || - columns_matcher_parser.parse(pos, tmp, expected)) + if (asterisk_parser.parse(pos, tmp, expected) + || columns_matcher_parser.parse(pos, tmp, expected)) { if (auto * asterisk = tmp->as()) { @@ -2833,6 +2833,17 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po layers.back()->pushOperand(std::move(tmp)); return Action::OPERATOR; } + + /// If it is an identifier, + /// replace it with literal, because an expression `expr().elem` + /// should be transformed to `tupleElement(expr(), 'elem')` for query analysis, + /// otherwise the identifier `elem` will not be found. + if (ParserIdentifier().parse(pos, tmp, expected)) + { + layers.back()->pushOperator(op); + layers.back()->pushOperand(std::make_shared(tmp->as()->name())); + return Action::OPERATOR; + } } /// isNull & isNotNull are postfix unary operators @@ -2863,7 +2874,7 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po layers.push_back(std::make_unique()); if (op.type == OperatorType::StartBetween || op.type == OperatorType::StartNotBetween) - layers.back()->between_counter++; + ++layers.back()->between_counter; return Action::OPERAND; } diff --git a/tests/queries/0_stateless/03224_tuple_element_identifier.reference b/tests/queries/0_stateless/03224_tuple_element_identifier.reference new file mode 100644 index 00000000000..0fc9e7410c1 --- /dev/null +++ b/tests/queries/0_stateless/03224_tuple_element_identifier.reference @@ -0,0 +1,4 @@ +([('wtf')]) [('wtf')] wtf +([('wtf')]) [('wtf')] wtf +Hello +('Hello') Hello Hello Hello diff --git a/tests/queries/0_stateless/03224_tuple_element_identifier.sql b/tests/queries/0_stateless/03224_tuple_element_identifier.sql new file mode 100644 index 00000000000..2a7fb9a97a3 --- /dev/null +++ b/tests/queries/0_stateless/03224_tuple_element_identifier.sql @@ -0,0 +1,13 @@ +SET enable_analyzer = 1; + +SELECT JSONExtract('{"hello":[{"world":"wtf"}]}', 'Tuple(hello Array(Tuple(world String)))') AS x, + x.hello, x.hello[1].world; + +SELECT JSONExtract('{"hello":[{" wow ":"wtf"}]}', 'Tuple(hello Array(Tuple(` wow ` String)))') AS x, + x.hello, x.hello[1].` wow `; + +SELECT JSONExtract('{"hello":[{" wow ":"wtf"}]}', 'Tuple(hello Array(Tuple(` wow ` String)))') AS x, + x.hello, x.hello[1].`wow`; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK } + +SELECT ('Hello' AS world,).world; +SELECT ('Hello' AS world,) AS t, t.world, (t).world, identity(t).world; From b1c18e2dd42051fb045431278eceae9dd22563bd Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:29:25 +0000 Subject: [PATCH 06/12] Fix check for allowed date symbols in date time best effort schema infernece --- src/IO/parseDateTimeBestEffort.cpp | 4 ++-- tests/queries/0_stateless/03222_date_time_inference.reference | 1 + tests/queries/0_stateless/03222_date_time_inference.sql | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 8d798bf725c..03429b46b2a 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -350,7 +350,7 @@ ReturnType parseDateTimeBestEffortImpl( if (month && !day_of_month) day_of_month = hour_or_day_of_month_or_month; } - else if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) + else if ((!in.eof() && isSymbolIn(*in.position(), allowed_date_delimiters)) && (checkChar('/', in) || checkChar('.', in) || checkChar('-', in))) { if (day_of_month) return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: day of month is duplicated"); @@ -399,7 +399,7 @@ ReturnType parseDateTimeBestEffortImpl( if (month > 12) std::swap(month, day_of_month); - if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) + if ((!in.eof() && isSymbolIn(*in.position(), allowed_date_delimiters)) && (checkChar('/', in) || checkChar('.', in) || checkChar('-', in))) { if (year) return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: year component is duplicated"); diff --git a/tests/queries/0_stateless/03222_date_time_inference.reference b/tests/queries/0_stateless/03222_date_time_inference.reference index 3288308a1d0..838b103f106 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.reference +++ b/tests/queries/0_stateless/03222_date_time_inference.reference @@ -251,3 +251,4 @@ Mar 2000 00:00:00.000 String 2000 00:00:00.000 String Mar 2000-01-01 00:00:00 String Mar 2000-01-01 00:00:00.000 String +1.7.10 String diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql index ebd472294be..b0c4df530cf 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.sql +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -265,5 +265,5 @@ select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00.000"}'); - +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "1.7.10"}'); From 4975264c9d39d560c32799dcf14bd3f9d40fcfa4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 14:08:07 +0000 Subject: [PATCH 07/12] More strict parsing --- src/IO/parseDateTimeBestEffort.cpp | 23 ++++--- .../03222_date_time_inference.reference | 67 ++++++++++++------- .../0_stateless/03222_date_time_inference.sql | 19 ++++++ 3 files changed, 75 insertions(+), 34 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 03429b46b2a..52bcdc6bbb4 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -180,8 +180,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 10 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow timestamps"); /// This is unix timestamp. readDecimalNumber<10>(res, digits); @@ -189,8 +189,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 9 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow timestamps"); /// This is unix timestamp. readDecimalNumber<9>(res, digits); @@ -198,8 +198,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 14 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); /// This is YYYYMMDDhhmmss readDecimalNumber<4>(year, digits); @@ -212,8 +212,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 8 && !year) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); /// This is YYYYMMDD readDecimalNumber<4>(year, digits); @@ -222,6 +222,9 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 6) { + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); + /// This is YYYYMM or hhmmss if (!year && !month) { @@ -593,8 +596,8 @@ ReturnType parseDateTimeBestEffortImpl( else return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: unexpected word"); -// while (!in.eof() && isAlphaASCII(*in.position())) -// ++in.position(); + while (!in.eof() && isAlphaASCII(*in.position())) + ++in.position(); /// For RFC 2822 if (has_day_of_week) diff --git a/tests/queries/0_stateless/03222_date_time_inference.reference b/tests/queries/0_stateless/03222_date_time_inference.reference index 838b103f106..221ab1fe5f5 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.reference +++ b/tests/queries/0_stateless/03222_date_time_inference.reference @@ -45,34 +45,34 @@ DateTime/DateTime64 best effort 2000-01-01 00:00:00 DateTime 2000-01-01 01:00:00 DateTime 2000-01-01 01:00:00.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) -2017-01-01 21:02:03 DateTime -2017-01-01 21:02:03.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) -2017-01-02 01:02:03 DateTime -2017-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) +02/01/17 010203 MSK String +02/01/17 010203.000 MSK String +02/01/17 010203 MSK+0100 String +02/01/17 010203.000 MSK+0100 String +02/01/17 010203 UTC+0300 String +02/01/17 010203.000 UTC+0300 String +02/01/17 010203Z String +02/01/17 010203.000Z String +02/01/1970 010203Z String +02/01/1970 010203.000Z String +02/01/70 010203Z String +02/01/70 010203.000Z String 2018-02-11 03:40:50 DateTime 2018-02-11 03:40:50.000000000 DateTime64(9) 2000-04-17 01:02:03 DateTime 2000-04-17 01:02:03.000000000 DateTime64(9) -1970-01-02 01:00:00 DateTime -1970-01-02 01:00:00.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) +19700102 01:00:00 String +19700102 01:00:00.000 String +19700102010203Z String +19700102010203Z.000 String +1970/01/02 010203Z String +1970/01/02 010203.000Z String 2015-12-31 20:00:00 DateTime 2015-12-31 20:00:00 DateTime 2016-01-01 00:00:00 DateTime 2016-01-01 00:00:00 DateTime -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) +201701 02 010203 UTC+0300 String +201701 02 010203.000 UTC+0300 String 2017-01-02 03:04:05 DateTime 2017-01-02 03:04:05.000000000 DateTime64(9) 2017-01-02 03:04:05 DateTime @@ -117,8 +117,8 @@ DateTime/DateTime64 best effort 2017-01-02 03:04:05.000000000 DateTime64(9) 2017-04-01 11:22:33 DateTime 2017-04-01 11:22:33.000000000 DateTime64(9) -2017-04-01 22:02:03 DateTime -2017-04-01 22:02:03.000000000 DateTime64(9) +2017 Apr 02 010203 UTC+0300 String +2017 Apr 02 010203.000 UTC+0300 String 2017-04-01 22:02:03 DateTime 2017-04-01 22:02:03.000000000 DateTime64(9) 2017-04-02 01:02:03 DateTime @@ -143,8 +143,8 @@ DateTime/DateTime64 best effort 2017-04-01 21:02:03.000000000 DateTime64(9) 2017-04-02 01:02:03 DateTime 2017-04-02 01:02:03.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) +2017 Jan 02 010203 UTC+0300 String +2017 Jan 02 010203.000 UTC+0300 String 2017-04-25 01:02:03 DateTime 2017-04-25 01:02:03.000000000 DateTime64(9) 2017-04-25 01:02:03 DateTime @@ -231,6 +231,25 @@ Mar01012020010101 String Mar 01012020010101 String Mar01012020010101.000 String Mar 0101202001010101.000 String +Sun String +Sun1 String +Sun 1 String +Sun01 String +Sun 01 String +Sun2020 String +Sun 2020 String +Sun012020 String +Sun 012020 String +Sun01012020 String +Sun 01012020 String +Sun0101202001 String +Sun 0101202001 String +Sun010120200101 String +Sun 010120200101 String +Sun01012020010101 String +Sun 01012020010101 String +Sun01012020010101.000 String +Sun 0101202001010101.000 String 2000 01 01 01:00:00 String 2000 01 01 01:00:00.000 String 2000a01a01 01:00:00 String diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql index b0c4df530cf..b16f72c72f4 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.sql +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -245,6 +245,25 @@ select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 01012020010101"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 0101202001010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 0101202001010101.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000a01a01 01:00:00"}'); From 0dba461d7f9228fa81facdeabeffc983e181d9e0 Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Fri, 16 Aug 2024 16:46:57 +0100 Subject: [PATCH 08/12] Update gui.md: Links, formatting, spelling --- docs/en/interfaces/third-party/gui.md | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/interfaces/third-party/gui.md b/docs/en/interfaces/third-party/gui.md index 5b7615485ca..8d9dce983bc 100644 --- a/docs/en/interfaces/third-party/gui.md +++ b/docs/en/interfaces/third-party/gui.md @@ -10,7 +10,7 @@ sidebar_label: Visual Interfaces ### ch-ui {#ch-ui} -[ch-ui](https://github.com/caioricciuti/ch-ui) is a simple React.js app interface for ClickHouse databases, designed for executing queries and visualizing data. Built with React and the ClickHouse client for web, it offers a sleek and user-friendly UI for easy database interactions. +[ch-ui](https://github.com/caioricciuti/ch-ui) is a simple React.js app interface for ClickHouse databases designed for executing queries and visualizing data. Built with React and the ClickHouse client for web, it offers a sleek and user-friendly UI for easy database interactions. Features: @@ -25,7 +25,7 @@ Web interface for ClickHouse in the [Tabix](https://github.com/tabixio/tabix) pr Features: -- Works with ClickHouse directly from the browser, without the need to install additional software. +- Works with ClickHouse directly from the browser without the need to install additional software. - Query editor with syntax highlighting. - Auto-completion of commands. - Tools for graphical analysis of query execution. @@ -63,7 +63,7 @@ Features: - Table list with filtering and metadata. - Table preview with filtering and sorting. -- Read-only queries execution. +- Read-only query execution. ### Redash {#redash} @@ -75,23 +75,23 @@ Features: - Powerful editor of queries. - Database explorer. -- Visualization tools, that allow you to represent data in different forms. +- Visualization tool that allows you to represent data in different forms. ### Grafana {#grafana} [Grafana](https://grafana.com/grafana/plugins/grafana-clickhouse-datasource/) is a platform for monitoring and visualization. -"Grafana allows you to query, visualize, alert on and understand your metrics no matter where they are stored. Create, explore, and share dashboards with your team and foster a data driven culture. Trusted and loved by the community" — grafana.com. +"Grafana allows you to query, visualize, alert on and understand your metrics no matter where they are stored. Create, explore, and share dashboards with your team and foster a data-driven culture. Trusted and loved by the community" — grafana.com. -ClickHouse datasource plugin provides a support for ClickHouse as a backend database. +ClickHouse data source plugin provides support for ClickHouse as a backend database. -### qryn (#qryn) +### qryn {#qryn} [qryn](https://metrico.in) is a polyglot, high-performance observability stack for ClickHouse _(formerly cLoki)_ with native Grafana integrations allowing users to ingest and analyze logs, metrics and telemetry traces from any agent supporting Loki/LogQL, Prometheus/PromQL, OTLP/Tempo, Elastic, InfluxDB and many more. Features: -- Built in Explore UI and LogQL CLI for querying, extracting and visualizing data +- Built-in Explore UI and LogQL CLI for querying, extracting and visualizing data - Native Grafana APIs support for querying, processing, ingesting, tracing and alerting without plugins - Powerful pipeline to dynamically search, filter and extract data from logs, events, traces and beyond - Ingestion and PUSH APIs transparently compatible with LogQL, PromQL, InfluxDB, Elastic and many more @@ -139,7 +139,7 @@ Features: ### DBM {#dbm} -[DBM](https://dbm.incubator.edurt.io/) DBM is a visual management tool for ClickHouse! +[DBM](https://github.com/devlive-community/dbm) DBM is a visual management tool for ClickHouse! Features: @@ -151,7 +151,7 @@ Features: - Support custom query - Support multiple data sources management(connection test, monitoring) - Support monitor (processor, connection, query) -- Support migrate data +- Support migrating data ### Bytebase {#bytebase} @@ -169,7 +169,7 @@ Features: ### Zeppelin-Interpreter-for-ClickHouse {#zeppelin-interpreter-for-clickhouse} -[Zeppelin-Interpreter-for-ClickHouse](https://github.com/SiderZhang/Zeppelin-Interpreter-for-ClickHouse) is a [Zeppelin](https://zeppelin.apache.org) interpreter for ClickHouse. Compared with JDBC interpreter, it can provide better timeout control for long running queries. +[Zeppelin-Interpreter-for-ClickHouse](https://github.com/SiderZhang/Zeppelin-Interpreter-for-ClickHouse) is a [Zeppelin](https://zeppelin.apache.org) interpreter for ClickHouse. Compared with the JDBC interpreter, it can provide better timeout control for long-running queries. ### ClickCat {#clickcat} @@ -179,7 +179,7 @@ Features: - An online SQL editor which can run your SQL code without any installing. - You can observe all processes and mutations. For those unfinished processes, you can kill them in ui. -- The Metrics contains Cluster Analysis,Data Analysis,Query Analysis. +- The Metrics contain Cluster Analysis, Data Analysis, and Query Analysis. ### ClickVisual {#clickvisual} @@ -332,7 +332,7 @@ Learn more about the product at [TABLUM.IO](https://tablum.io/) ### CKMAN {#ckman} -[CKMAN] (https://www.github.com/housepower/ckman) is a tool for managing and monitoring ClickHouse clusters! +[CKMAN](https://www.github.com/housepower/ckman) is a tool for managing and monitoring ClickHouse clusters! Features: From fd266284e1a2c5170183f7e1d2425b4b2b6ca103 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 19:49:47 +0200 Subject: [PATCH 09/12] Fix test `02122_join_group_by_timeout` --- tests/queries/0_stateless/02122_join_group_by_timeout.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02122_join_group_by_timeout.sh b/tests/queries/0_stateless/02122_join_group_by_timeout.sh index b4644878544..17d8200f20c 100755 --- a/tests/queries/0_stateless/02122_join_group_by_timeout.sh +++ b/tests/queries/0_stateless/02122_join_group_by_timeout.sh @@ -15,6 +15,7 @@ fi # TCP CLIENT: As of today (02/12/21) uses PullingAsyncPipelineExecutor ### Should be cancelled after 1 second and return a 159 exception (timeout) +### However, in the test, the server can be overloaded, so we assert query duration in the interval of 1 to 60 seconds. query_id=$(random_str 12) $CLICKHOUSE_CLIENT --query_id "$query_id" --max_execution_time 1 -q " SELECT * FROM @@ -33,7 +34,7 @@ $CLICKHOUSE_CLIENT --query_id "$query_id" --max_execution_time 1 -q " FORMAT Null " 2>&1 | grep -m1 -o "Code: 159" $CLICKHOUSE_CLIENT -q "system flush logs" -${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) BETWEEN 1 AND 60 from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" ### Should stop pulling data and return what has been generated already (return code 0) @@ -52,7 +53,7 @@ $CLICKHOUSE_CLIENT --query_id "$query_id" -q " " echo $? $CLICKHOUSE_CLIENT -q "system flush logs" -${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" +${CLICKHOUSE_CURL} -q -sS "$CLICKHOUSE_URL" -d "select 'query_duration', round(query_duration_ms/1000) BETWEEN 1 AND 60 from system.query_log where current_database = '$CLICKHOUSE_DATABASE' and query_id = '$query_id' and type != 'QueryStart'" # HTTP CLIENT: As of today (02/12/21) uses PullingPipelineExecutor From faad7f4ba27f665ecdb4fa6212a695f86413746a Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 16 Aug 2024 20:35:44 +0000 Subject: [PATCH 10/12] Fix Broken pipe error for 03149_numbers_max_block_size_zero.sh The error appeared while debugging https://github.com/ClickHouse/clickhouse-private/issues/14225 Logs: https://pastila.nl/?00628486/754eaf7d96fd03ceecdf1a45458867dc#B9vFn07WAielph/Z5lHbrQ== From the `man grep`: > -q, --quiet, --silent > <...> Exit immediately with zero status if any match is found, even if an error was detected. When `grep -q` finds a match, it immediately exits with status `0 and closes its side of the pipe. If the clickhouse-client is still trying to send data through the pipe, it leads to SIGPIPE signal. Use grep -c instead. It is less efficient, but the output in this test is small. We should also revisit how we handle SIGPIPE signal, e.g. the server should not try to send logs if it already encountered the Broken pipe error. --- tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh index 6f70a0d2536..2c4669325bb 100755 --- a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh @@ -4,4 +4,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | grep -q "Sanity check: 'max_block_size' cannot be 0. Set to default value" && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | [ $(grep -c "Sanity check: 'max_block_size' cannot be 0. Set to default value") -gt 0 ] && echo "OK" || echo "FAIL" From 4b11522990f0f3557355a1123f877a9394c438d3 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Fri, 16 Aug 2024 14:06:55 -0700 Subject: [PATCH 11/12] Update JSON link --- docs/en/sql-reference/data-types/json.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index 12ab9f1dddc..e48b308a620 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -79,6 +79,5 @@ SELECT * FROM json FORMAT JSONEachRow ## Related Content -- [Using JSON in ClickHouse](/docs/en/integrations/data-formats/json) +- [Using JSON in ClickHouse](/en/integrations/data-formats/json/overview) - [Getting Data Into ClickHouse - Part 2 - A JSON detour](https://clickhouse.com/blog/getting-data-into-clickhouse-part-2-json) -- \ No newline at end of file From 2a5de86fe4f21d77671a317d52b3f378062c7555 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Fri, 16 Aug 2024 21:46:21 +0000 Subject: [PATCH 12/12] shellcheck --- .../queries/0_stateless/03149_numbers_max_block_size_zero.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh index 2c4669325bb..7f606d889a7 100755 --- a/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh +++ b/tests/queries/0_stateless/03149_numbers_max_block_size_zero.sh @@ -1,7 +1,9 @@ #!/usr/bin/env bash +# shellcheck disable=SC2266 CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | [ $(grep -c "Sanity check: 'max_block_size' cannot be 0. Set to default value") -gt 0 ] && echo "OK" || echo "FAIL" +$CLICKHOUSE_CLIENT -q "SELECT count(*) FROM numbers(10) AS a, numbers(11) AS b, numbers(12) AS c SETTINGS max_block_size = 0" 2>&1 | + [ "$(grep -c "Sanity check: 'max_block_size' cannot be 0. Set to default value")" -gt 0 ] && echo "OK" || echo "FAIL"