From e40ee1a173c86af5a7202d29bdde19dcc6c4d668 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 11 Jun 2020 06:45:12 +0300 Subject: [PATCH 01/47] Return non-Nullable results from COUNT(DISTINCT) --- .../AggregateFunctionCount.h | 6 +++ .../AggregateFunctionFactory.cpp | 10 ++-- .../AggregateFunctionNull.cpp | 47 +++++++++++++------ .../AggregateFunctionNull.h | 32 +++++++------ .../AggregateFunctionUniq.h | 12 +++++ .../AggregateFunctionWindowFunnel.h | 5 +- src/AggregateFunctions/IAggregateFunction.h | 6 +++ 7 files changed, 83 insertions(+), 35 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 092ffc6b6cf..e54f014f7a4 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -67,6 +67,12 @@ public: { data(place).count = new_count; } + + /// The function returns non-Nullable type even when wrapped with Null combinator. + bool returnDefaultWhenOnlyNull() const override + { + return true; + } }; diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index aeb4fb6db96..3982c48700b 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -63,14 +63,15 @@ AggregateFunctionPtr AggregateFunctionFactory::get( { auto type_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types); - /// If one of types is Nullable, we apply aggregate function combinator "Null". + /// If one of the types is Nullable, we apply aggregate function combinator "Null". if (std::any_of(type_without_low_cardinality.begin(), type_without_low_cardinality.end(), [](const auto & type) { return type->isNullable(); })) { AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null"); if (!combinator) - throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", + ErrorCodes::LOGICAL_ERROR); DataTypes nested_types = combinator->transformArguments(type_without_low_cardinality); Array nested_parameters = combinator->transformParameters(parameters); @@ -132,9 +133,10 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( auto hints = this->getHints(name); if (!hints.empty()) - throw Exception("Unknown aggregate function " + name + ". Maybe you meant: " + toString(hints), ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); + throw Exception(fmt::format("Unknown aggregate function {}. Maybe you meant: {}", name, toString(hints)), + ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); else - throw Exception("Unknown aggregate function " + name, ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); + throw Exception(fmt::format("Unknown aggregate function {}", name), ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION); } diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index 60712636562..77687f9f328 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -49,35 +49,52 @@ public: } if (!has_nullable_types) - throw Exception("Aggregate function combinator 'Null' requires at least one argument to be Nullable", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (nested_function) - if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params)) - return adapter; - - /// Special case for 'count' function. It could be called with Nullable arguments - /// - that means - count number of calls, when all arguments are not NULL. - if (nested_function && nested_function->getName() == "count") - return std::make_shared(arguments[0], params); + throw Exception("Aggregate function combinator 'Null' requires at least one argument to be Nullable", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (has_null_types) return std::make_shared(arguments, params); - bool return_type_is_nullable = nested_function->getReturnType()->canBeInsideNullable(); + assert(nested_function); + + if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params)) + return adapter; + + /// Special case for 'count' function. It could be called with Nullable arguments + /// - that means - count number of calls, when all arguments are not NULL. + if (nested_function->getName() == "count") + return std::make_shared(arguments[0], params); + + bool return_type_is_nullable = !nested_function->returnDefaultWhenOnlyNull() && nested_function->getReturnType()->canBeInsideNullable(); + bool serialize_flag = return_type_is_nullable || nested_function->returnDefaultWhenOnlyNull(); if (arguments.size() == 1) { if (return_type_is_nullable) - return std::make_shared>(nested_function, arguments, params); + { + return std::make_shared>(nested_function, arguments, params); + } else - return std::make_shared>(nested_function, arguments, params); + { + if (serialize_flag) + return std::make_shared>(nested_function, arguments, params); + else + return std::make_shared>(nested_function, arguments, params); + } } else { if (return_type_is_nullable) - return std::make_shared>(nested_function, arguments, params); + { + return std::make_shared>(nested_function, arguments, params); + } else - return std::make_shared>(nested_function, arguments, params); + { + if (serialize_flag) + return std::make_shared>(nested_function, arguments, params); + else + return std::make_shared>(nested_function, arguments, params); + } } } }; diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 55d610207f1..d6f0079232c 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -28,7 +28,10 @@ namespace ErrorCodes /// If all rows had NULL, the behaviour is determined by "result_is_nullable" template parameter. /// true - return NULL; false - return value from empty aggregation state of nested function. -template +/// When serialize_flag is set to true, the flag about presense of values is serialized +/// regardless to the "result_is_nullable" even if it's unneeded - for protocol compatibility. + +template class AggregateFunctionNullBase : public IAggregateFunctionHelper { protected: @@ -129,7 +132,7 @@ public: void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override { bool flag = getFlag(place); - if constexpr (result_is_nullable) + if constexpr (serialize_flag) writeBinary(flag, buf); if (flag) nested_function->serialize(nestedPlace(place), buf); @@ -138,7 +141,7 @@ public: void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override { bool flag = 1; - if constexpr (result_is_nullable) + if constexpr (serialize_flag) readBinary(flag, buf); if (flag) { @@ -183,12 +186,15 @@ public: /** There are two cases: for single argument and variadic. * Code for single argument is much more efficient. */ -template -class AggregateFunctionNullUnary final : public AggregateFunctionNullBase> +template +class AggregateFunctionNullUnary final + : public AggregateFunctionNullBase> { public: AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params) + : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params) { } @@ -218,12 +224,15 @@ public: }; -template -class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> +template +class AggregateFunctionNullVariadic final + : public AggregateFunctionNullBase> { public: AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) - : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), + : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { if (number_of_arguments == 1) @@ -263,11 +272,6 @@ public: this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); } - bool allocatesMemoryInArena() const override - { - return this->nested_function->allocatesMemoryInArena(); - } - private: enum { MAX_ARGS = 8 }; size_t number_of_arguments = 0; diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 334e809ebe7..1588611b8a2 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -244,6 +244,12 @@ public: { assert_cast(to).getData().push_back(this->data(place).set.size()); } + + /// The function returns non-Nullable type even when wrapped with Null combinator. + bool returnDefaultWhenOnlyNull() const override + { + return true; + } }; @@ -298,6 +304,12 @@ public: { assert_cast(to).getData().push_back(this->data(place).set.size()); } + + /// The function returns non-Nullable type even when wrapped with Null combinator. + bool returnDefaultWhenOnlyNull() const override + { + return true; + } }; } diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 726656d1ca8..b5704203ade 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -240,9 +240,10 @@ public: return std::make_shared(); } - AggregateFunctionPtr getOwnNullAdapter(const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override { - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 0087a41d437..439a5e07c2e 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -171,6 +171,12 @@ public: return nullptr; } + /** When the function is wrapped with Null combinator, + * should we return Nullable type with NULL when no values were aggregated + * or we should return non-Nullable type with default value (example: count, countDistinct). + */ + virtual bool returnDefaultWhenOnlyNull() const { return false; } + const DataTypes & getArgumentTypes() const { return argument_types; } const Array & getParameters() const { return parameters; } From 7b8c1657a5397350f911bffa37da4cbb4c13910d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Jun 2020 19:34:36 +0300 Subject: [PATCH 02/47] Fix nullable prewhere column type. --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index f2b53fbb70a..6cdd0270dea 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -905,7 +905,9 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (prewhere->remove_prewhere_column) result.columns.erase(result.columns.begin() + prewhere_column_pos); else - result.columns[prewhere_column_pos] = DataTypeUInt8().createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); + result.columns[prewhere_column_pos] = + result.block_before_prewhere.getByPosition(prewhere_column_pos).type-> + createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); } } /// Filter in WHERE instead From 8234a430a0221a1ef078f624ae169b2bc81b7277 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Jun 2020 19:40:44 +0300 Subject: [PATCH 03/47] Added test. --- .../0_stateless/01305_nullable-prewhere_bug.reference | 1 + tests/queries/0_stateless/01305_nullable-prewhere_bug.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01305_nullable-prewhere_bug.reference create mode 100644 tests/queries/0_stateless/01305_nullable-prewhere_bug.sql diff --git a/tests/queries/0_stateless/01305_nullable-prewhere_bug.reference b/tests/queries/0_stateless/01305_nullable-prewhere_bug.reference new file mode 100644 index 00000000000..bd81ae06cec --- /dev/null +++ b/tests/queries/0_stateless/01305_nullable-prewhere_bug.reference @@ -0,0 +1 @@ +some_field_value 1 diff --git a/tests/queries/0_stateless/01305_nullable-prewhere_bug.sql b/tests/queries/0_stateless/01305_nullable-prewhere_bug.sql new file mode 100644 index 00000000000..35d3762660f --- /dev/null +++ b/tests/queries/0_stateless/01305_nullable-prewhere_bug.sql @@ -0,0 +1,5 @@ +drop table if exists data; +CREATE TABLE data (ts DateTime, field String, num_field Nullable(Float64)) ENGINE = MergeTree() PARTITION BY ts ORDER BY ts; +insert into data values(toDateTime('2020-05-14 02:08:00'),'some_field_value',7.); +SELECT field, countIf(num_field > 6.0) FROM data PREWHERE (num_field>6.0) GROUP BY field; +drop table if exists data; From 787163d0b489c464d33b3ba9ee003dcdaa4bf6a9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 12 Jun 2020 17:03:00 +0300 Subject: [PATCH 04/47] Minor modifications after merging #11554 --- src/Functions/extractAllGroups.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index a9206e7327e..b75e54b490e 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -54,7 +54,7 @@ public: size_t getNumberOfArguments() const override { return 2; } - bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override From 83155e139c681836adc4a843d88a8f6839981f88 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Jun 2020 17:59:14 +0300 Subject: [PATCH 05/47] Try fix tests. --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 6cdd0270dea..667b0b2da96 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -906,7 +906,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r result.columns.erase(result.columns.begin() + prewhere_column_pos); else result.columns[prewhere_column_pos] = - result.block_before_prewhere.getByPosition(prewhere_column_pos).type-> + getSampleBlock().getByName(prewhere->prewhere_column_name).type-> createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); } } From e5897bbfb6073b087488ce58e940683ad9f55133 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 12 Jun 2020 18:59:11 +0300 Subject: [PATCH 06/47] size_t -> ssize_t --- src/Processors/Transforms/AggregatingInOrderTransform.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 3cac1c9602c..7edeff65ec8 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -100,9 +100,9 @@ void AggregatingInOrderTransform::consume(Chunk chunk) params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); ++cur_block_size; } - size_t mid = 0; - size_t high = 0; - size_t low = -1; + ssize_t mid = 0; + ssize_t high = 0; + ssize_t low = -1; /// Will split block into segments with the same key while (key_end != rows) { From 9901e4d528bf0ebb3594150116077a93666ec450 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jun 2020 20:20:42 +0300 Subject: [PATCH 07/47] Remove debug output #11554 --- src/Functions/extractAllGroups.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index b75e54b490e..8216a528b2c 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -227,7 +227,6 @@ public: row_offset = next_row_offset; } } - DUMP(Kind, needle, column_haystack, root_offsets_col, nested_offsets_col); ColumnArray::MutablePtr nested_array_col = ColumnArray::create(std::move(data_col), std::move(nested_offsets_col)); ColumnArray::MutablePtr root_array_col = ColumnArray::create(std::move(nested_array_col), std::move(root_offsets_col)); From d8312d0f0051fd4523716924a5bd8951e5276bb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jun 2020 20:25:13 +0300 Subject: [PATCH 08/47] Remove "fail" from test name; fix typo in test name #11151 --- ... => 01281_unsucceeded_insert_select_queries_counter.reference} | 0 ...er.sql => 01281_unsucceeded_insert_select_queries_counter.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01281_failed_insert_select_queries_couner.reference => 01281_unsucceeded_insert_select_queries_counter.reference} (100%) rename tests/queries/0_stateless/{01281_failed_insert_select_queries_couner.sql => 01281_unsucceeded_insert_select_queries_counter.sql} (100%) diff --git a/tests/queries/0_stateless/01281_failed_insert_select_queries_couner.reference b/tests/queries/0_stateless/01281_unsucceeded_insert_select_queries_counter.reference similarity index 100% rename from tests/queries/0_stateless/01281_failed_insert_select_queries_couner.reference rename to tests/queries/0_stateless/01281_unsucceeded_insert_select_queries_counter.reference diff --git a/tests/queries/0_stateless/01281_failed_insert_select_queries_couner.sql b/tests/queries/0_stateless/01281_unsucceeded_insert_select_queries_counter.sql similarity index 100% rename from tests/queries/0_stateless/01281_failed_insert_select_queries_couner.sql rename to tests/queries/0_stateless/01281_unsucceeded_insert_select_queries_counter.sql From 2cd82a25f52b684e6fc13010f8efe0965c1e94dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jun 2020 23:59:20 +0300 Subject: [PATCH 09/47] Remove trivial count query optimization if row-level security is set #11352 --- src/Interpreters/ExpressionAnalyzer.h | 2 ++ src/Interpreters/InterpreterSelectQuery.cpp | 17 ++++++++++------- src/Interpreters/InterpreterSelectQuery.h | 4 +++- .../ReplicatedMergeTreeBlockOutputStream.cpp | 3 ++- 4 files changed, 17 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index ed07ab3fe36..c69cb61162f 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -208,7 +208,9 @@ struct ExpressionAnalysisResult const FilterInfoPtr & filter_info, const Block & source_header); + /// Filter for row-level security. bool hasFilter() const { return filter_info.get(); } + bool hasJoin() const { return before_join.get(); } bool hasPrewhere() const { return prewhere_info.get(); } bool hasWhere() const { return before_where.get(); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f9072e6176a..98cf36cc30b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -94,7 +94,8 @@ namespace ErrorCodes } /// Assumes `storage` is set and the table filter (row-level security) is not empty. -String InterpreterSelectQuery::generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const +String InterpreterSelectQuery::generateFilterActions( + ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const { const auto & db_name = table_id.getDatabaseName(); const auto & table_name = table_id.getTableName(); @@ -474,8 +475,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() second_stage, options.only_analyze, filter_info, - source_header - ); + source_header); if (options.to_stage == QueryProcessingStage::Enum::FetchColumns) { @@ -979,10 +979,13 @@ void InterpreterSelectQuery::executeFetchColumns( /// Optimization for trivial query like SELECT count() FROM table. bool optimize_trivial_count = - syntax_analyzer_result->optimize_trivial_count && storage && - processing_stage == QueryProcessingStage::FetchColumns && - query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) && - typeid_cast(query_analyzer->aggregates()[0].function.get()); + syntax_analyzer_result->optimize_trivial_count + && storage + && !filter_info + && processing_stage == QueryProcessingStage::FetchColumns + && query_analyzer->hasAggregation() + && (query_analyzer->aggregates().size() == 1) + && typeid_cast(query_analyzer->aggregates()[0].function.get()); if (optimize_trivial_count) { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 34d255e398e..c60451d5f4a 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -132,7 +132,8 @@ private: void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const std::unordered_map & subqueries_for_sets); void executeMergeSorted(QueryPipeline & pipeline, const SortDescription & sort_description, UInt64 limit); - String generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const; + String generateFilterActions( + ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const; enum class Modificator { @@ -159,6 +160,7 @@ private: /// Is calculated in getSampleBlock. Is used later in readImpl. ExpressionAnalysisResult analysis_result; + /// For row-level security. FilterInfoPtr filter_info; QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index c67ea11f56f..1bbc56d940d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -198,7 +198,8 @@ void ReplicatedMergeTreeBlockOutputStream::writeExistingPart(MergeTreeData::Muta } -void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id) +void ReplicatedMergeTreeBlockOutputStream::commitPart( + zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id) { storage.check(part->getColumns()); assertSessionIsNotExpired(zookeeper); From a421e7e4b40155ddaaab72d0269a68db4f289020 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 00:13:52 +0300 Subject: [PATCH 10/47] Added a test --- programs/server/users.d/access_management.xml | 7 +++++++ ...1308_row_policy_and_trivial_count_query.reference | 3 +++ .../01308_row_policy_and_trivial_count_query.sql | 12 ++++++++++++ 3 files changed, 22 insertions(+) create mode 100644 programs/server/users.d/access_management.xml create mode 100644 tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.reference create mode 100644 tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql diff --git a/programs/server/users.d/access_management.xml b/programs/server/users.d/access_management.xml new file mode 100644 index 00000000000..7e799cb7b10 --- /dev/null +++ b/programs/server/users.d/access_management.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.reference b/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.reference new file mode 100644 index 00000000000..61150aca43c --- /dev/null +++ b/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.reference @@ -0,0 +1,3 @@ +3 +2 +3 diff --git a/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql b/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql new file mode 100644 index 00000000000..c105885cb60 --- /dev/null +++ b/tests/queries/0_stateless/01308_row_policy_and_trivial_count_query.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS t; + +CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY x; +INSERT INTO t VALUES (1), (2), (3); + +SELECT count() FROM t; +CREATE ROW POLICY filter ON t USING (x % 2 = 1) TO ALL; +SELECT count() FROM t; +DROP ROW POLICY filter ON t; +SELECT count() FROM t; + +DROP TABLE t; From 844140467e6fdb2be464dd48b2153994fbdc0e5c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 01:18:48 +0300 Subject: [PATCH 11/47] Simplify the code in comparison functions --- src/Functions/FunctionsComparison.h | 187 ++++++---------------------- 1 file changed, 41 insertions(+), 146 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 642995974b5..12da4c772d1 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -22,6 +22,7 @@ #include #include +#include #include #include @@ -812,94 +813,49 @@ private: } } - bool executeDateOrDateTimeOrEnumOrUUIDWithConstString( + bool executeWithConstString( Block & block, size_t result, const IColumn * col_left_untyped, const IColumn * col_right_untyped, - const DataTypePtr & left_type, const DataTypePtr & right_type, bool left_is_num, size_t input_rows_count) + const DataTypePtr & left_type, const DataTypePtr & right_type, size_t input_rows_count) { - /// This is no longer very special case - comparing dates, datetimes, and enumerations with a string constant. - const IColumn * column_string_untyped = !left_is_num ? col_left_untyped : col_right_untyped; - const IColumn * column_number = left_is_num ? col_left_untyped : col_right_untyped; - const IDataType * number_type = left_is_num ? left_type.get() : right_type.get(); + /// To compare something with const string, we cast constant to appropriate type and compare as usual. + /// We should deal with possible overflows, e.g. toUInt8(1) = '257' should return false. - WhichDataType which(number_type); + const ColumnConst * left_const = checkAndGetColumnConstStringOrFixedString(col_left_untyped); + const ColumnConst * right_const = checkAndGetColumnConstStringOrFixedString(col_right_untyped); - const bool legal_types = which.isDateOrDateTime() || which.isEnum() || which.isUUID(); - - const auto column_string = checkAndGetColumnConst(column_string_untyped); - if (!column_string || !legal_types) + if (!left_const && !right_const) return false; - StringRef string_value = column_string->getDataAt(0); + const IDataType * type_string = left_const ? left_type.get() : right_type.get(); + const DataTypePtr & type_to_compare = !left_const ? left_type : right_type; - if (which.isDate()) + Field string_value = left_const ? left_const->getField() : right_const->getField(); + Field converted = convertFieldToType(string_value, *type_to_compare, type_string); + + /// If not possible to convert, comparison yields to false. + if (converted.isNull()) { - DayNum date; - ReadBufferFromMemory in(string_value.data, string_value.size); - readDateText(date, in); - if (!in.eof()) - throw Exception("String is too long for Date: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - - ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, date); - const ColumnConst * parsed_const_date = assert_cast(parsed_const_date_holder.get()); - executeNumLeftType(block, result, - left_is_num ? col_left_untyped : parsed_const_date, - left_is_num ? parsed_const_date : col_right_untyped); + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 0); } - else if (which.isDateTime()) + else { - time_t date_time; - ReadBufferFromMemory in(string_value.data, string_value.size); - readDateTimeText(date_time, in, dynamic_cast(*number_type).getTimeZone()); - if (!in.eof()) - throw Exception("String is too long for DateTime: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); + auto column_converted = type_to_compare->createColumnConst(input_rows_count, converted); - ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time)); - const ColumnConst * parsed_const_date_time = assert_cast(parsed_const_date_time_holder.get()); - executeNumLeftType(block, result, - left_is_num ? col_left_untyped : parsed_const_date_time, - left_is_num ? parsed_const_date_time : col_right_untyped); + Block tmp_block + { + { left_const ? column_converted : col_left_untyped->getPtr(), type_to_compare, "" }, + { !left_const ? column_converted : col_right_untyped->getPtr(), type_to_compare, "" }, + block.getByPosition(result) + }; + + executeImpl(tmp_block, {0, 1}, 2, input_rows_count); + + block.getByPosition(result).column = std::move(tmp_block.getByPosition(2).column); } - else if (which.isUUID()) - { - UUID uuid; - ReadBufferFromMemory in(string_value.data, string_value.size); - readText(uuid, in); - if (!in.eof()) - throw Exception("String is too long for UUID: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - - ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, uuid); - const ColumnConst * parsed_const_uuid = assert_cast(parsed_const_uuid_holder.get()); - executeNumLeftType(block, result, - left_is_num ? col_left_untyped : parsed_const_uuid, - left_is_num ? parsed_const_uuid : col_right_untyped); - } - - else if (which.isEnum8()) - executeEnumWithConstString(block, result, column_number, column_string, - number_type, left_is_num, input_rows_count); - else if (which.isEnum16()) - executeEnumWithConstString(block, result, column_number, column_string, - number_type, left_is_num, input_rows_count); return true; } - /// Comparison between DataTypeEnum and string constant containing the name of an enum element - template - void executeEnumWithConstString( - Block & block, const size_t result, const IColumn * column_number, const ColumnConst * column_string, - const IDataType * type_untyped, const bool left_is_num, size_t input_rows_count) - { - const auto type = static_cast(type_untyped); - - const Field x = castToNearestFieldType(type->getValue(column_string->getValue())); - const auto enum_col = type->createColumnConst(input_rows_count, x); - - executeNumLeftType(block, result, - left_is_num ? column_number : enum_col.get(), - left_is_num ? enum_col.get() : column_number); - } - void executeTuple(Block & block, size_t result, const ColumnWithTypeAndName & c0, const ColumnWithTypeAndName & c1, size_t input_rows_count) { @@ -1124,17 +1080,11 @@ public: bool has_date = left.isDate() || right.isDate(); if (!((both_represented_by_number && !has_date) /// Do not allow compare date and number. - || (left.isStringOrFixedString() && right.isStringOrFixedString()) + || (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion. /// You can compare the date, datetime, or datatime64 and an enumeration with a constant string. - || (left.isString() && right.isDateOrDateTime()) - || (left.isDateOrDateTime() && right.isString()) || (left.isDateOrDateTime() && right.isDateOrDateTime() && left.idx == right.idx) /// only date vs date, or datetime vs datetime || (left.isUUID() && right.isUUID()) - || (left.isUUID() && right.isString()) - || (left.isString() && right.isUUID()) || (left.isEnum() && right.isEnum() && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against - || (left.isEnum() && right.isString()) - || (left.isString() && right.isEnum()) || (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size()) || (arguments[0]->equals(*arguments[1])))) { @@ -1151,7 +1101,8 @@ public: if (left_tuple && right_tuple) { - auto adaptor = FunctionOverloadResolverAdaptor(std::make_unique(FunctionComparison::create(context))); + auto adaptor = FunctionOverloadResolverAdaptor(std::make_unique( + FunctionComparison::create(context))); size_t size = left_tuple->getElements().size(); for (size_t i = 0; i < size; ++i) @@ -1201,6 +1152,9 @@ public: const bool left_is_num = col_left_untyped->isNumeric(); const bool right_is_num = col_right_untyped->isNumeric(); + const bool left_is_string = isStringOrFixedString(which_left); + const bool right_is_string = isStringOrFixedString(which_right); + bool date_and_datetime = (left_type != right_type) && which_left.isDateOrDateTime() && which_right.isDateOrDateTime(); @@ -1226,64 +1180,14 @@ public: { executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count); } - else if (which_left.idx != which_right.idx - && (which_left.isDateTime64() || which_right.isDateTime64()) - && (which_left.isStringOrFixedString() || which_right.isStringOrFixedString())) + else if (left_is_string && right_is_string && executeString(block, result, col_left_untyped, col_right_untyped)) + { + } + else if (executeWithConstString( + block, result, col_left_untyped, col_right_untyped, + left_type, right_type, + input_rows_count)) { - /** Special case of comparing DateTime64 against a string. - * - * Can't be moved to executeDateOrDateTimeOrEnumOrUUIDWithConstString() - * since DateTime64 is basically a Decimal, but we do similar things, except type inference. - * Outline: - * - Extract string content - * - Parse it as a ColumnDateTime64 value (same type as DateTime64, means same precision) - * - Fabricate a column with type and name - * - Compare left and right comlumns as DateTime64 columns. - */ - - const size_t datetime64_col_index = which_left.isDateTime64() ? 0 : 1; - const size_t string_col_index = which_left.isStringOrFixedString() ? 0 : 1; - - const auto & datetime64_col_with_type_and_name = block.getByPosition(arguments[datetime64_col_index]); - const auto & string_col_with_type_and_name = block.getByPosition(arguments[string_col_index]); - - if (!isColumnConst(*string_col_with_type_and_name.column)) - throw Exception(getName() + ", illegal column type of argument #" + std::to_string(string_col_index) - + " '" + string_col_with_type_and_name.name + "'" - " expected const String or const FixedString," - " got " + string_col_with_type_and_name.type->getName(), - ErrorCodes::ILLEGAL_COLUMN); - - if (datetime64_col_with_type_and_name.column->size() == 0 || string_col_with_type_and_name.column->size() == 0) - { - // For some reason, when both left and right columns are empty (dry run while building a header block) - // executeDecimal() fills result column with bogus value. - block.getByPosition(result).column = ColumnUInt8::create(); - return; - } - - auto parsed_tmp_column_holder = datetime64_col_with_type_and_name.type->createColumn(); - - { - const StringRef string_value = string_col_with_type_and_name.column->getDataAt(0); - ReadBufferFromMemory in(string_value.data, string_value.size); - datetime64_col_with_type_and_name.type->deserializeAsWholeText(*parsed_tmp_column_holder, in, FormatSettings{}); - - if (!in.eof()) - throw Exception(getName() + ": String is too long for " + datetime64_col_with_type_and_name.type->getName() + " : " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); - } - - // It is necessary to wrap tmp column in ColumnConst to avoid overflow when comparing. - // (non-const columns are expected to have same number of rows as every other column in block). - const ColumnWithTypeAndName parsed_tmp_col_with_type_and_name{ - ColumnConst::create(std::move(parsed_tmp_column_holder), 1), - datetime64_col_with_type_and_name.type, - string_col_with_type_and_name.name}; - - executeDecimal(block, result, - which_left.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name, - which_right.isDateTime64() ? datetime64_col_with_type_and_name : parsed_tmp_col_with_type_and_name); - } else if (isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type)) { @@ -1294,19 +1198,10 @@ public: executeDecimal(block, result, col_with_type_and_name_left, col_with_type_and_name_right); } - else if (!left_is_num && !right_is_num && executeString(block, result, col_left_untyped, col_right_untyped)) - { - } else if (left_type->equals(*right_type)) { executeGenericIdenticalTypes(block, result, col_left_untyped, col_right_untyped); } - else if (executeDateOrDateTimeOrEnumOrUUIDWithConstString( - block, result, col_left_untyped, col_right_untyped, - left_type, right_type, - left_is_num, input_rows_count)) - { - } else { executeGeneric(block, result, col_with_type_and_name_left, col_with_type_and_name_right); From f2677a784132adf7b11a2f50e26494a807f0df3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 01:30:17 +0300 Subject: [PATCH 12/47] Added a test just in case --- tests/queries/0_stateless/01310_enum_comparison.reference | 2 ++ tests/queries/0_stateless/01310_enum_comparison.sql | 6 ++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01310_enum_comparison.reference create mode 100644 tests/queries/0_stateless/01310_enum_comparison.sql diff --git a/tests/queries/0_stateless/01310_enum_comparison.reference b/tests/queries/0_stateless/01310_enum_comparison.reference new file mode 100644 index 00000000000..b261da18d51 --- /dev/null +++ b/tests/queries/0_stateless/01310_enum_comparison.reference @@ -0,0 +1,2 @@ +1 +0 diff --git a/tests/queries/0_stateless/01310_enum_comparison.sql b/tests/queries/0_stateless/01310_enum_comparison.sql new file mode 100644 index 00000000000..26901a61b2b --- /dev/null +++ b/tests/queries/0_stateless/01310_enum_comparison.sql @@ -0,0 +1,6 @@ +CREATE TEMPORARY TABLE enum (x Enum('hello' = 1, 'world' = 2)); +INSERT INTO enum VALUES ('hello'); + +SELECT count() FROM enum WHERE x = 'hello'; +SELECT count() FROM enum WHERE x = 'world'; +SELECT count() FROM enum WHERE x = 'xyz'; -- { serverError 36 } From 41afea0165aa2bf66fd46903044bb08c874be54e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 02:10:10 +0300 Subject: [PATCH 13/47] Fix style --- src/Functions/FunctionsComparison.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 12da4c772d1..9cd13df826d 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -52,7 +52,6 @@ namespace DB namespace ErrorCodes { - extern const int TOO_LARGE_STRING_SIZE; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int LOGICAL_ERROR; From 30f1f8811855a75d282eeba5cf2fd23191e2b656 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 03:43:42 +0300 Subject: [PATCH 14/47] Allow case-insensitive regexps; added a test #11101 --- src/Common/OptimizedRegularExpression.cpp | 37 ++++++++++++++++++- .../01312_case_insensitive_regexp.reference | 8 ++++ .../01312_case_insensitive_regexp.sql | 8 ++++ 3 files changed, 51 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01312_case_insensitive_regexp.reference create mode 100644 tests/queries/0_stateless/01312_case_insensitive_regexp.sql diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 8c4aa00f171..1464923e6ab 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -38,6 +38,7 @@ void OptimizedRegularExpressionImpl::analyze( required_substring_is_prefix = false; required_substring.clear(); bool has_alternative_on_depth_0 = false; + bool has_case_insensitive_flag = false; /// Substring with a position. using Substring = std::pair; @@ -65,7 +66,17 @@ void OptimizedRegularExpressionImpl::analyze( switch (*pos) { - case '|': case '(': case ')': case '^': case '$': case '.': case '[': case '?': case '*': case '+': case '{': + case '|': + case '(': + case ')': + case '^': + case '$': + case '.': + case '[': + case '?': + case '*': + case '+': + case '{': if (depth == 0 && !in_curly_braces && !in_square_braces) { if (last_substring->first.empty()) @@ -110,6 +121,28 @@ void OptimizedRegularExpressionImpl::analyze( trivial_substrings.resize(trivial_substrings.size() + 1); last_substring = &trivial_substrings.back(); } + + /// Check for case-insensitive flag. + if (pos + 1 < end && pos[1] == '?') + { + for (size_t offset = 2; pos + offset < end; ++offset) + { + if (pos[offset] == '-' /// it means flag negation + /// various possible flags, actually only imsU are supported by re2 + || (pos[offset] >= 'a' && pos[offset] <= 'z') + || (pos[offset] >= 'A' && pos[offset] <= 'Z')) + { + if (pos[offset] == 'i') + { + /// Actually it can be negated case-insensitive flag. But we don't care. + has_case_insensitive_flag = true; + break; + } + } + else + break; + } + } } ++pos; break; @@ -209,7 +242,7 @@ void OptimizedRegularExpressionImpl::analyze( if (!is_trivial) { - if (!has_alternative_on_depth_0) + if (!has_alternative_on_depth_0 && !has_case_insensitive_flag) { /// We choose the non-alternative substring of the maximum length for first search. diff --git a/tests/queries/0_stateless/01312_case_insensitive_regexp.reference b/tests/queries/0_stateless/01312_case_insensitive_regexp.reference new file mode 100644 index 00000000000..c18b4e9b082 --- /dev/null +++ b/tests/queries/0_stateless/01312_case_insensitive_regexp.reference @@ -0,0 +1,8 @@ +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01312_case_insensitive_regexp.sql b/tests/queries/0_stateless/01312_case_insensitive_regexp.sql new file mode 100644 index 00000000000..ca13989599d --- /dev/null +++ b/tests/queries/0_stateless/01312_case_insensitive_regexp.sql @@ -0,0 +1,8 @@ +SELECT match('Too late', 'Too late'); +select match('Too late', '(?i)Too late'); +select match('Too late', '(?i)too late'); +select match('Too late', '(?i:too late)'); +select match('Too late', '(?i)to{2} late'); +select match('Too late', '(?i)to(?)o late'); +select match('Too late', '(?i)to+ late'); +select match('Too late', '(?i)to(?:o|o) late'); From 970a8e3ecc7fbabb2590fd8a0b093b472e6adfb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 03:56:13 +0300 Subject: [PATCH 15/47] Skip empty URL parameters #10749 --- src/Server/HTTPHandler.cpp | 4 ++++ .../0_stateless/01312_skip_empty_params.reference | 3 +++ tests/queries/0_stateless/01312_skip_empty_params.sh | 10 ++++++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/01312_skip_empty_params.reference create mode 100755 tests/queries/0_stateless/01312_skip_empty_params.sh diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 84d23f10a55..e866af2f49b 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -430,6 +430,10 @@ void HTTPHandler::processQuery( auto param_could_be_skipped = [&] (const String & name) { + /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. + if (name.empty()) + return true; + if (reserved_param_names.count(name)) return true; diff --git a/tests/queries/0_stateless/01312_skip_empty_params.reference b/tests/queries/0_stateless/01312_skip_empty_params.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/01312_skip_empty_params.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/01312_skip_empty_params.sh b/tests/queries/0_stateless/01312_skip_empty_params.sh new file mode 100755 index 00000000000..2e3541aee35 --- /dev/null +++ b/tests/queries/0_stateless/01312_skip_empty_params.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query=select%201&log_queries=1" +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&&query=select%201&log_queries=1" +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&query=select%201&&&log_queries=1" From 6c278fee616e8326b40b2dbf62bed98be77a5ba6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 04:07:47 +0300 Subject: [PATCH 16/47] Better exception message --- src/Core/SettingsCollectionImpl.h | 13 ++++++++++++- src/Interpreters/Context.cpp | 11 ++++++++++- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsCollectionImpl.h b/src/Core/SettingsCollectionImpl.h index 8210b04e2da..877567a7caf 100644 --- a/src/Core/SettingsCollectionImpl.h +++ b/src/Core/SettingsCollectionImpl.h @@ -7,6 +7,8 @@ */ #include +#include + namespace DB { @@ -91,7 +93,16 @@ Field SettingsCollection::const_reference::getValue() const template Field SettingsCollection::valueToCorrespondingType(size_t index, const Field & value) { - return members()[index].value_to_corresponding_type(value); + try + { + return members()[index].value_to_corresponding_type(value); + } + catch (Exception & e) + { + e.addMessage(fmt::format("in attempt to set the value of setting to {}", + applyVisitor(FieldVisitorToString(), value))); + throw; + } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1431f3fd62c..cb780443e03 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -982,7 +982,16 @@ void Context::setSetting(const StringRef & name, const Field & value) void Context::applySettingChange(const SettingChange & change) { - setSetting(change.name, change.value); + try + { + setSetting(change.name, change.value); + } + catch (Exception & e) + { + e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", + change.name, applyVisitor(FieldVisitorToString(), change.value))); + throw; + } } From 400e9fb64f68e020907f9b4a9251b72c415455d4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 04:23:53 +0300 Subject: [PATCH 17/47] Allow to drop table if there is no metadata in ZooKeeper; allow to rename --- src/Storages/StorageReplicatedMergeTree.cpp | 22 +++++++++++++++++++-- src/Storages/StorageReplicatedMergeTree.h | 2 ++ 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 57535466558..5931bca17ea 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -244,6 +244,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( { LOG_WARNING(log, "No metadata in ZooKeeper: table will be in readonly mode."); is_readonly = true; + has_metadata_in_zookeeper = false; return; } @@ -620,9 +621,14 @@ void StorageReplicatedMergeTree::createReplica() void StorageReplicatedMergeTree::drop() { + /// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster, + /// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table. + + if (has_metadata_in_zookeeper) { auto zookeeper = tryGetZooKeeper(); + /// If probably there is metadata in ZooKeeper, we don't allow to drop the table. if (is_readonly || !zookeeper) throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY); @@ -4032,8 +4038,20 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c MergeTreeData::rename(new_path_to_table_data, new_table_id); /// Update table name in zookeeper - auto zookeeper = getZooKeeper(); - zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString()); + if (!is_readonly) + { + /// We don't do it for readonly tables, because it will be updated on next table startup. + /// It is also Ok to skip ZK error for the same reason. + try + { + auto zookeeper = getZooKeeper(); + zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString()); + } + catch (Coordination::Exception & e) + { + LOG_WARNING(log, "Cannot update the value of 'host' node (replica address) in ZooKeeper: {}", e.displayText()); + } + } /// TODO: You can update names of loggers. } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ec38eb7e842..52ce1aada08 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -210,6 +210,8 @@ private: /// If true, the table is offline and can not be written to it. std::atomic_bool is_readonly {false}; + /// If false - ZooKeeper is available, but there is no table metadata. It's safe to drop table in this case. + bool has_metadata_in_zookeeper = true; String zookeeper_path; String replica_name; From fae12d5e42048805873ee8c2fb6cd6eee8ae1e3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 04:45:03 +0300 Subject: [PATCH 18/47] Return NULL or zero when value is not parsed completely in parseDateTimeBestEffortOr* functions --- src/Functions/FunctionsConversion.h | 4 ++-- ...3_parse_date_time_best_effort_null_zero.reference | 6 ++++++ .../01313_parse_date_time_best_effort_null_zero.sql | 12 ++++++++++++ 3 files changed, 20 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.reference create mode 100644 tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 879b885cf66..83417a3229b 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -722,10 +722,10 @@ struct ConvertThroughParsing parsed = ToDataType::tryReadText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale()); else parsed = tryParseImpl(vec_to[i], read_buffer, local_time_zone); - - parsed = parsed && isAllRead(read_buffer); } + parsed = parsed && isAllRead(read_buffer); + if (!parsed) vec_to[i] = 0; diff --git a/tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.reference b/tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.reference new file mode 100644 index 00000000000..90bb776ca10 --- /dev/null +++ b/tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.reference @@ -0,0 +1,6 @@ +\N +0000-00-00 00:00:00 +\N +0000-00-00 00:00:00.000 +\N +0000-00-00 00:00:00 diff --git a/tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.sql b/tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.sql new file mode 100644 index 00000000000..69b66b46df7 --- /dev/null +++ b/tests/queries/0_stateless/01313_parse_date_time_best_effort_null_zero.sql @@ -0,0 +1,12 @@ +SELECT parseDateTimeBestEffort(''); -- { serverError 6 } +SELECT parseDateTimeBestEffortOrNull(''); +SELECT parseDateTimeBestEffortOrZero(''); + +SELECT parseDateTime64BestEffort(''); -- { serverError 6 } +SELECT parseDateTime64BestEffortOrNull(''); +SELECT parseDateTime64BestEffortOrZero(''); + +SET date_time_input_format = 'best_effort'; +SELECT toDateTime(''); -- { serverError 41 } +SELECT toDateTimeOrNull(''); +SELECT toDateTimeOrZero(''); From d6cf62e5872036099bb607bd075e68275aadf642 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 04:56:22 +0300 Subject: [PATCH 19/47] Added column "position" to system.columns and "column_position" to system.parts_columns --- src/Storages/System/StorageSystemColumns.cpp | 5 +++++ src/Storages/System/StorageSystemPartsColumns.cpp | 5 ++++- .../01314_position_in_system_columns.reference | 6 ++++++ .../0_stateless/01314_position_in_system_columns.sql | 8 ++++++++ 4 files changed, 23 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01314_position_in_system_columns.reference create mode 100644 tests/queries/0_stateless/01314_position_in_system_columns.sql diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index b4f5da22c17..90e52ad373e 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -32,6 +32,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_) { "table", std::make_shared() }, { "name", std::make_shared() }, { "type", std::make_shared() }, + { "position", std::make_shared() }, { "default_kind", std::make_shared() }, { "default_expression", std::make_shared() }, { "data_compressed_bytes", std::make_shared() }, @@ -131,8 +132,10 @@ protected: bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name); + size_t position = 0; for (const auto & column : columns) { + ++position; if (check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name)) continue; @@ -147,6 +150,8 @@ protected: res_columns[res_index++]->insert(column.name); if (columns_mask[src_index++]) res_columns[res_index++]->insert(column.type->getName()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(position); if (column.default_desc.expression) { diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index b8acdc5f995..479621fd47f 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -49,6 +49,7 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const std::string & name_) {"column", std::make_shared()}, {"type", std::make_shared()}, + {"column_position", std::make_shared()}, {"default_kind", std::make_shared()}, {"default_expression", std::make_shared()}, {"column_bytes_on_disk", std::make_shared()}, @@ -101,9 +102,10 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co using State = IMergeTreeDataPart::State; + size_t column_position = 0; for (const auto & column : part->getColumns()) - { + ++column_position; size_t j = 0; { WriteBufferFromOwnString out; @@ -143,6 +145,7 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co columns_[j++]->insert(column.name); columns_[j++]->insert(column.type->getName()); + columns_[j++]->insert(column_position); auto column_info_it = columns_info.find(column.name); if (column_info_it != columns_info.end()) diff --git a/tests/queries/0_stateless/01314_position_in_system_columns.reference b/tests/queries/0_stateless/01314_position_in_system_columns.reference new file mode 100644 index 00000000000..32e0ae5900e --- /dev/null +++ b/tests/queries/0_stateless/01314_position_in_system_columns.reference @@ -0,0 +1,6 @@ +x UInt8 1 +y String 2 +z Array(String) 3 +x UInt8 1 +y String 2 +z Array(String) 3 diff --git a/tests/queries/0_stateless/01314_position_in_system_columns.sql b/tests/queries/0_stateless/01314_position_in_system_columns.sql new file mode 100644 index 00000000000..7bb0f3b5a96 --- /dev/null +++ b/tests/queries/0_stateless/01314_position_in_system_columns.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (x UInt8, y String, z Array(String)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO test (x) VALUES (1); + +SELECT name, type, position FROM system.columns WHERE database = currentDatabase() AND table = 'test'; +SELECT column, type, column_position FROM system.parts_columns WHERE database = currentDatabase() AND table = 'test'; + +DROP TABLE test; From 4a052f60c7eb40ee0c0a323a5039f9f0931c3bd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 06:10:35 +0300 Subject: [PATCH 20/47] Don't use debug info from ELF file if it doesn't correspond to the running binary. --- src/Common/Elf.cpp | 46 ++++++++++++++++++++++++++++++++++++++ src/Common/Elf.h | 7 ++++++ src/Common/SymbolIndex.cpp | 37 ++++++++++++++++++++++++++++++ 3 files changed, 90 insertions(+) diff --git a/src/Common/Elf.cpp b/src/Common/Elf.cpp index 11d454abd71..ee32586d95e 100644 --- a/src/Common/Elf.cpp +++ b/src/Common/Elf.cpp @@ -54,6 +54,18 @@ Elf::Elf(const std::string & path) throw Exception("The ELF is truncated (section names string table points after end of file)", ErrorCodes::CANNOT_PARSE_ELF); section_names = reinterpret_cast(mapped + section_names_offset); + + /// Get program headers + + ElfOff program_header_offset = header->e_phoff; + uint16_t program_header_num_entries = header->e_phnum; + + if (!program_header_offset + || !program_header_num_entries + || program_header_offset + program_header_num_entries * sizeof(ElfPhdr) > elf_size) + throw Exception("The ELF is truncated (program header points after end of file)", ErrorCodes::CANNOT_PARSE_ELF); + + program_headers = reinterpret_cast(mapped + program_header_offset); } @@ -104,6 +116,40 @@ std::optional Elf::findSectionByName(const char * name) const } +String Elf::getBuildID() const +{ + for (size_t idx = 0; idx < header->e_phnum; ++idx) + { + const ElfPhdr & phdr = program_headers[idx]; + + if (phdr.p_type == PT_NOTE) + return getBuildID(mapped + phdr.p_offset, phdr.p_filesz); + } + return {}; +} + + +String Elf::getBuildID(const char * nhdr_pos, size_t size) +{ + const char * nhdr_end = nhdr_pos + size; + + while (nhdr_pos < nhdr_end) + { + const ElfNhdr & nhdr = *reinterpret_cast(nhdr_pos); + + nhdr_pos += sizeof(ElfNhdr) + nhdr.n_namesz; + if (nhdr.n_type == NT_GNU_BUILD_ID) + { + const char * build_id = nhdr_pos; + return {build_id, nhdr.n_descsz}; + } + nhdr_pos += nhdr.n_descsz; + } + + return {}; +} + + const char * Elf::Section::name() const { if (!elf.section_names) diff --git a/src/Common/Elf.h b/src/Common/Elf.h index f3aafc8e5a9..632d7e6f0b1 100644 --- a/src/Common/Elf.h +++ b/src/Common/Elf.h @@ -17,6 +17,7 @@ using ElfEhdr = ElfW(Ehdr); using ElfOff = ElfW(Off); using ElfPhdr = ElfW(Phdr); using ElfShdr = ElfW(Shdr); +using ElfNhdr = ElfW(Nhdr); using ElfSym = ElfW(Sym); @@ -53,12 +54,18 @@ public: const char * end() const { return mapped + elf_size; } size_t size() const { return elf_size; } + /// Obtain build id from PT_NOTES section of program headers. Return empty string if does not exist. + /// The string is returned in binary. Note that "readelf -n ./clickhouse-server" prints it in hex. + String getBuildID() const; + static String getBuildID(const char * nhdr_pos, size_t size); + private: MMapReadBufferFromFile in; size_t elf_size; const char * mapped; const ElfEhdr * header; const ElfShdr * section_headers; + const ElfPhdr * program_headers; const char * section_names = nullptr; }; diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 482c6fd0bad..54789695dd1 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -196,6 +196,20 @@ void collectSymbolsFromProgramHeaders(dl_phdr_info * info, } +String getBuildIDFromProgramHeaders(dl_phdr_info * info) +{ + for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) + { + const ElfPhdr & phdr = info->dlpi_phdr[header_index]; + if (phdr.p_type != PT_NOTE) + continue; + + return Elf::getBuildID(reinterpret_cast(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz); + } + return {}; +} + + void collectSymbolsFromELFSymbolTable( dl_phdr_info * info, const Elf & elf, @@ -283,8 +297,31 @@ void collectSymbolsFromELF(dl_phdr_info * info, object_name = std::filesystem::exists(debug_info_path) ? debug_info_path : canonical_path; + /// But we have to compare Build ID to check that debug info corresponds to the same executable. + String our_build_id = getBuildIDFromProgramHeaders(info); + SymbolIndex::Object object; object.elf = std::make_unique(object_name); + + String file_build_id = object.elf->getBuildID(); + + if (our_build_id != file_build_id) + { + /// If debug info doesn't correspond to our binary, fallback to the info in our binary. + if (object_name != canonical_path) + { + object_name = canonical_path; + object.elf = std::make_unique(object_name); + + /// But it can still be outdated, for example, if executable file was deleted from filesystem and replaced by another file. + file_build_id = object.elf->getBuildID(); + if (our_build_id != file_build_id) + return; + } + else + return; + } + object.address_begin = reinterpret_cast(info->dlpi_addr); object.address_end = reinterpret_cast(info->dlpi_addr + object.elf->size()); object.name = object_name; From cb395ff099ff6f6602356d0ee7ab14690ad5bb80 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:04:19 +0300 Subject: [PATCH 21/47] Update test --- tests/queries/0_stateless/00700_decimal_compare.reference | 2 ++ tests/queries/0_stateless/00700_decimal_compare.sql | 2 +- tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql | 8 ++++---- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00700_decimal_compare.reference b/tests/queries/0_stateless/00700_decimal_compare.reference index 32f0b0a6dea..2325847045f 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.reference +++ b/tests/queries/0_stateless/00700_decimal_compare.reference @@ -1,3 +1,5 @@ +0 +1 -42 -42 1 0 0 0 1 1 42 42 1 0 0 0 1 1 -42 -42.42000 0 0 1 1 0 1 diff --git a/tests/queries/0_stateless/00700_decimal_compare.sql b/tests/queries/0_stateless/00700_decimal_compare.sql index 24b4ce588e5..ae2f5790570 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/tests/queries/0_stateless/00700_decimal_compare.sql @@ -19,7 +19,7 @@ INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, SELECT a > toFloat64(0) FROM decimal; -- { serverError 43 } SELECT g > toFloat32(0) FROM decimal; -- { serverError 43 } -SELECT a > '0.0' FROM decimal; -- { serverError 43 } +SELECT a > '0.0' FROM decimal ORDER BY a; SELECT a, b, a = b, a < b, a > b, a != b, a <= b, a >= b FROM decimal ORDER BY a; SELECT a, g, a = g, a < g, a > g, a != g, a <= g, a >= g FROM decimal ORDER BY a; diff --git a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql index c65bf668d71..7848b4aaf24 100644 --- a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql +++ b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql @@ -1,9 +1,9 @@ -- Error cases: -- non-const string column -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = materialize(S); -- {serverError 44} -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materialize(S) = toDateTime64(S, 3); -- {serverError 44} -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 44} -WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 44} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = materialize(S); -- {serverError 43} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materialize(S) = toDateTime64(S, 3); -- {serverError 43} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 43} +WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 43} SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 131} -- invalid DateTime64 string SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; -- {serverError 131} -- invalid string length From 59d4df19f03ab10588f043efa4f24e3292c2698d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:05:31 +0300 Subject: [PATCH 22/47] Update test --- tests/queries/0_stateless/00981_no_virtual_columns.reference | 2 +- tests/queries/0_stateless/00981_no_virtual_columns.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00981_no_virtual_columns.reference b/tests/queries/0_stateless/00981_no_virtual_columns.reference index a7ec77dc030..587be6b4c3f 100644 --- a/tests/queries/0_stateless/00981_no_virtual_columns.reference +++ b/tests/queries/0_stateless/00981_no_virtual_columns.reference @@ -1 +1 @@ -default merge_ab x UInt8 0 0 0 0 0 0 0 +x diff --git a/tests/queries/0_stateless/00981_no_virtual_columns.sql b/tests/queries/0_stateless/00981_no_virtual_columns.sql index 476377b4ddf..b3946154581 100644 --- a/tests/queries/0_stateless/00981_no_virtual_columns.sql +++ b/tests/queries/0_stateless/00981_no_virtual_columns.sql @@ -6,7 +6,7 @@ CREATE TABLE merge_a (x UInt8) ENGINE = StripeLog; CREATE TABLE merge_b (x UInt8) ENGINE = StripeLog; CREATE TABLE merge_ab AS merge(currentDatabase(), '^merge_[ab]$'); -SELECT * FROM system.columns WHERE database = currentDatabase() AND table = 'merge_ab'; +SELECT name FROM system.columns WHERE database = currentDatabase() AND table = 'merge_ab'; DROP TABLE merge_a; DROP TABLE merge_b; From fb040ef09fea5b59d573ae874192f62b07f0d5b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:09:02 +0300 Subject: [PATCH 23/47] Update test result (now it is better) --- .../00569_parse_date_time_best_effort.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference b/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference index b873e8b848d..bf11e1c1d71 100644 --- a/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference +++ b/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference @@ -13,7 +13,7 @@ 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 17 Apr 2 1:2:3 2000-04-17 01:02:03 2000-04-17 01:02:03 19700102 01:00:00 1970-01-02 01:00:00 1970-01-02 01:00:00 - 1970010201:00:00 2032-06-06 02:03:21 2032-06-06 02:03:21 + 1970010201:00:00 ᴺᵁᴸᴸ 0000-00-00 00:00:00 19700102010203 1970-01-02 01:02:03 1970-01-02 01:02:03 19700102010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 @@ -61,7 +61,7 @@ 2017/01/32 0000-00-00 00:00:00 0000-00-00 00:00:00 2017-01 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 201701 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 - 2017 25 1:2:3 0000-00-00 00:00:00 0000-00-00 00:00:00 + 2017 25 1:2:3 ᴺᵁᴸᴸ 0000-00-00 00:00:00 2017 25 Apr 1:2:3 2017-04-01 01:02:03 2017-04-01 01:02:03 2017 Apr 01 11:22:33 2017-04-01 11:22:33 2017-04-01 11:22:33 2017 Apr 02 01/02/03 UTC+0300 ᴺᵁᴸᴸ 0000-00-00 00:00:00 @@ -74,8 +74,8 @@ 2017 Apr 02 1:2:3 2017-04-02 01:02:03 2017-04-02 01:02:03 2017 Apr 02 1:2:33 2017-04-02 01:02:33 2017-04-02 01:02:33 2017 Apr 02 1:2:3 MSK 2017-04-01 22:02:03 2017-04-01 22:02:03 - 2017 Apr 02 1:2:3 MSK 2017 2017-04-01 22:02:03 2017-04-01 22:02:03 - 2017 Apr 02 1:2:3 MSK 2018 2017-04-01 22:02:03 2017-04-01 22:02:03 + 2017 Apr 02 1:2:3 MSK 2017 ᴺᵁᴸᴸ 0000-00-00 00:00:00 + 2017 Apr 02 1:2:3 MSK 2018 ᴺᵁᴸᴸ 0000-00-00 00:00:00 2017 Apr 02 1:2:3 UTC+0000 2017-04-02 01:02:03 2017-04-02 01:02:03 2017 Apr 02 1:2:3 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03 2017 Apr 02 1:2:3 UTC+0400 2017-04-01 21:02:03 2017-04-01 21:02:03 @@ -101,6 +101,6 @@ 25 Jan 2017 1:2:3 Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03 25 Jan 2017 1:2:3Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03 25 Jan 2017 1:2:3 Z PM +03:00 2017-01-25 10:02:03 2017-01-25 10:02:03 - Jun, 11 Feb 2018 06:40:50 +0300 2000-06-01 00:00:00 2000-06-01 00:00:00 + Jun, 11 Feb 2018 06:40:50 +0300 ᴺᵁᴸᴸ 0000-00-00 00:00:00 Sun 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 Sun, 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 From e24576c56c4d12bf5ab66500c4462c9163c85976 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:13:27 +0300 Subject: [PATCH 24/47] Update test --- .../0_stateless/00578_merge_table_shadow_virtual_column.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql b/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql index 3071e307517..e729bfdf188 100644 --- a/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql +++ b/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS numbers2; CREATE TABLE numbers1 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); CREATE TABLE numbers2 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); -SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 43 } +SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 53 } SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table=1; DROP TABLE numbers1; From eec5abde071d98942446f3292b4a50ed0284cb4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:16:23 +0300 Subject: [PATCH 25/47] Fix test --- src/Interpreters/convertFieldToType.cpp | 65 ------------------------- 1 file changed, 65 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 3cb774596c0..6328ed76924 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -124,42 +124,6 @@ static Field convertDecimalType(const Field & from, const To & type) } -DayNum stringToDate(const String & s) -{ - ReadBufferFromString in(s); - DayNum date{}; - - readDateText(date, in); - if (!in.eof()) - throw Exception("String is too long for Date: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE); - - return date; -} - -UInt64 stringToDateTime(const String & s) -{ - ReadBufferFromString in(s); - time_t date_time{}; - - readDateTimeText(date_time, in); - if (!in.eof()) - throw Exception("String is too long for DateTime: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE); - - return UInt64(date_time); -} - -DateTime64::NativeType stringToDateTime64(const String & s, UInt32 scale) -{ - ReadBufferFromString in(s); - DateTime64 datetime64 {0}; - - readDateTime64Text(datetime64, scale, in); - if (!in.eof()) - throw Exception("String is too long for DateTime64: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE); - - return datetime64.value; -} - Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const IDataType * from_type_hint) { WhichDataType which_type(type); @@ -215,35 +179,6 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return src; } // TODO (vnemkov): extra cases for DateTime64: converting from integer, converting from Decimal - - if (src.getType() == Field::Types::String) - { - if (which_type.isDate()) - { - /// Convert 'YYYY-MM-DD' Strings to Date - return stringToDate(src.get()); - } - else if (which_type.isDateTime()) - { - /// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime - return stringToDateTime(src.get()); - } - else if (which_type.isDateTime64()) - { - const auto * date_time64 = typeid_cast(&type); - /// Convert 'YYYY-MM-DD hh:mm:ss.NNNNNNNNN' Strings to DateTime - return stringToDateTime64(src.get(), date_time64->getScale()); - } - else if (which_type.isUUID()) - { - return stringToUUID(src.get()); - } - else if (which_type.isEnum()) - { - /// Convert String to Enum's value - return dynamic_cast(type).castToValue(src); - } - } } else if (which_type.isStringOrFixedString()) { From ff3e5e1a2ebaba695b8ebbad8a047f9c08259f74 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 02:09:40 +0300 Subject: [PATCH 26/47] Allow implicit conversion from String in IN, VALUES and comparison #11630 --- src/Interpreters/convertFieldToType.cpp | 29 ++++++++++++++++--------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 6328ed76924..b71d2ffbaa7 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -148,7 +148,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } - else if (type.isValueRepresentedByNumber()) + else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { if (which_type.isUInt8()) return convertNumericType(src, type); if (which_type.isUInt16()) return convertNumericType(src, type); @@ -164,9 +164,6 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); - if (!which_type.isDateOrDateTime() && !which_type.isUUID() && !which_type.isEnum()) - throw Exception{"Cannot convert field to type " + type.getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; - if (which_type.isEnum() && (src.getType() == Field::Types::UInt64 || src.getType() == Field::Types::Int64)) { /// Convert UInt64 or Int64 to Enum's value @@ -263,17 +260,29 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return src; } + /// Conversion from string by parsing. if (src.getType() == Field::Types::String) { - const auto col = type.createColumn(); - ReadBufferFromString buffer(src.get()); - type.deserializeAsTextEscaped(*col, buffer, FormatSettings{}); + /// Promote data type to avoid overflows. Note that overflows in the largest data type are still possible. + const IDataType * type_to_parse = &type; + DataTypePtr holder; - return (*col)[0]; + if (type.canBePromoted()) + { + holder = type.promoteNumericType(); + type_to_parse = holder.get(); + } + + const auto col = type_to_parse->createColumn(); + ReadBufferFromString in_buffer(src.get()); + type_to_parse->deserializeAsWholeText(*col, in_buffer, FormatSettings{}); + if (!in_buffer.eof()) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String is too long for {}: {}", type.getName(), src.get()); + + Field parsed = (*col)[0]; + return convertFieldToType(parsed, type, from_type_hint); } - - // TODO (nemkov): should we attempt to parse value using or `type.deserializeAsTextEscaped()` type.deserializeAsTextEscaped() ? throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: " + Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH); } From 402484079591be39d37079b96757a964c1a0144e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 02:19:16 +0300 Subject: [PATCH 27/47] Add a test --- src/Interpreters/convertFieldToType.cpp | 10 +++++++++- ..._comparison_with_constant_string.reference | 20 +++++++++++++++++++ .../01311_comparison_with_constant_string.sql | 20 +++++++++++++++++++ 3 files changed, 49 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01311_comparison_with_constant_string.reference create mode 100644 tests/queries/0_stateless/01311_comparison_with_constant_string.sql diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index b71d2ffbaa7..050be6ba956 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -275,7 +275,15 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID const auto col = type_to_parse->createColumn(); ReadBufferFromString in_buffer(src.get()); - type_to_parse->deserializeAsWholeText(*col, in_buffer, FormatSettings{}); + try + { + type_to_parse->deserializeAsWholeText(*col, in_buffer, FormatSettings{}); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while converting '{}' to {}", src.get(), type.getName())); + throw; + } if (!in_buffer.eof()) throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String is too long for {}: {}", type.getName(), src.get()); diff --git a/tests/queries/0_stateless/01311_comparison_with_constant_string.reference b/tests/queries/0_stateless/01311_comparison_with_constant_string.reference new file mode 100644 index 00000000000..48c6fc05950 --- /dev/null +++ b/tests/queries/0_stateless/01311_comparison_with_constant_string.reference @@ -0,0 +1,20 @@ +0 +1 +0 +--- +1 +0 +1 +--- +1 +0 +0 +--- +0 +--- +1 +0 +--- +--- +1 +--- diff --git a/tests/queries/0_stateless/01311_comparison_with_constant_string.sql b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql new file mode 100644 index 00000000000..2cfec6ca05e --- /dev/null +++ b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql @@ -0,0 +1,20 @@ +SELECT number = '1' FROM numbers(3); +SELECT '---'; +SELECT '1' != number FROM numbers(3); +SELECT '---'; +SELECT '1' > number FROM numbers(3); +SELECT '---'; +SELECT 1 = '257'; +SELECT '---'; +SELECT 1 IN (1.23, '1', 2); +SELECT 1 IN (1.23, '2', 2); +SELECT '---'; + +-- it should work but it doesn't. +SELECT 1 = '1.0'; -- { serverError 131 } +SELECT '---'; + +SELECT toDateTime('2020-06-13 01:02:03') = '2020-06-13T01:02:03'; +SELECT '---'; + +SELECT 0 = ''; -- { serverError 32 } From e2f7a41a1a640f9c4d8e402f992ae8e2fbdcf77d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 02:27:10 +0300 Subject: [PATCH 28/47] Fix error; clarify more results in test --- src/Functions/FunctionsComparison.h | 6 ++++-- .../01311_comparison_with_constant_string.reference | 12 ++++++++++++ .../01311_comparison_with_constant_string.sql | 13 +++++++++++++ 3 files changed, 29 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 9cd13df826d..91525f84c14 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -817,6 +817,7 @@ private: const DataTypePtr & left_type, const DataTypePtr & right_type, size_t input_rows_count) { /// To compare something with const string, we cast constant to appropriate type and compare as usual. + /// It is ok to throw exception if value is not convertible. /// We should deal with possible overflows, e.g. toUInt8(1) = '257' should return false. const ColumnConst * left_const = checkAndGetColumnConstStringOrFixedString(col_left_untyped); @@ -831,10 +832,11 @@ private: Field string_value = left_const ? left_const->getField() : right_const->getField(); Field converted = convertFieldToType(string_value, *type_to_compare, type_string); - /// If not possible to convert, comparison yields to false. + /// If not possible to convert, comparison with =, <, >, <=, >= yields to false and comparison with != yields to true. if (converted.isNull()) { - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 0); + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, + std::is_same_v, NotEqualsOp>); } else { diff --git a/tests/queries/0_stateless/01311_comparison_with_constant_string.reference b/tests/queries/0_stateless/01311_comparison_with_constant_string.reference index 48c6fc05950..7865f42932d 100644 --- a/tests/queries/0_stateless/01311_comparison_with_constant_string.reference +++ b/tests/queries/0_stateless/01311_comparison_with_constant_string.reference @@ -16,5 +16,17 @@ 0 --- --- +0 +--- +1 +--- +0 +--- +0 +--- +0 +--- +0 +--- 1 --- diff --git a/tests/queries/0_stateless/01311_comparison_with_constant_string.sql b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql index 2cfec6ca05e..6ca736ba146 100644 --- a/tests/queries/0_stateless/01311_comparison_with_constant_string.sql +++ b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql @@ -14,6 +14,19 @@ SELECT '---'; SELECT 1 = '1.0'; -- { serverError 131 } SELECT '---'; +SELECT 1 = '257'; +SELECT '---'; +SELECT 1 != '257'; +SELECT '---'; +SELECT 1 < '257'; -- this is wrong for now +SELECT '---'; +SELECT 1 > '257'; +SELECT '---'; +SELECT 1 <= '257'; -- this is wrong for now +SELECT '---'; +SELECT 1 >= '257'; +SELECT '---'; + SELECT toDateTime('2020-06-13 01:02:03') = '2020-06-13T01:02:03'; SELECT '---'; From fed6843e64a1e7ae9f0cc1d3884a4c11976e8dda Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 03:27:00 +0300 Subject: [PATCH 29/47] Fix style --- src/Interpreters/convertFieldToType.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 050be6ba956..9c1136e5df6 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -34,7 +34,6 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int TYPE_MISMATCH; extern const int TOO_LARGE_STRING_SIZE; - extern const int CANNOT_CONVERT_TYPE; } From bba0140d8fa0e42056bb9f5ba89ffe4002c4ee8f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:20:52 +0300 Subject: [PATCH 30/47] Fix tests --- src/Interpreters/convertFieldToType.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 9c1136e5df6..d60bb6cee6c 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -33,7 +33,6 @@ namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; extern const int TYPE_MISMATCH; - extern const int TOO_LARGE_STRING_SIZE; } @@ -284,7 +283,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID throw; } if (!in_buffer.eof()) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String is too long for {}: {}", type.getName(), src.get()); + throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert string {} to type {}", src.get(), type.getName()); Field parsed = (*col)[0]; return convertFieldToType(parsed, type, from_type_hint); From 44221139e9f2d637a5e1a10a628692552c062291 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:35:50 +0300 Subject: [PATCH 31/47] Fix test --- src/DataTypes/DataTypeUUID.cpp | 7 ++++++- src/DataTypes/DataTypeUUID.h | 3 +++ src/Interpreters/convertFieldToType.cpp | 7 +++++++ 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeUUID.cpp b/src/DataTypes/DataTypeUUID.cpp index 87e306cf477..94a043eb472 100644 --- a/src/DataTypes/DataTypeUUID.cpp +++ b/src/DataTypes/DataTypeUUID.cpp @@ -16,13 +16,18 @@ void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBu writeText(UUID(assert_cast(column).getData()[row_num]), ostr); } -void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const +void DataTypeUUID::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { UUID x; readText(x, istr); assert_cast(column).getData().push_back(x); } +void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeText(column, istr, settings); +} + void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeText(column, row_num, ostr, settings); diff --git a/src/DataTypes/DataTypeUUID.h b/src/DataTypes/DataTypeUUID.h index 75e634bc625..e9f1d22325b 100644 --- a/src/DataTypes/DataTypeUUID.h +++ b/src/DataTypes/DataTypeUUID.h @@ -17,6 +17,7 @@ public: bool equals(const IDataType & rhs) const override; void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; @@ -30,6 +31,8 @@ public: bool canBeUsedInBitOperations() const override { return true; } bool canBeInsideNullable() const override { return true; } + + bool canBePromoted() const override { return false; } }; } diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index d60bb6cee6c..21cf9422c32 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -173,6 +173,13 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID /// We don't need any conversion UInt64 is under type of Date and DateTime return src; } + + if (which_type.isUUID() && src.getType() == Field::Types::UInt128) + { + /// Already in needed type. + return src; + } + // TODO (vnemkov): extra cases for DateTime64: converting from integer, converting from Decimal } else if (which_type.isStringOrFixedString()) From 4953b5fc84c2b2b59cb8a9751553caa762961114 Mon Sep 17 00:00:00 2001 From: Bharat Nallan Date: Sat, 13 Jun 2020 21:41:55 -0700 Subject: [PATCH 32/47] remove unused imports from HTTPHandlerFactory This removes unused imports from `src/Server/HTTPHandlerFactory.cpp`: ```bash - #include - #include - #include ``` --- src/Server/HTTPHandlerFactory.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 2f00aa0aa72..3bcb29672e7 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -1,9 +1,6 @@ #include "HTTPHandlerFactory.h" -#include #include -#include -#include #include #include "HTTPHandler.h" From 3958a032acd8e3da194704505715d9be22b33787 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 08:15:29 +0300 Subject: [PATCH 33/47] Added a test --- src/AggregateFunctions/AggregateFunctionNull.cpp | 10 +++++----- ...1315_count_distinct_return_not_nullable.reference | 9 +++++++++ .../01315_count_distinct_return_not_nullable.sql | 12 ++++++++++++ 3 files changed, 26 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01315_count_distinct_return_not_nullable.reference create mode 100644 tests/queries/0_stateless/01315_count_distinct_return_not_nullable.sql diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index 77687f9f328..993cb93c991 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -33,6 +33,11 @@ public: AggregateFunctionPtr transformAggregateFunction( const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override { + /// Special case for 'count' function. It could be called with Nullable arguments + /// - that means - count number of calls, when all arguments are not NULL. + if (nested_function && nested_function->getName() == "count") + return std::make_shared(arguments[0], params); + bool has_nullable_types = false; bool has_null_types = false; for (const auto & arg_type : arguments) @@ -60,11 +65,6 @@ public: if (auto adapter = nested_function->getOwnNullAdapter(nested_function, arguments, params)) return adapter; - /// Special case for 'count' function. It could be called with Nullable arguments - /// - that means - count number of calls, when all arguments are not NULL. - if (nested_function->getName() == "count") - return std::make_shared(arguments[0], params); - bool return_type_is_nullable = !nested_function->returnDefaultWhenOnlyNull() && nested_function->getReturnType()->canBeInsideNullable(); bool serialize_flag = return_type_is_nullable || nested_function->returnDefaultWhenOnlyNull(); diff --git a/tests/queries/0_stateless/01315_count_distinct_return_not_nullable.reference b/tests/queries/0_stateless/01315_count_distinct_return_not_nullable.reference new file mode 100644 index 00000000000..f8b77704aa3 --- /dev/null +++ b/tests/queries/0_stateless/01315_count_distinct_return_not_nullable.reference @@ -0,0 +1,9 @@ +0 +0 +0 +5 +5 +5 +0 +\N +\N diff --git a/tests/queries/0_stateless/01315_count_distinct_return_not_nullable.sql b/tests/queries/0_stateless/01315_count_distinct_return_not_nullable.sql new file mode 100644 index 00000000000..2d9b5ef54aa --- /dev/null +++ b/tests/queries/0_stateless/01315_count_distinct_return_not_nullable.sql @@ -0,0 +1,12 @@ +SELECT uniq(number >= 10 ? number : NULL) FROM numbers(10); +SELECT uniqExact(number >= 10 ? number : NULL) FROM numbers(10); +SELECT count(DISTINCT number >= 10 ? number : NULL) FROM numbers(10); + +SELECT uniq(number >= 5 ? number : NULL) FROM numbers(10); +SELECT uniqExact(number >= 5 ? number : NULL) FROM numbers(10); +SELECT count(DISTINCT number >= 5 ? number : NULL) FROM numbers(10); + +SELECT count(NULL); +-- These two returns NULL for now, but we want to change them to return 0. +SELECT uniq(NULL); +SELECT count(DISTINCT NULL); From 1531f0bd0f794abba5ace7b220058d8d055dc883 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 14 Jun 2020 17:52:03 +0300 Subject: [PATCH 34/47] Update performance.html --- website/templates/index/performance.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/index/performance.html b/website/templates/index/performance.html index 2a9b766c74b..61cd31a06ca 100644 --- a/website/templates/index/performance.html +++ b/website/templates/index/performance.html @@ -6,7 +6,7 @@

ClickHouse's performance exceeds comparable column-oriented database management systems currently available on the market. It processes hundreds of millions to more than a billion rows and tens of gigabytes of data per single server per second.

- Detailed comparison + Detailed comparison
From fcd23d02eec3fd7029d3090f31a117f75bbe3be1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 20:57:50 +0300 Subject: [PATCH 35/47] Fix timeout in sql_fuzzy test --- tests/queries/0_stateless/00746_sql_fuzzy.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00746_sql_fuzzy.sh b/tests/queries/0_stateless/00746_sql_fuzzy.sh index f417f0146c2..539d0659d36 100755 --- a/tests/queries/0_stateless/00746_sql_fuzzy.sh +++ b/tests/queries/0_stateless/00746_sql_fuzzy.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > $S # if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=10}); do - env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 + env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 if [[ `$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"` != 'Still alive' ]]; then break fi From 5d891f6c878562329c8a488debd704317c63b90d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 21:13:39 +0300 Subject: [PATCH 36/47] Fix tests --- .../0_stateless/01277_convert_field_to_type_logical_error.sql | 2 +- .../0_stateless/01311_comparison_with_constant_string.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql b/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql index 05295575cf1..4712c124237 100644 --- a/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql +++ b/tests/queries/0_stateless/01277_convert_field_to_type_logical_error.sql @@ -1 +1 @@ -SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 70 } +SELECT -2487, globalNullIn(toIntervalMinute(-88074), 'qEkek..'), [-27.537293]; -- { serverError 53 } diff --git a/tests/queries/0_stateless/01311_comparison_with_constant_string.sql b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql index 6ca736ba146..d6641a50c45 100644 --- a/tests/queries/0_stateless/01311_comparison_with_constant_string.sql +++ b/tests/queries/0_stateless/01311_comparison_with_constant_string.sql @@ -11,7 +11,7 @@ SELECT 1 IN (1.23, '2', 2); SELECT '---'; -- it should work but it doesn't. -SELECT 1 = '1.0'; -- { serverError 131 } +SELECT 1 = '1.0'; -- { serverError 53 } SELECT '---'; SELECT 1 = '257'; From db0fc6c9a661ed1f1443e95f1f31252a12bf8244 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 20:48:10 +0300 Subject: [PATCH 37/47] Fix tests --- src/DataTypes/DataTypeDateTime64.h | 2 ++ src/Interpreters/convertFieldToType.cpp | 8 +++++++- tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql | 4 ++-- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypeDateTime64.h b/src/DataTypes/DataTypeDateTime64.h index b575e9d81c1..249da255eb0 100644 --- a/src/DataTypes/DataTypeDateTime64.h +++ b/src/DataTypes/DataTypeDateTime64.h @@ -45,6 +45,8 @@ public: void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; bool equals(const IDataType & rhs) const override; + + bool canBePromoted() const override { return false; } }; /** Tansform-type wrapper for DateTime64, applies given Transform to DateTime64 value or only to a whole part of it. diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 21cf9422c32..d46573d0461 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -180,7 +180,13 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return src; } - // TODO (vnemkov): extra cases for DateTime64: converting from integer, converting from Decimal + if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64) + { + /// Already in needed type. + return src; + } + + /// TODO Conversion from integers to DateTime64 } else if (which_type.isStringOrFixedString()) { diff --git a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql index 7848b4aaf24..4ffcf3be3c9 100644 --- a/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql +++ b/tests/queries/0_stateless/01268_DateTime64_in_WHERE.sql @@ -5,8 +5,8 @@ WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT materiali WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE DT64 = materialize(S); -- {serverError 43} WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT * WHERE materialize(S) = DT64; -- {serverError 43} -SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 131} -- invalid DateTime64 string -SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; -- {serverError 131} -- invalid string length +SELECT * WHERE toDateTime64(123.345, 3) == 'ABCD'; -- {serverError 53} -- invalid DateTime64 string +SELECT * WHERE toDateTime64(123.345, 3) == '2020-02-05 14:34:12.33333333333333333333333333333333333333333333333333333333'; -- {serverError 53} -- invalid string length SELECT 'in SELECT'; WITH '2020-02-05 14:34:12.333' as S, toDateTime64(S, 3) as DT64 SELECT DT64 = S; From f6c52fe1c225cc53a3e184d6a8e9433733d2b59c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 03:16:01 +0300 Subject: [PATCH 38/47] Allow comparison with String in index analysis; simplify code #11630 --- src/Common/FieldVisitors.h | 303 ++++++++---------------- src/Storages/MergeTree/KeyCondition.cpp | 4 +- 2 files changed, 98 insertions(+), 209 deletions(-) diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index 90f80974ab1..257994a6bd2 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include class SipHash; @@ -184,232 +186,119 @@ template <> constexpr bool isDecimalField>() { return t class FieldVisitorAccurateEquals : public StaticVisitor { public: - bool operator() (const UInt64 &, const Null &) const { return false; } - bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const UInt64 & l, const UInt128 & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } - - bool operator() (const Int64 &, const Null &) const { return false; } - bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const Int64 & l, const UInt128 & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const Int64 & r) const { return l == r; } - bool operator() (const Int64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } - - bool operator() (const Float64 &, const Null &) const { return false; } - bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const Float64 & l, const UInt128 & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const Float64 & l, const Float64 & r) const { return l == r; } - bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } - - template - bool operator() (const Null &, const T &) const - { - return std::is_same_v; - } - - template - bool operator() (const String & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return stringToUUID(l) == r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const UInt128 & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return l == stringToUUID(r); - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Array & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Tuple & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - template - bool operator() (const DecimalField & l, const U & r) const + bool operator() (const T & l, const U & r) const { - if constexpr (isDecimalField()) - return l == r; - if constexpr (std::is_same_v || std::is_same_v) - return l == DecimalField(r, 0); - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } + if constexpr (std::is_same_v || std::is_same_v) + return std::is_same_v; + else + { + if constexpr (std::is_same_v) + return l == r; - template bool operator() (const UInt64 & l, const DecimalField & r) const { return DecimalField(l, 0) == r; } - template bool operator() (const Int64 & l, const DecimalField & r) const { return DecimalField(l, 0) == r; } - template bool operator() (const Float64 & l, const DecimalField & r) const { return cantCompare(l, r); } + if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + return accurate::equalsOp(l, r); - template - bool operator() (const AggregateFunctionStateData & l, const T & r) const - { - if constexpr (std::is_same_v) - return l == r; - return cantCompare(l, r); - } + if constexpr (isDecimalField() && isDecimalField()) + return l == r; + + if constexpr (isDecimalField() && std::is_arithmetic_v) + return l == DecimalField(r, 0); + + if constexpr (std::is_arithmetic_v && isDecimalField()) + return DecimalField(l, 0) == r; + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return stringToUUID(l) == r; + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(l); + T parsed; + readText(parsed, in); + return operator()(parsed, r); + } + } + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return l == stringToUUID(r); + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(r); + T parsed; + readText(parsed, in); + return operator()(l, parsed); + } + } + } -private: - template - bool cantCompare(const T &, const U &) const - { - if constexpr (std::is_same_v) - return false; throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()), - ErrorCodes::BAD_TYPE_OF_FIELD); + ErrorCodes::BAD_TYPE_OF_FIELD); } }; + class FieldVisitorAccurateLess : public StaticVisitor { public: - bool operator() (const UInt64 &, const Null &) const { return false; } - bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const UInt64 & l, const UInt128 & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); } - bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } - - bool operator() (const Int64 &, const Null &) const { return false; } - bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const Int64 & l, const UInt128 & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const Int64 & r) const { return l < r; } - bool operator() (const Int64 & l, const Float64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); } - bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } - - bool operator() (const Float64 &, const Null &) const { return false; } - bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const Float64 & l, const UInt128 & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const Int64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const Float64 & l, const Float64 & r) const { return l < r; } - bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); } - bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } - - template - bool operator() (const Null &, const T &) const - { - return !std::is_same_v; - } - - template - bool operator() (const String & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return stringToUUID(l) < r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const UInt128 & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return l < stringToUUID(r); - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Array & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - - template - bool operator() (const Tuple & l, const T & r) const - { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return false; - return cantCompare(l, r); - } - template - bool operator() (const DecimalField & l, const U & r) const + bool operator() (const T & l, const U & r) const { - if constexpr (isDecimalField()) - return l < r; - if constexpr (std::is_same_v || std::is_same_v) - return l < DecimalField(r, 0); - if constexpr (std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) return false; - return cantCompare(l, r); - } + else + { + if constexpr (std::is_same_v) + return l < r; - template bool operator() (const UInt64 & l, const DecimalField & r) const { return DecimalField(l, 0) < r; } - template bool operator() (const Int64 & l, const DecimalField & r) const { return DecimalField(l, 0) < r; } - template bool operator() (const Float64 &, const DecimalField &) const { return false; } + if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + return accurate::lessOp(l, r); - template - bool operator() (const AggregateFunctionStateData & l, const T & r) const - { - return cantCompare(l, r); - } + if constexpr (isDecimalField() && isDecimalField()) + return l < r; + + if constexpr (isDecimalField() && std::is_arithmetic_v) + return l < DecimalField(r, 0); + + if constexpr (std::is_arithmetic_v && isDecimalField()) + return DecimalField(l, 0) < r; + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return stringToUUID(l) < r; + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(l); + T parsed; + readText(parsed, in); + return operator()(parsed, r); + } + } + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return l < stringToUUID(r); + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(r); + T parsed; + readText(parsed, in); + return operator()(l, parsed); + } + } + } -private: - template - bool cantCompare(const T &, const U &) const - { throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()), - ErrorCodes::BAD_TYPE_OF_FIELD); + ErrorCodes::BAD_TYPE_OF_FIELD); } }; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dad73b6a003..7265e818b51 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -826,8 +826,8 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & cont } bool cast_not_needed = - is_set_const /// Set args are already casted inside Set::createFromAST - || (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast. + is_set_const /// Set args are already casted inside Set::createFromAST + || (isNativeNumber(key_expr_type) && isNativeNumber(const_type)); /// Numbers are accurately compared without cast. if (!cast_not_needed) castValueToType(key_expr_type, const_value, const_type, node); From 3aedef99ce8395dd6e2c947dce4e999e95fa9bc6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 03:24:55 +0300 Subject: [PATCH 39/47] Added a test --- ...onstant_string_in_index_analysis.reference | 12 +++++++ ...with_constant_string_in_index_analysis.sql | 32 +++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.reference create mode 100644 tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql diff --git a/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.reference b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.reference new file mode 100644 index 00000000000..ee98bdf033b --- /dev/null +++ b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.reference @@ -0,0 +1,12 @@ +1 +999999 +100000 +899999 +100001 +900000 +1 +999999 +100000 +899999 +100001 +900000 diff --git a/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql new file mode 100644 index 00000000000..e37f647e81f --- /dev/null +++ b/tests/queries/0_stateless/01312_comparison_with_constant_string_in_index_analysis.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000; +INSERT INTO test SELECT * FROM numbers(1000000); +OPTIMIZE TABLE test; + +SET max_rows_to_read = 2000; +SELECT count() FROM test WHERE x = 100000; +SET max_rows_to_read = 1000000; +SELECT count() FROM test WHERE x != 100000; +SET max_rows_to_read = 101000; +SELECT count() FROM test WHERE x < 100000; +SET max_rows_to_read = 900000; +SELECT count() FROM test WHERE x > 100000; +SET max_rows_to_read = 101000; +SELECT count() FROM test WHERE x <= 100000; +SET max_rows_to_read = 901000; +SELECT count() FROM test WHERE x >= 100000; + +SET max_rows_to_read = 2000; +SELECT count() FROM test WHERE x = '100000'; +SET max_rows_to_read = 1000000; +SELECT count() FROM test WHERE x != '100000'; +SET max_rows_to_read = 101000; +SELECT count() FROM test WHERE x < '100000'; +SET max_rows_to_read = 900000; +SELECT count() FROM test WHERE x > '100000'; +SET max_rows_to_read = 101000; +SELECT count() FROM test WHERE x <= '100000'; +SET max_rows_to_read = 901000; +SELECT count() FROM test WHERE x >= '100000'; + +DROP TABLE test; From 067cf4cc403e512d09a9dbcc4e0178d5b29278d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 07:52:28 +0300 Subject: [PATCH 40/47] Fix gcc build --- src/Common/Arena.h | 2 +- src/Common/ArenaWithFreeLists.h | 2 +- src/Core/Defines.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/Arena.h b/src/Common/Arena.h index d203a92d4a3..aaf71cac525 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -4,7 +4,7 @@ #include #include #include -#if __has_include() +#if __has_include() && defined(ADDRESS_SANITIZER) # include #endif #include diff --git a/src/Common/ArenaWithFreeLists.h b/src/Common/ArenaWithFreeLists.h index 6092f03ce19..3ae727fdaa5 100644 --- a/src/Common/ArenaWithFreeLists.h +++ b/src/Common/ArenaWithFreeLists.h @@ -1,6 +1,6 @@ #pragma once -#if __has_include() +#if __has_include() && defined(ADDRESS_SANITIZER) # include #endif #include diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 13070c565b4..8b26f486c9d 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -87,7 +87,7 @@ #define DBMS_DISTRIBUTED_SIGNATURE_HEADER 0xCAFEDACEull #define DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT 0xCAFECABEull -#if !__has_include() +#if !__has_include() || !defined(ADDRESS_SANITIZER) # define ASAN_UNPOISON_MEMORY_REGION(a, b) # define ASAN_POISON_MEMORY_REGION(a, b) #endif From 8dac30ae955a1ef0b78826d8d7b06594e583263d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 21:42:10 +0300 Subject: [PATCH 41/47] Split file for better build times --- src/Common/FieldVisitors.h | 128 ----------------- src/Common/FieldVisitorsAccurateComparison.h | 142 +++++++++++++++++++ src/Functions/array/arrayIndex.h | 2 +- src/Interpreters/FillingRow.cpp | 2 + src/Interpreters/FillingRow.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 7 files changed, 148 insertions(+), 132 deletions(-) create mode 100644 src/Common/FieldVisitorsAccurateComparison.h diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index 257994a6bd2..ddeddb8fbf6 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -1,10 +1,7 @@ #pragma once #include -#include #include -#include -#include class SipHash; @@ -16,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CONVERT_TYPE; - extern const int BAD_TYPE_OF_FIELD; extern const int LOGICAL_ERROR; } @@ -179,130 +175,6 @@ template <> constexpr bool isDecimalField>() { return tr template <> constexpr bool isDecimalField>() { return true; } -/** More precise comparison, used for index. - * Differs from Field::operator< and Field::operator== in that it also compares values of different types. - * Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query). - */ -class FieldVisitorAccurateEquals : public StaticVisitor -{ -public: - template - bool operator() (const T & l, const U & r) const - { - if constexpr (std::is_same_v || std::is_same_v) - return std::is_same_v; - else - { - if constexpr (std::is_same_v) - return l == r; - - if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) - return accurate::equalsOp(l, r); - - if constexpr (isDecimalField() && isDecimalField()) - return l == r; - - if constexpr (isDecimalField() && std::is_arithmetic_v) - return l == DecimalField(r, 0); - - if constexpr (std::is_arithmetic_v && isDecimalField()) - return DecimalField(l, 0) == r; - - if constexpr (std::is_same_v) - { - if constexpr (std::is_same_v) - return stringToUUID(l) == r; - - if constexpr (std::is_arithmetic_v) - { - ReadBufferFromString in(l); - T parsed; - readText(parsed, in); - return operator()(parsed, r); - } - } - - if constexpr (std::is_same_v) - { - if constexpr (std::is_same_v) - return l == stringToUUID(r); - - if constexpr (std::is_arithmetic_v) - { - ReadBufferFromString in(r); - T parsed; - readText(parsed, in); - return operator()(l, parsed); - } - } - } - - throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()), - ErrorCodes::BAD_TYPE_OF_FIELD); - } -}; - - -class FieldVisitorAccurateLess : public StaticVisitor -{ -public: - template - bool operator() (const T & l, const U & r) const - { - if constexpr (std::is_same_v || std::is_same_v) - return false; - else - { - if constexpr (std::is_same_v) - return l < r; - - if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) - return accurate::lessOp(l, r); - - if constexpr (isDecimalField() && isDecimalField()) - return l < r; - - if constexpr (isDecimalField() && std::is_arithmetic_v) - return l < DecimalField(r, 0); - - if constexpr (std::is_arithmetic_v && isDecimalField()) - return DecimalField(l, 0) < r; - - if constexpr (std::is_same_v) - { - if constexpr (std::is_same_v) - return stringToUUID(l) < r; - - if constexpr (std::is_arithmetic_v) - { - ReadBufferFromString in(l); - T parsed; - readText(parsed, in); - return operator()(parsed, r); - } - } - - if constexpr (std::is_same_v) - { - if constexpr (std::is_same_v) - return l < stringToUUID(r); - - if constexpr (std::is_arithmetic_v) - { - ReadBufferFromString in(r); - T parsed; - readText(parsed, in); - return operator()(l, parsed); - } - } - } - - throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()), - ErrorCodes::BAD_TYPE_OF_FIELD); - } -}; - - /** Implements `+=` operation. * Returns false if the result is zero. */ diff --git a/src/Common/FieldVisitorsAccurateComparison.h b/src/Common/FieldVisitorsAccurateComparison.h new file mode 100644 index 00000000000..91fa4bf28de --- /dev/null +++ b/src/Common/FieldVisitorsAccurateComparison.h @@ -0,0 +1,142 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_TYPE_OF_FIELD; +} + +/** More precise comparison, used for index. + * Differs from Field::operator< and Field::operator== in that it also compares values of different types. + * Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query). + */ +class FieldVisitorAccurateEquals : public StaticVisitor +{ +public: + template + bool operator() (const T & l, const U & r) const + { + if constexpr (std::is_same_v || std::is_same_v) + return std::is_same_v; + else + { + if constexpr (std::is_same_v) + return l == r; + + if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + return accurate::equalsOp(l, r); + + if constexpr (isDecimalField() && isDecimalField()) + return l == r; + + if constexpr (isDecimalField() && std::is_arithmetic_v) + return l == DecimalField(r, 0); + + if constexpr (std::is_arithmetic_v && isDecimalField()) + return DecimalField(l, 0) == r; + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return stringToUUID(l) == r; + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(l); + T parsed; + readText(parsed, in); + return operator()(parsed, r); + } + } + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return l == stringToUUID(r); + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(r); + T parsed; + readText(parsed, in); + return operator()(l, parsed); + } + } + } + + throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()), + ErrorCodes::BAD_TYPE_OF_FIELD); + } +}; + + +class FieldVisitorAccurateLess : public StaticVisitor +{ +public: + template + bool operator() (const T & l, const U & r) const + { + if constexpr (std::is_same_v || std::is_same_v) + return false; + else + { + if constexpr (std::is_same_v) + return l < r; + + if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + return accurate::lessOp(l, r); + + if constexpr (isDecimalField() && isDecimalField()) + return l < r; + + if constexpr (isDecimalField() && std::is_arithmetic_v) + return l < DecimalField(r, 0); + + if constexpr (std::is_arithmetic_v && isDecimalField()) + return DecimalField(l, 0) < r; + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return stringToUUID(l) < r; + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(l); + T parsed; + readText(parsed, in); + return operator()(parsed, r); + } + } + + if constexpr (std::is_same_v) + { + if constexpr (std::is_same_v) + return l < stringToUUID(r); + + if constexpr (std::is_arithmetic_v) + { + ReadBufferFromString in(r); + T parsed; + readText(parsed, in); + return operator()(l, parsed); + } + } + } + + throw Exception("Cannot compare " + demangle(typeid(T).name()) + " with " + demangle(typeid(U).name()), + ErrorCodes::BAD_TYPE_OF_FIELD); + } +}; + +} diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index fab1332cbda..50214ee790f 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index dc48b5347c4..7e32d9514a6 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -1,4 +1,6 @@ #include +#include + namespace DB { diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 1753508e139..0e1d60d0d7a 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include + namespace DB { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f9072e6176a..dc32371b6c1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -37,7 +37,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7265e818b51..281f8511a59 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include From 3663e2a47e1ac04ccd7aa0dd3ce41b8685a2de1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 22:11:32 +0300 Subject: [PATCH 42/47] Fix syntax hilite in CREATE USER query --- src/Parsers/ASTCreateUserQuery.cpp | 3 ++- .../0_stateless/01316_create_user_syntax_hilite.reference | 1 + .../0_stateless/01316_create_user_syntax_hilite.sh | 8 ++++++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01316_create_user_syntax_hilite.reference create mode 100755 tests/queries/0_stateless/01316_create_user_syntax_hilite.sh diff --git a/src/Parsers/ASTCreateUserQuery.cpp b/src/Parsers/ASTCreateUserQuery.cpp index e5c1178285b..1d61303860a 100644 --- a/src/Parsers/ASTCreateUserQuery.cpp +++ b/src/Parsers/ASTCreateUserQuery.cpp @@ -65,7 +65,8 @@ namespace settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IDENTIFIED WITH " << authentication_type_name << (settings.hilite ? IAST::hilite_none : ""); if (password) - settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " BY " << quoteString(*password); + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " BY " << (settings.hilite ? IAST::hilite_none : "") + << quoteString(*password); } diff --git a/tests/queries/0_stateless/01316_create_user_syntax_hilite.reference b/tests/queries/0_stateless/01316_create_user_syntax_hilite.reference new file mode 100644 index 00000000000..ed7daeb3609 --- /dev/null +++ b/tests/queries/0_stateless/01316_create_user_syntax_hilite.reference @@ -0,0 +1 @@ +CREATE USER user IDENTIFIED WITH plaintext_password BY 'hello' diff --git a/tests/queries/0_stateless/01316_create_user_syntax_hilite.sh b/tests/queries/0_stateless/01316_create_user_syntax_hilite.sh new file mode 100755 index 00000000000..1031a96363c --- /dev/null +++ b/tests/queries/0_stateless/01316_create_user_syntax_hilite.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +$CLICKHOUSE_FORMAT --hilite <<< "CREATE USER user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'" From 07ba7ffea52dbc0719cd7eccea77079125e39ebd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 22:23:05 +0300 Subject: [PATCH 43/47] Clear password from command line #11624 --- programs/benchmark/Benchmark.cpp | 5 ++++- programs/client/Client.cpp | 2 ++ src/Common/clearPasswordFromCommandLine.cpp | 18 ++++++++++++++++++ src/Common/clearPasswordFromCommandLine.h | 6 ++++++ src/Common/ya.make | 1 + 5 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 src/Common/clearPasswordFromCommandLine.cpp create mode 100644 src/Common/clearPasswordFromCommandLine.h diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index e17320b39ea..bb814f474e3 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -539,7 +540,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("password", value()->default_value(""), "") ("database", value()->default_value("default"), "") ("stacktrace", "print stack traces of exceptions") - ("confidence", value()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)") + ("confidence", value()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)") ("query_id", value()->default_value(""), "") ; @@ -550,6 +551,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); boost::program_options::notify(options); + clearPasswordFromCommandLine(argc, argv); + if (options.count("help")) { std::cout << "Usage: " << argv[0] << " [options] < queries.txt\n"; diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 7808120d09e..63467c1129d 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -2006,6 +2007,7 @@ public: argsToConfig(common_arguments, config(), 100); + clearPasswordFromCommandLine(argc, argv); } }; diff --git a/src/Common/clearPasswordFromCommandLine.cpp b/src/Common/clearPasswordFromCommandLine.cpp new file mode 100644 index 00000000000..0ff56e25c3f --- /dev/null +++ b/src/Common/clearPasswordFromCommandLine.cpp @@ -0,0 +1,18 @@ +#include +#include "clearPasswordFromCommandLine.h" + +void clearPasswordFromCommandLine(int argc, char ** argv) +{ + for (int arg = 1; arg < argc; ++arg) + { + if (arg + 1 < argc && 0 == strcmp(argv[arg], "--password")) + { + ++arg; + memset(argv[arg], 0, strlen(argv[arg])); + } + else if (0 == strncmp(argv[arg], "--password=", strlen("--password="))) + { + memset(argv[arg] + strlen("--password="), 0, strlen(argv[arg]) - strlen("--password=")); + } + } +} diff --git a/src/Common/clearPasswordFromCommandLine.h b/src/Common/clearPasswordFromCommandLine.h new file mode 100644 index 00000000000..cf90fea1dc8 --- /dev/null +++ b/src/Common/clearPasswordFromCommandLine.h @@ -0,0 +1,6 @@ +#pragma once + +/** If there are --password=... or --password ... arguments in command line, replace their values with zero bytes. + * This is needed to prevent password exposure in 'ps' and similar tools. + */ +void clearPasswordFromCommandLine(int argc, char ** argv); diff --git a/src/Common/ya.make b/src/Common/ya.make index 83a419212bd..327089ff31d 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -30,6 +30,7 @@ SRCS( Config/configReadClient.cpp Config/ConfigReloader.cpp createHardLink.cpp + clearPasswordFromCommandLine.cpp CurrentMetrics.cpp CurrentThread.cpp DNSResolver.cpp From 22366471d03876402a46f1bd4e40602022562cf8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jun 2020 22:31:45 +0300 Subject: [PATCH 44/47] Added a test --- ...1317_no_password_in_command_line.reference | 2 ++ .../01317_no_password_in_command_line.sh | 23 +++++++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/01317_no_password_in_command_line.reference create mode 100755 tests/queries/0_stateless/01317_no_password_in_command_line.sh diff --git a/tests/queries/0_stateless/01317_no_password_in_command_line.reference b/tests/queries/0_stateless/01317_no_password_in_command_line.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/01317_no_password_in_command_line.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/01317_no_password_in_command_line.sh b/tests/queries/0_stateless/01317_no_password_in_command_line.sh new file mode 100755 index 00000000000..1a3ae88616a --- /dev/null +++ b/tests/queries/0_stateless/01317_no_password_in_command_line.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS user" +$CLICKHOUSE_CLIENT --query "CREATE USER user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'" + +# False positive result due to race condition with sleeps is Ok. + +$CLICKHOUSE_CLIENT --user user --password hello --query "SELECT sleep(1)" & +sleep 0.1 +ps auxw | grep -F -- '--password' | grep -F hello ||: +wait + +$CLICKHOUSE_CLIENT --user user --password=hello --query "SELECT sleep(1)" & +sleep 0.1 +ps auxw | grep -F -- '--password' | grep -F hello ||: +wait + +$CLICKHOUSE_CLIENT --query "DROP USER user" From 6467302ad32b4dd6205542675d54fa89944d0ff1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jun 2020 01:29:22 +0300 Subject: [PATCH 45/47] Fix gcc build --- src/Common/Arena.h | 2 +- src/Common/ArenaWithFreeLists.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Arena.h b/src/Common/Arena.h index aaf71cac525..44a9b444ff2 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -4,10 +4,10 @@ #include #include #include +#include #if __has_include() && defined(ADDRESS_SANITIZER) # include #endif -#include #include #include #include diff --git a/src/Common/ArenaWithFreeLists.h b/src/Common/ArenaWithFreeLists.h index 3ae727fdaa5..1284c3586c0 100644 --- a/src/Common/ArenaWithFreeLists.h +++ b/src/Common/ArenaWithFreeLists.h @@ -1,9 +1,9 @@ #pragma once +#include #if __has_include() && defined(ADDRESS_SANITIZER) # include #endif -#include #include #include From 6ddc6d7f085aae86e5ed261be4788a6a19db66cf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jun 2020 07:51:27 +0300 Subject: [PATCH 46/47] Make the test faster #11637 --- tests/queries/0_stateless/01307_multiple_leaders.reference | 4 ++-- tests/queries/0_stateless/01307_multiple_leaders.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01307_multiple_leaders.reference b/tests/queries/0_stateless/01307_multiple_leaders.reference index 576441b288d..62cda31dff8 100644 --- a/tests/queries/0_stateless/01307_multiple_leaders.reference +++ b/tests/queries/0_stateless/01307_multiple_leaders.reference @@ -1,2 +1,2 @@ -2000 1999000 -2000 1999000 +400 79800 +400 79800 diff --git a/tests/queries/0_stateless/01307_multiple_leaders.sh b/tests/queries/0_stateless/01307_multiple_leaders.sh index 0bf5e0b13bf..e19a10bcecb 100755 --- a/tests/queries/0_stateless/01307_multiple_leaders.sh +++ b/tests/queries/0_stateless/01307_multiple_leaders.sh @@ -22,8 +22,8 @@ function thread() } -thread 0 1000 & -thread 1 1000 & +thread 0 200 & +thread 1 200 & wait From eccd8d61dd5b67220267a171841827bfebcc2eca Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 15 Jun 2020 10:13:29 +0300 Subject: [PATCH 47/47] Update build.py --- docs/tools/build.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index 95e887f046f..b7ddbc29629 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -220,7 +220,7 @@ if __name__ == '__main__': arg_parser.add_argument('--website-dir', default=website_dir) arg_parser.add_argument('--output-dir', default='build') arg_parser.add_argument('--enable-stable-releases', action='store_true') - arg_parser.add_argument('--stable-releases-limit', type=int, default='4') + arg_parser.add_argument('--stable-releases-limit', type=int, default='3') arg_parser.add_argument('--lts-releases-limit', type=int, default='2') arg_parser.add_argument('--nav-limit', type=int, default='0') arg_parser.add_argument('--version-prefix', type=str, default='')