From e118d055da129e7bf59c082ec818678e06487c7e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Oct 2015 22:00:50 +0300 Subject: [PATCH 01/16] dbms: fixed error with OLAP compatibility layer [#METR-17929]. --- dbms/src/Server/OLAPHTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Server/OLAPHTTPHandler.cpp b/dbms/src/Server/OLAPHTTPHandler.cpp index ca212941208..5e0bf53e193 100644 --- a/dbms/src/Server/OLAPHTTPHandler.cpp +++ b/dbms/src/Server/OLAPHTTPHandler.cpp @@ -50,8 +50,8 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco: Context context = *server.global_context; context.setGlobalContext(*server.global_context); - context.setUser(user, password, request.clientAddress().host(), quota_key); context.setSetting("profile", profile); + context.setUser(user, password, request.clientAddress().host(), quota_key); context.setInterface(Context::Interface::OLAP_HTTP); context.setHTTPMethod(Context::HTTPMethod::POST); From 0a7b4dd536ec2c6cd8747bffcb6f7b92d733e4ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Oct 2015 23:01:09 +0300 Subject: [PATCH 02/16] dbms: fixed little error [#METR-16457]. --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f110b117ac5..14725490205 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -389,6 +389,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( if (sum_marks > 0 && settings.merge_tree_uniform_read_distribution == 1) { + /// Уменьшим количество потоков, если данных мало. + if (sum_marks < threads * min_marks_for_concurrent_read && parts.size() < threads) + threads = std::max((sum_marks + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, parts.size()); + MergeTreeReadPoolPtr pool = std::make_shared( threads, sum_marks, min_marks_for_concurrent_read, parts, data, prewhere_actions, prewhere_column, true, column_names); From a7fc2040d91de94e60196d7e206460f33c1c4e9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Oct 2015 00:22:49 +0300 Subject: [PATCH 03/16] dbms: handling of compatible types in IN section with explicitly provided set [#METR-18382]. --- dbms/src/Interpreters/Set.cpp | 212 +++++++++++++++------------------- 1 file changed, 90 insertions(+), 122 deletions(-) diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 8d74f222b45..b87ff47b359 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -230,144 +230,106 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set) } -/** Чтобы корректно работали выражения вида 1.0 IN (1). +/** Проверка попадания Field from, имеющим тип From в диапазон значений типа To. + * From и To - числовые типы. Могут быть типами с плавающей запятой. + * From - это одно из UInt64, Int64, Float64, + * тогда как To может быть также 8, 16, 32 битным. + * + * Если попадает в диапазон, то from конвертируется в Field ближайшего к To типа. + * Если не попадает - возвращается Field(Null). + */ + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" + +template +static Field convertNumericTypeImpl(const Field & from) +{ + From value = from.get(); + + if (value != To(value) + || (std::is_unsigned::value && std::is_unsigned::value && value > std::numeric_limits::max()) + || (std::is_signed::value && std::is_signed::value && (value < std::numeric_limits::min() || value > std::numeric_limits::max())) + || (std::is_signed::value && std::is_unsigned::value && (value < 0 || static_cast(value) > std::numeric_limits::max())) + || (std::is_unsigned::value && std::is_signed::value && value > static_cast(std::numeric_limits::max()))) + return {}; + + return Field(typename NearestFieldType::Type(value)); +} + +#pragma GCC diagnostic pop + +template +static Field convertNumericType(const Field & from, const IDataType & type) +{ + if (from.getType() == Field::Types::UInt64) + return convertNumericTypeImpl(from); + if (from.getType() == Field::Types::Int64) + return convertNumericTypeImpl(from); + if (from.getType() == Field::Types::Float64) + return convertNumericTypeImpl(from); + + throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " + + Field::Types::toString(from.getType()) + " at right"); +} + + +/** Чтобы корректно работали выражения вида 1.0 IN (1) или чтобы 1 IN (1, 2.0, 2.5, -1) работало так же, как 1 IN (1, 2). * Проверяет совместимость типов, проверяет попадание значений в диапазон допустимых значений типа, делает преобразование типа. - * Код слегка дурацкий. + * Если значение не попадает в диапазон - возвращает Null. */ static Field convertToType(const Field & src, const IDataType & type) { if (type.isNumeric()) { - bool is_uint8 = false; - bool is_uint16 = false; - bool is_uint32 = false; - bool is_uint64 = false; - bool is_int8 = false; - bool is_int16 = false; - bool is_int32 = false; - bool is_int64 = false; - bool is_float32 = false; - bool is_float64 = false; - bool is_date = false; - bool is_datetime = false; + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); + if (typeid_cast(&type)) return convertNumericType(src, type); - false - || (is_uint8 = typeid_cast(&type)) - || (is_uint16 = typeid_cast(&type)) - || (is_uint32 = typeid_cast(&type)) - || (is_uint64 = typeid_cast(&type)) - || (is_int8 = typeid_cast(&type)) - || (is_int16 = typeid_cast(&type)) - || (is_int32 = typeid_cast(&type)) - || (is_int64 = typeid_cast(&type)) - || (is_float32 = typeid_cast(&type)) - || (is_float64 = typeid_cast(&type)) - || (is_date = typeid_cast(&type)) - || (is_datetime = typeid_cast(&type)) - ; + bool is_date = typeid_cast(&type); + bool is_datetime = typeid_cast(&type); - if (is_uint8 || is_uint16 || is_uint32 || is_uint64) + if (!is_date && !is_datetime) + throw Exception("Logical error: unknown numeric type " + type.getName(), ErrorCodes::LOGICAL_ERROR); + + if (src.getType() == Field::Types::UInt64) + return src; + + if (src.getType() == Field::Types::String) { - if (src.getType() == Field::Types::Int64) - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, signed at right"); + /// Возможность сравнивать даты и даты-с-временем со строкой. + const String & str = src.get(); + ReadBufferFromString in(str); - if (src.getType() == Field::Types::Float64) - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right"); - - if (src.getType() == Field::Types::UInt64) + if (is_date) { - UInt64 value = src.get(); - if ((is_uint8 && value > std::numeric_limits::max()) - || (is_uint16 && value > std::numeric_limits::max()) - || (is_uint32 && value > std::numeric_limits::max())) - throw Exception("Value (" + toString(value) + ") in IN section is out of range of type " + type.getName() + " at left"); + DayNum_t date{}; + readDateText(date, in); + if (!in.eof()) + throw Exception("String is too long for Date: " + str); - return src; + return Field(UInt64(date)); } - - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " at right"); - } - else if (is_int8 || is_int16 || is_int32 || is_int64) - { - if (src.getType() == Field::Types::Float64) - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, floating point at right"); - - if (src.getType() == Field::Types::UInt64) + else { - UInt64 value = src.get(); + time_t date_time{}; + readDateTimeText(date_time, in); + if (!in.eof()) + throw Exception("String is too long for DateTime: " + str); - if ((is_int8 && value > uint8_t(std::numeric_limits::max())) - || (is_int16 && value > uint16_t(std::numeric_limits::max())) - || (is_int32 && value > uint32_t(std::numeric_limits::max())) - || (is_int64 && value > uint64_t(std::numeric_limits::max()))) - throw Exception("Value (" + toString(value) + ") in IN section is out of range of type " + type.getName() + " at left"); - - return Field(Int64(value)); + return Field(UInt64(date_time)); } - - if (src.getType() == Field::Types::Int64) - { - Int64 value = src.get(); - if ((is_int8 && (value < std::numeric_limits::min() || value > std::numeric_limits::max())) - || (is_int16 && (value < std::numeric_limits::min() || value > std::numeric_limits::max())) - || (is_int32 && (value < std::numeric_limits::min() || value > std::numeric_limits::max()))) - throw Exception("Value (" + toString(value) + ") in IN section is out of range of type " + type.getName() + " at left"); - - return src; - } - - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " at right"); } - else if (is_float32 || is_float64) - { - if (src.getType() == Field::Types::UInt64) - return Field(Float64(src.get())); - if (src.getType() == Field::Types::Int64) - return Field(Float64(src.get())); - - if (src.getType() == Field::Types::Float64) - return src; - - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " at right"); - } - else if (is_date || is_datetime) - { - if (src.getType() == Field::Types::UInt64) - return src; - - if (src.getType() == Field::Types::String) - { - /// Возможность сравнивать даты и даты-с-временем со строкой. - const String & str = src.get(); - ReadBufferFromString in(str); - - if (is_date) - { - DayNum_t date{}; - readDateText(date, in); - if (!in.eof()) - throw Exception("String is too long for Date: " + str); - - return Field(UInt64(date)); - } - else - { - time_t date_time{}; - readDateTimeText(date_time, in); - if (!in.eof()) - throw Exception("String is too long for DateTime: " + str); - - return Field(UInt64(date_time)); - } - } - - throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " at right"); - } + throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " + + Field::Types::toString(src.getType()) + " at right"); } else { @@ -448,7 +410,10 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, const Context & context, { if (data_types.size() == 1) { - block.getByPosition(0).column->insert(extractValueFromNode(*it, *data_types[0], context)); + Field value = extractValueFromNode(*it, *data_types[0], context); + + if (!value.isNull()) + block.getByPosition(0).column->insert(value); } else if (ASTFunction * func = typeid_cast(&**it)) { @@ -461,7 +426,10 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, const Context & context, for (size_t j = 0; j < tuple_size; ++j) { - block.getByPosition(j).column->insert(extractValueFromNode(func->arguments->children[j], *data_types[j], context)); + Field value = extractValueFromNode(func->arguments->children[j], *data_types[j], context); + + if (!value.isNull()) + block.getByPosition(j).column->insert(value); } } else From 75c6e83b7150ca265a4ce2818905c3d5bebb2311 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Oct 2015 00:29:17 +0300 Subject: [PATCH 04/16] dbms: added test [#METR-18382]. --- .../00239_type_conversion_in_in.reference | 14 ++++++++++++++ .../0_stateless/00239_type_conversion_in_in.sql | 13 +++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00239_type_conversion_in_in.reference create mode 100644 dbms/tests/queries/0_stateless/00239_type_conversion_in_in.sql diff --git a/dbms/tests/queries/0_stateless/00239_type_conversion_in_in.reference b/dbms/tests/queries/0_stateless/00239_type_conversion_in_in.reference new file mode 100644 index 00000000000..c699eb7014b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00239_type_conversion_in_in.reference @@ -0,0 +1,14 @@ +1 1 +1 1 +1 0 +1 1 1 1 0 1 +0 +0 +1 +0 +0 +1 +0 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00239_type_conversion_in_in.sql b/dbms/tests/queries/0_stateless/00239_type_conversion_in_in.sql new file mode 100644 index 00000000000..6e76a31ac56 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00239_type_conversion_in_in.sql @@ -0,0 +1,13 @@ +select 1 as x, x = 1 or x = 2 or x = 3 or x = -1; +select 1 as x, x = 1.0 or x = 2 or x = 3 or x = -1; +select 1 as x, x = 1.5 or x = 2 or x = 3 or x = -1; + +SELECT + 1 IN (1, -1, 2.0, 2.5), + 1.0 IN (1, -1, 2.0, 2.5), + 1 IN (1.0, -1, 2.0, 2.5), + 1.0 IN (1.0, -1, 2.0, 2.5), + 1 IN (1.1, -1, 2.0, 2.5), + -1 IN (1, -1, 2.0, 2.5); + +SELECT -number IN (1, 2, 3, -5.0, -2.0) FROM system.numbers LIMIT 10; From 8e4f30fff020afb0273d740eccfef3f88c02df65 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Fri, 9 Oct 2015 17:51:31 +0300 Subject: [PATCH 05/16] dbms: MongoDB dictionary source draft (actually it works, but there are some improvements to be made) [#METR-17854] --- dbms/include/DB/Core/ErrorCodes.h | 1 + .../Dictionaries/ClickHouseDictionarySource.h | 2 +- .../DB/Dictionaries/DictionarySourceFactory.h | 6 + .../DB/Dictionaries/MongoDBBlockInputStream.h | 316 ++++++++++++++++++ .../DB/Dictionaries/MongoDBDictionarySource.h | 128 +++++++ .../DB/Dictionaries/MySQLDictionarySource.h | 3 +- 6 files changed, 453 insertions(+), 3 deletions(-) create mode 100644 dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h create mode 100644 dbms/include/DB/Dictionaries/MongoDBDictionarySource.h diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index 8b5fb98e94f..ffcc6c5f6ec 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -294,6 +294,7 @@ namespace ErrorCodes LIMIT_EXCEEDED = 290, DATABASE_ACCESS_DENIED = 291, LEADERSHIP_CHANGED = 292, + MONGODB_INIT_FAILED = 293, KEEPER_EXCEPTION = 999, POCO_EXCEPTION = 1000, diff --git a/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h b/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h index d7ecab46e1d..acd47287bde 100644 --- a/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h +++ b/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h @@ -25,7 +25,7 @@ public: ClickHouseDictionarySource(const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - Block & sample_block, Context & context) + const Block & sample_block, Context & context) : dict_struct{dict_struct}, host{config.getString(config_prefix + ".host")}, port(config.getInt(config_prefix + ".port")), diff --git a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h index aeffaffe6ff..5b331441ade 100644 --- a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h +++ b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -84,6 +85,11 @@ public: return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context); } + else if ("mongodb" == source_type) + { + return std::make_unique(dict_struct, config, config_prefix + ".mongodb", + sample_block, context); + } throw Exception{ name + ": unknown dictionary source type: " + source_type, diff --git a/dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h b/dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h new file mode 100644 index 00000000000..585ff562f2c --- /dev/null +++ b/dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h @@ -0,0 +1,316 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining +class MongoDBBlockInputStream final : public IProfilingBlockInputStream +{ + enum struct value_type_t + { + UInt8, + UInt16, + UInt32, + UInt64, + Int8, + Int16, + Int32, + Int64, + Float32, + Float64, + String, + Date, + DateTime + }; + +public: + MongoDBBlockInputStream( + std::unique_ptr cursor_, const Block & sample_block, const std::size_t max_block_size) + : cursor{std::move(cursor_)}, sample_block{sample_block}, max_block_size{max_block_size} + { + if (!cursor->more()) + return; + + types.reserve(sample_block.columns()); + + for (const auto idx : ext::range(0, sample_block.columns())) + { + const auto & column = sample_block.getByPosition(idx); + const auto type = column.type.get(); + + if (typeid_cast(type)) + types.push_back(value_type_t::UInt8); + else if (typeid_cast(type)) + types.push_back(value_type_t::UInt16); + else if (typeid_cast(type)) + types.push_back(value_type_t::UInt32); + else if (typeid_cast(type)) + types.push_back(value_type_t::UInt64); + else if (typeid_cast(type)) + types.push_back(value_type_t::Int8); + else if (typeid_cast(type)) + types.push_back(value_type_t::Int16); + else if (typeid_cast(type)) + types.push_back(value_type_t::Int32); + else if (typeid_cast(type)) + types.push_back(value_type_t::Int64); + else if (typeid_cast(type)) + types.push_back(value_type_t::Float32); + else if (typeid_cast(type)) + types.push_back(value_type_t::Float64); + else if (typeid_cast(type)) + types.push_back(value_type_t::String); + else if (typeid_cast(type)) + types.push_back(value_type_t::Date); + else if (typeid_cast(type)) + types.push_back(value_type_t::DateTime); + else + throw Exception{ + "Unsupported type " + type->getName(), + ErrorCodes::UNKNOWN_TYPE + }; + + names.emplace_back(column.name); + } + } + + String getName() const override { return "MongoDB"; } + + String getID() const override + { + using stream = std::ostringstream; + + return "MongoDB(@" + static_cast(stream{} << cursor.get()).str() + ")"; + } + +private: + Block readImpl() override + { + if (!cursor->more()) + return {}; + + + auto block = sample_block.cloneEmpty(); + + /// cache pointers returned by the calls to getByPosition + std::vector columns(block.columns()); + const auto size = columns.size(); + + for (const auto i : ext::range(0, size)) + columns[i] = block.getByPosition(i).column.get(); + + std::size_t num_rows = 0; + while (cursor->more()) + { + const auto row = cursor->next(); + + for (const auto idx : ext::range(0, size)) + { + const auto value = row[names[idx]]; + if (value.ok()) + insertValue(columns[idx], types[idx], value); + else + insertDefaultValue(columns[idx], types[idx]); + } + + ++num_rows; + if (num_rows == max_block_size) + break; + } + + return block; + } + + static void insertValue(IColumn * const column, const value_type_t type, const mongo::BSONElement & value) + { + switch (type) + { + case value_type_t::UInt8: + { + if (value.type() != mongo::Bool) + throw Exception{ + "Type mismatch, expected Bool, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.boolean()); + break; + } + case value_type_t::UInt16: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.numberInt()); + break; + } + case value_type_t::UInt32: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.numberInt()); + break; + } + case value_type_t::UInt64: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.numberLong()); + break; + } + case value_type_t::Int8: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.numberInt()); + break; + } + case value_type_t::Int16: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.numberInt()); + break; + } + case value_type_t::Int32: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.numberInt()); + break; + } + case value_type_t::Int64: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.numberLong()); + break; + } + case value_type_t::Float32: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.number()); + break; + } + case value_type_t::Float64: + { + if (!value.isNumber()) + throw Exception{ + "Type mismatch, expected a number, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.number()); + break; + } + case value_type_t::String: + { + if (value.type() != mongo::String) + throw Exception{ + "Type mismatch, expected String, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + const auto string = value.String(); + static_cast(column)->insertDataWithTerminatingZero(string.data(), string.size() + 1); + break; + } + case value_type_t::Date: + { + if (value.type() != mongo::Date) + throw Exception{ + "Type mismatch, expected Date, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert( + UInt16{DateLUT::instance().toDayNum(value.date().toTimeT())}); + break; + } + case value_type_t::DateTime: + { + if (value.type() != mongo::Date) + throw Exception{ + "Type mismatch, expected Date, got " + std::string{mongo::typeName(value.type())}, + ErrorCodes::TYPE_MISMATCH + }; + + static_cast(column)->insert(value.date().toTimeT()); + break; + } + } + } + + /// @todo insert default value from the dictionary attribute definition + static void insertDefaultValue(IColumn * const column, const value_type_t type) + { + switch (type) + { + case value_type_t::UInt8: static_cast(column)->insertDefault(); break; + case value_type_t::UInt16: static_cast(column)->insertDefault(); break; + case value_type_t::UInt32: static_cast(column)->insertDefault(); break; + case value_type_t::UInt64: static_cast(column)->insertDefault(); break; + case value_type_t::Int8: static_cast(column)->insertDefault(); break; + case value_type_t::Int16: static_cast(column)->insertDefault(); break; + case value_type_t::Int32: static_cast(column)->insertDefault(); break; + case value_type_t::Int64: static_cast(column)->insertDefault(); break; + case value_type_t::Float32: static_cast(column)->insertDefault(); break; + case value_type_t::Float64: static_cast(column)->insertDefault(); break; + case value_type_t::String: static_cast(column)->insertDefault(); break; + case value_type_t::Date: static_cast(column)->insertDefault(); break; + case value_type_t::DateTime: static_cast(column)->insertDefault(); break; + } + } + + std::unique_ptr cursor; + Block sample_block; + const std::size_t max_block_size; + std::vector types; + std::vector names; +}; + +} diff --git a/dbms/include/DB/Dictionaries/MongoDBDictionarySource.h b/dbms/include/DB/Dictionaries/MongoDBDictionarySource.h new file mode 100644 index 00000000000..ecbcd11c95c --- /dev/null +++ b/dbms/include/DB/Dictionaries/MongoDBDictionarySource.h @@ -0,0 +1,128 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/// Allows loading dictionaries from a MySQL database +class MongoDBDictionarySource final : public IDictionarySource +{ + MongoDBDictionarySource( + const DictionaryStructure & dict_struct, const std::string & host, const std::string & port, + const std::string & user, const std::string & password, + const std::string & db, const std::string & collection, + const Block & sample_block, Context & context) + : dict_struct{dict_struct}, host{host}, port{port}, user{user}, password{password}, + db{db}, collection{collection}, sample_block{sample_block}, context(context), + connection{true} + { + init(); + + connection.connect(host + ':' + port); + + if (!user.empty()) + { + std::string error; + if (!connection.auth(db, user, password, error)) + throw DB::Exception{ + "Could not authenticate to a MongoDB database " + db + " with provided credentials: " + error, + ErrorCodes::WRONG_PASSWORD + }; + } + + /// compose BSONObj containing all requested fields + mongo::BSONObjBuilder builder; + builder << "_id" << 0; + + for (const auto & column : sample_block.getColumns()) + builder << column.name << 1; + + fields_to_query = builder.obj(); + } + + static void init() + { + static const auto mongo_init_status = mongo::client::initialize(); + + if (!mongo_init_status.isOK()) + throw DB::Exception{ + "mongo::client::initialize() failed: " + mongo_init_status.toString(), + ErrorCodes::MONGODB_INIT_FAILED + }; + + LOG_TRACE(&Logger::get("MongoDBDictionarySource"), "mongo::client::initialize() ok"); + } + +public: + MongoDBDictionarySource( + const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, Block & sample_block, Context & context) + : MongoDBDictionarySource{ + dict_struct, + config.getString(config_prefix + ".host"), + config.getString(config_prefix + ".port"), + config.getString(config_prefix + ".user", ""), + config.getString(config_prefix + ".password", ""), + config.getString(config_prefix + ".db", ""), + config.getString(config_prefix + ".collection"), + sample_block, context + } + { + } + + MongoDBDictionarySource(const MongoDBDictionarySource & other) + : MongoDBDictionarySource{ + other.dict_struct, other.host, other.port, other.user, other.password, + other.db, other.collection, other.sample_block, other.context + } + { + } + + BlockInputStreamPtr loadAll() override + { + return new MongoDBBlockInputStream{ + connection.query(db + '.' + collection, {}, 0, 0, &fields_to_query), + sample_block, 8192 + }; + } + + bool supportsSelectiveLoad() const override { return true; } + + BlockInputStreamPtr loadIds(const std::vector & ids) override + { + return new MongoDBBlockInputStream{ + connection.query(db + '.' + collection, {}, 0, 0, &fields_to_query), + sample_block, 8192 + }; + } + + bool isModified() const override { return false; } + + DictionarySourcePtr clone() const override { return std::make_unique(*this); } + + std::string toString() const override + { + return "MongoDB: " + db + '.' + collection + ',' + (user.empty() ? " " : " " + user + '@') + host + ':' + port; + } + +private: + const DictionaryStructure dict_struct; + const std::string host; + const std::string port; + const std::string user; + const std::string password; + const std::string db; + const std::string collection; + Block sample_block; + Context & context; + + mongo::DBClientConnection connection; + mongo::BSONObj fields_to_query; +}; + +} diff --git a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h index b8f2e5e8d25..eb77c91ef77 100644 --- a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h +++ b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h @@ -19,7 +19,7 @@ class MySQLDictionarySource final : public IDictionarySource public: MySQLDictionarySource(const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - Block & sample_block) + const Block & sample_block) : dict_struct{dict_struct}, db{config.getString(config_prefix + ".db", "")}, table{config.getString(config_prefix + ".table")}, @@ -78,7 +78,6 @@ public: private: Logger * log = &Logger::get("MySQLDictionarySource"); - static std::string quoteForLike(const std::string s) { std::string tmp; From d5caf72ca7edbdd3d54546c9d0faef7aa7f7e926 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Fri, 9 Oct 2015 17:52:45 +0300 Subject: [PATCH 06/16] Merge --- dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h b/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h index 165ea34b61b..45946c2eaba 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -225,6 +225,11 @@ private: std::size_t getIdentifiersColumnSize(const IdentifierNameSet & identifiers) const { + /** for expressions containing no columns (or where columns could not be determined otherwise) assume maximum + * possible size so they do not have priority in eligibility over other expressions. */ + if (identifiers.empty()) + return std::numeric_limits::max(); + std::size_t size{}; for (const auto & identifier : identifiers) From 3733ac9ead01c046dc9888ee5fc1b9ffeaecfb37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Oct 2015 23:21:20 +0300 Subject: [PATCH 07/16] dbms: fixed comment [#METR-17854]. --- dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h b/dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h index 585ff562f2c..af322581359 100644 --- a/dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h +++ b/dbms/include/DB/Dictionaries/MongoDBBlockInputStream.h @@ -15,7 +15,7 @@ namespace DB { -/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining +/// Allows processing results of a MongoDB query as a sequence of Blocks, simplifies chaining class MongoDBBlockInputStream final : public IProfilingBlockInputStream { enum struct value_type_t @@ -101,7 +101,6 @@ private: if (!cursor->more()) return {}; - auto block = sample_block.cloneEmpty(); /// cache pointers returned by the calls to getByPosition From 31e94fb6cbf0b27736bb42a0df3df34bd36cb4a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Oct 2015 23:22:33 +0300 Subject: [PATCH 08/16] =?UTF-8?q?dbms:=20OLAP=20compatibility:=20fixed=20e?= =?UTF-8?q?rror=20for=20=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88?= =?UTF-8?q?=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88.yandex-t?= =?UTF-8?q?eam.ru=20[#METR-17929].?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbms/src/Server/OLAPQueryConverter.cpp | 25 ++++++++++++++----------- dbms/src/Server/OLAPQueryParser.cpp | 6 +++--- dbms/src/Server/OLAPQueryParser.h | 2 ++ 3 files changed, 19 insertions(+), 14 deletions(-) diff --git a/dbms/src/Server/OLAPQueryConverter.cpp b/dbms/src/Server/OLAPQueryConverter.cpp index b2f7c4e1a95..6d51912e6b7 100644 --- a/dbms/src/Server/OLAPQueryConverter.cpp +++ b/dbms/src/Server/OLAPQueryConverter.cpp @@ -48,7 +48,7 @@ void QueryConverter::OLAPServerQueryToClickHouse(const QueryParseResult & query, if (query.format != FORMAT_TAB) throw Exception("Only tab-separated output format is supported", ErrorCodes::UNSUPPORTED_PARAMETER); - /// Учтем некоторые настройки (пока далеко не все). + /// Учтем некоторые настройки (далеко не все). Settings new_settings = inout_context.getSettings(); @@ -61,17 +61,20 @@ void QueryConverter::OLAPServerQueryToClickHouse(const QueryParseResult & query, if (query.max_result_size != 0) new_settings.limits.max_rows_to_group_by = query.max_result_size; - switch (query.overflow_mode) + if (query.has_overflow_mode) { - case OLAP::OVERFLOW_MODE_THROW: - new_settings.limits.group_by_overflow_mode = DB::OverflowMode::THROW; - break; - case OLAP::OVERFLOW_MODE_BREAK: - new_settings.limits.group_by_overflow_mode = DB::OverflowMode::BREAK; - break; - case OLAP::OVERFLOW_MODE_ANY: - new_settings.limits.group_by_overflow_mode = DB::OverflowMode::ANY; - break; + switch (query.overflow_mode) + { + case OLAP::OVERFLOW_MODE_THROW: + new_settings.limits.group_by_overflow_mode = DB::OverflowMode::THROW; + break; + case OLAP::OVERFLOW_MODE_BREAK: + new_settings.limits.group_by_overflow_mode = DB::OverflowMode::BREAK; + break; + case OLAP::OVERFLOW_MODE_ANY: + new_settings.limits.group_by_overflow_mode = DB::OverflowMode::ANY; + break; + } } inout_context.setSettings(new_settings); diff --git a/dbms/src/Server/OLAPQueryParser.cpp b/dbms/src/Server/OLAPQueryParser.cpp index c15ddf622b5..7891b5c5613 100644 --- a/dbms/src/Server/OLAPQueryParser.cpp +++ b/dbms/src/Server/OLAPQueryParser.cpp @@ -100,7 +100,6 @@ QueryParseResult QueryParser::parse(std::istream & s) result.cut_date_last = false; result.cut_dates_for_goals = false; - result.overflow_mode = OVERFLOW_MODE_THROW; result.concurrency = 0; result.max_threads_per_counter = 0; result.limit = 0; @@ -152,9 +151,10 @@ QueryParseResult QueryParser::parse(std::istream & s) throw Exception(std::string("Unknown overflow mode: ") + overflow_mode_str, ErrorCodes::UNKNOWN_OVERFLOW_MODE); + result.has_overflow_mode = true; result.overflow_mode = overflow_mode_str == "throw" ? OVERFLOW_MODE_THROW - : (overflow_mode_str == "break" ? OVERFLOW_MODE_BREAK - : OVERFLOW_MODE_ANY); + : (overflow_mode_str == "break" ? OVERFLOW_MODE_BREAK + : OVERFLOW_MODE_ANY); } else if (settings_child_nodes->item(i)->nodeName() == "concurrency") { diff --git a/dbms/src/Server/OLAPQueryParser.h b/dbms/src/Server/OLAPQueryParser.h index ba2ef30c1f7..bfde55407c4 100644 --- a/dbms/src/Server/OLAPQueryParser.h +++ b/dbms/src/Server/OLAPQueryParser.h @@ -87,6 +87,8 @@ struct QueryParseResult float sample; Format format; + + bool has_overflow_mode = false; OverflowMode overflow_mode; Poco::AutoPtr query; From 62061c396915b04b5bb5b908add648e20a651abb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Oct 2015 04:08:39 +0300 Subject: [PATCH 09/16] dbms: temporarily disabled MongoDB dictionary source (to enable automatic builds) [#METR-17854]. --- dbms/include/DB/Dictionaries/DictionarySourceFactory.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h index 5b331441ade..20af7e8331c 100644 --- a/dbms/include/DB/Dictionaries/DictionarySourceFactory.h +++ b/dbms/include/DB/Dictionaries/DictionarySourceFactory.h @@ -5,7 +5,7 @@ #include #include #include -#include +//#include #include #include #include @@ -85,11 +85,11 @@ public: return std::make_unique(dict_struct, config, config_prefix + ".clickhouse", sample_block, context); } - else if ("mongodb" == source_type) +/* else if ("mongodb" == source_type) { return std::make_unique(dict_struct, config, config_prefix + ".mongodb", sample_block, context); - } + }*/ throw Exception{ name + ": unknown dictionary source type: " + source_type, From ff06c18850eef6fe8c352aafaadf9aeba55d8957 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 11 Oct 2015 07:25:02 +0300 Subject: [PATCH 10/16] dbms: fixed error with 'replace' functions [#METR-18401]. --- .../DB/Functions/FunctionsStringSearch.h | 17 +- .../DB/Functions/FunctionsVisitParam.h | 4 +- .../00240_replace_substring_loop.reference | 192 ++++++++++++++++++ .../00240_replace_substring_loop.sql | 101 +++++++++ 4 files changed, 304 insertions(+), 10 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00240_replace_substring_loop.reference create mode 100644 dbms/tests/queries/0_stateless/00240_replace_substring_loop.sql diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index 3f3cbe17d8d..4da88809724 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -75,7 +75,7 @@ struct PositionImpl while (pos < end && end != (pos = searcher.search(pos, end - pos))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res[i] = 0; ++i; @@ -163,7 +163,7 @@ struct PositionUTF8Impl while (pos < end && end != (pos = searcher.search(pos, end - pos))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res[i] = 0; ++i; @@ -364,7 +364,7 @@ public: while (pos < end && end != (pos = searcher.find(pos, end))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res[i] = 0; ++i; @@ -627,7 +627,7 @@ public: while (pos < end && end != (pos = searcher.find(pos, end))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res[i] = 0; ++i; @@ -898,7 +898,7 @@ struct MatchImpl while (pos < end && end != (pos = searcher.search(pos, end - pos))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res[i] = revert; ++i; @@ -965,7 +965,7 @@ struct MatchImpl while (pos < end && end != (pos = searcher.search(pos, end - pos))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res[i] = revert; ++i; @@ -1361,7 +1361,7 @@ struct ReplaceStringImpl memcpy(&res_data[res_offset], pos, match - pos); /// Определим, к какому индексу оно относится. - while (i < offsets.size() && begin + offsets[i] < match) + while (i < offsets.size() && begin + offsets[i] <= match) { res_offsets[i] = res_offset + ((begin + offsets[i]) - pos); ++i; @@ -1398,6 +1398,7 @@ struct ReplaceStringImpl res_offset += (begin + offsets[i] - pos); res_offsets[i] = res_offset; pos = begin + offsets[i]; + ++i; } } } @@ -1430,7 +1431,7 @@ struct ReplaceStringImpl memcpy(&res_data[res_offset], pos, match - pos); /// Определим, к какому индексу оно относится. - while (i < size && begin + n * (i + 1) < match) + while (i < size && begin + n * (i + 1) <= match) { res_offsets[i] = res_offset + ((begin + n * (i + 1)) - pos); ++i; diff --git a/dbms/include/DB/Functions/FunctionsVisitParam.h b/dbms/include/DB/Functions/FunctionsVisitParam.h index 90282d3e734..7770664a69c 100644 --- a/dbms/include/DB/Functions/FunctionsVisitParam.h +++ b/dbms/include/DB/Functions/FunctionsVisitParam.h @@ -306,7 +306,7 @@ struct ExtractParamImpl while (pos < end && end != (pos = searcher.search(pos, end - pos))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res[i] = 0; ++i; @@ -369,7 +369,7 @@ struct ExtractParamToStringImpl while (pos < end && end != (pos = searcher.search(pos, end - pos))) { /// Определим, к какому индексу оно относится. - while (begin + offsets[i] < pos) + while (begin + offsets[i] <= pos) { res_data.push_back(0); res_offsets[i] = res_data.size(); diff --git a/dbms/tests/queries/0_stateless/00240_replace_substring_loop.reference b/dbms/tests/queries/0_stateless/00240_replace_substring_loop.reference new file mode 100644 index 00000000000..390ec161dc2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00240_replace_substring_loop.reference @@ -0,0 +1,192 @@ +. . . 1 +. . . 1 +. . . 1 +._ .o .o 1 +. . . 1 +_. o. o. 1 +. . . 1 +_._ o.o o.o 1 +._ .o .o 1 +. . . 1 +._ .o .o 1 +._ .o .o 1 +._ .o .o 1 +_. o. o. 1 +._ .o .o 1 +_._ o.o o.o 1 +_. o. o. 1 +. . . 1 +_. o. o. 1 +._ .o .o 1 +_. o. o. 1 +_. o. o. 1 +_. o. o. 1 +_._ o.o o.o 1 +_._ o.o o.o 1 +. . . 1 +_._ o.o o.o 1 +._ .o .o 1 +_._ o.o o.o 1 +_. o. o. 1 +_._ o.o o.o 1 +_._ o.o o.o 1 +. . . 1 +. . . 1 +. . . 1 +._ .oo .oo 1 +. . . 1 +_. oo. oo. 1 +. . . 1 +_._ oo.oo oo.oo 1 +._ .oo .oo 1 +. . . 1 +._ .oo .oo 1 +._ .oo .oo 1 +._ .oo .oo 1 +_. oo. oo. 1 +._ .oo .oo 1 +_._ oo.oo oo.oo 1 +_. oo. oo. 1 +. . . 1 +_. oo. oo. 1 +._ .oo .oo 1 +_. oo. oo. 1 +_. oo. oo. 1 +_. oo. oo. 1 +_._ oo.oo oo.oo 1 +_._ oo.oo oo.oo 1 +. . . 1 +_._ oo.oo oo.oo 1 +._ .oo .oo 1 +_._ oo.oo oo.oo 1 +_. oo. oo. 1 +_._ oo.oo oo.oo 1 +_._ oo.oo oo.oo 1 +. . . 1 +. . . 1 +. . . 1 +.__ .oo .oo 1 +. . . 1 +__. oo. oo. 1 +. . . 1 +__.__ oo.oo oo.oo 1 +.__ .oo .oo 1 +. . . 1 +.__ .oo .oo 1 +.__ .oo .oo 1 +.__ .oo .oo 1 +__. oo. oo. 1 +.__ .oo .oo 1 +__.__ oo.oo oo.oo 1 +__. oo. oo. 1 +. . . 1 +__. oo. oo. 1 +.__ .oo .oo 1 +__. oo. oo. 1 +__. oo. oo. 1 +__. oo. oo. 1 +__.__ oo.oo oo.oo 1 +__.__ oo.oo oo.oo 1 +. . . 1 +__.__ oo.oo oo.oo 1 +.__ .oo .oo 1 +__.__ oo.oo oo.oo 1 +__. oo. oo. 1 +__.__ oo.oo oo.oo 1 +__.__ oo.oo oo.oo 1 +. . . 1 +. . . 1 +. . . 1 +._ .o .o 1 +. . . 1 +_. o. o. 1 +. . . 1 +_._ o._ o._ 1 +._ .o .o 1 +. . . 1 +._ .o .o 1 +._ .o .o 1 +._ .o .o 1 +_. o. o. 1 +._ .o .o 1 +_._ o._ o._ 1 +_. o. o. 1 +. . . 1 +_. o. o. 1 +._ .o .o 1 +_. o. o. 1 +_. o. o. 1 +_. o. o. 1 +_._ o._ o._ 1 +_._ o._ o._ 1 +. . . 1 +_._ o._ o._ 1 +._ .o .o 1 +_._ o._ o._ 1 +_. o. o. 1 +_._ o._ o._ 1 +_._ o._ o._ 1 +. . . 1 +. . . 1 +. . . 1 +._ .oo .oo 1 +. . . 1 +_. oo. oo. 1 +. . . 1 +_._ oo._ oo._ 1 +._ .oo .oo 1 +. . . 1 +._ .oo .oo 1 +._ .oo .oo 1 +._ .oo .oo 1 +_. oo. oo. 1 +._ .oo .oo 1 +_._ oo._ oo._ 1 +_. oo. oo. 1 +. . . 1 +_. oo. oo. 1 +._ .oo .oo 1 +_. oo. oo. 1 +_. oo. oo. 1 +_. oo. oo. 1 +_._ oo._ oo._ 1 +_._ oo._ oo._ 1 +. . . 1 +_._ oo._ oo._ 1 +._ .oo .oo 1 +_._ oo._ oo._ 1 +_. oo. oo. 1 +_._ oo._ oo._ 1 +_._ oo._ oo._ 1 +. . . 1 +. . . 1 +. . . 1 +.__ .o_ .o_ 1 +. . . 1 +__. o_. o_. 1 +. . . 1 +__.__ o_.__ o_.__ 1 +.__ .o_ .o_ 1 +. . . 1 +.__ .o_ .o_ 1 +.__ .o_ .o_ 1 +.__ .o_ .o_ 1 +__. o_. o_. 1 +.__ .o_ .o_ 1 +__.__ o_.__ o_.__ 1 +__. o_. o_. 1 +. . . 1 +__. o_. o_. 1 +.__ .o_ .o_ 1 +__. o_. o_. 1 +__. o_. o_. 1 +__. o_. o_. 1 +__.__ o_.__ o_.__ 1 +__.__ o_.__ o_.__ 1 +. . . 1 +__.__ o_.__ o_.__ 1 +.__ .o_ .o_ 1 +__.__ o_.__ o_.__ 1 +__. o_. o_. 1 +__.__ o_.__ o_.__ 1 +__.__ o_.__ o_.__ 1 diff --git a/dbms/tests/queries/0_stateless/00240_replace_substring_loop.sql b/dbms/tests/queries/0_stateless/00240_replace_substring_loop.sql new file mode 100644 index 00000000000..2c9157d5946 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00240_replace_substring_loop.sql @@ -0,0 +1,101 @@ +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '._']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '_._']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '._']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '_._']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '._']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '_._']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '._']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '_._']) AS s); + +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '._']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '_._']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '._']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '_._']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '._']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '_._']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '._']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '_.']) AS s); +SELECT s, replaceAll(s, '_', 'oo') AS a, replaceRegexpAll(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '_._']) AS s); + +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '.__']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '__.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '__.__']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '.__']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '__.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '__.__']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '.__']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '__.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '__.__']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '.__']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '__.']) AS s); +SELECT s, replaceAll(s, '_', 'o') AS a, replaceRegexpAll(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '__.__']) AS s); + +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '._']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '_._']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '._']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['._', '_._']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '._']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_.', '_._']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '._']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['_._', '_._']) AS s); + +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '._']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['.', '_._']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '._']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['._', '_._']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '._']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_.', '_._']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '._']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '_.']) AS s); +SELECT s, replaceOne(s, '_', 'oo') AS a, replaceRegexpOne(s, '_', 'oo') AS b, a = b FROM (SELECT arrayJoin(['_._', '_._']) AS s); + +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '.__']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '__.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.', '__.__']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '.__']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '__.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['.__', '__.__']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '.__']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '__.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.', '__.__']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '.__']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '__.']) AS s); +SELECT s, replaceOne(s, '_', 'o') AS a, replaceRegexpOne(s, '_', 'o') AS b, a = b FROM (SELECT arrayJoin(['__.__', '__.__']) AS s); From 8ab8ab5a2bf5c06e25ebdca46becfa7f91f87b0a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Oct 2015 02:36:45 +0300 Subject: [PATCH 11/16] dbms: added 'override' to functions [#METR-18149]. --- .../DB/Functions/FunctionsArithmetic.h | 12 +-- dbms/include/DB/Functions/FunctionsArray.h | 48 +++++----- dbms/include/DB/Functions/FunctionsCoding.h | 60 ++++++------- .../DB/Functions/FunctionsComparison.h | 6 +- .../DB/Functions/FunctionsConditional.h | 6 +- .../DB/Functions/FunctionsConversion.h | 12 +-- dbms/include/DB/Functions/FunctionsDateTime.h | 34 +++---- .../DB/Functions/FunctionsDictionaries.h | 24 ++--- .../DB/Functions/FunctionsFormatting.h | 12 +-- dbms/include/DB/Functions/FunctionsHashing.h | 24 ++--- .../DB/Functions/FunctionsHigherOrder.h | 4 +- dbms/include/DB/Functions/FunctionsLogical.h | 12 +-- .../DB/Functions/FunctionsMiscellaneous.h | 88 +++++++++---------- dbms/include/DB/Functions/FunctionsRandom.h | 12 +-- .../DB/Functions/FunctionsReinterpret.h | 12 +-- dbms/include/DB/Functions/FunctionsString.h | 26 +++--- .../DB/Functions/FunctionsStringArray.h | 6 +- .../DB/Functions/FunctionsStringSearch.h | 18 ++-- 18 files changed, 208 insertions(+), 208 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsArithmetic.h b/dbms/include/DB/Functions/FunctionsArithmetic.h index d468b260e63..fafc1ce791b 100644 --- a/dbms/include/DB/Functions/FunctionsArithmetic.h +++ b/dbms/include/DB/Functions/FunctionsArithmetic.h @@ -658,13 +658,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -692,7 +692,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!( executeLeftType(block, arguments, result) || executeLeftType(block, arguments, result) @@ -767,13 +767,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -799,7 +799,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!( executeType(block, arguments, result) || executeType(block, arguments, result) diff --git a/dbms/include/DB/Functions/FunctionsArray.h b/dbms/include/DB/Functions/FunctionsArray.h index f3bc6268477..6751522cab8 100644 --- a/dbms/include/DB/Functions/FunctionsArray.h +++ b/dbms/include/DB/Functions/FunctionsArray.h @@ -56,7 +56,7 @@ public: private: /// Получить имя функции. - String getName() const + String getName() const override { return name; } @@ -159,7 +159,7 @@ private: public: /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.empty()) throw Exception("Function array requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -184,7 +184,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const auto is_const = [&] { for (const auto arg_num : arguments) @@ -702,13 +702,13 @@ private: } public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -727,7 +727,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (executeTuple(block, arguments, result)) { @@ -1050,13 +1050,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1075,7 +1075,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!(executeNumber(block, arguments, result) || executeNumber(block, arguments, result) @@ -1104,13 +1104,13 @@ public: static IFunction * create (const Context & context) { return new FunctionArrayEnumerate; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1125,7 +1125,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (const ColumnArray * array = typeid_cast(&*block.getByPosition(arguments[0]).column)) { @@ -1180,13 +1180,13 @@ public: static IFunction * create(const Context & context) { return new FunctionArrayUniq; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() == 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1205,7 +1205,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (arguments.size() == 1 && executeConst(block, arguments, result)) return; @@ -1410,13 +1410,13 @@ public: static IFunction * create(const Context & context) { return new FunctionArrayEnumerateUniq; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() == 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1435,7 +1435,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (arguments.size() == 1 && executeConst(block, arguments, result)) return; @@ -1655,12 +1655,12 @@ struct FunctionEmptyArray : public IFunction static IFunction * create(const Context & context) { return new FunctionEmptyArray; } private: - String getName() const + String getName() const override { return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1670,7 +1670,7 @@ private: return new DataTypeArray{new DataType{}}; } - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { using UnderlyingColumnType = typename TypeToColumnType::ColumnType; @@ -1833,13 +1833,13 @@ public: static IFunction * create(const Context & context) { return new FunctionEmptyArrayToSingle; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1855,7 +1855,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (executeConst(block, arguments, result)) return; diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index 4a45cc6444e..4c8304a6755 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -173,9 +173,9 @@ public: static constexpr auto name = "IPv6NumToString"; static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; } - String getName() const { return name; } + String getName() const override { return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -259,9 +259,9 @@ public: static constexpr auto name = "cutIPv6"; static IFunction * create(const Context & context) { return new FunctionCutIPv6; } - String getName() const { return name; } + String getName() const override { return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 3) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -425,9 +425,9 @@ public: static constexpr auto name = "IPv6StringToNum"; static IFunction * create(const Context & context) { return new FunctionIPv6StringToNum; } - String getName() const { return name; } + String getName() const override { return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -594,7 +594,7 @@ public: memcpy(dst, tmp, sizeof(tmp)); } - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr & column = block.getByPosition(arguments[0]).column; @@ -642,13 +642,13 @@ public: static IFunction * create(const Context & context) { return new FunctionIPv4NumToString; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -697,7 +697,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column = block.getByPosition(arguments[0]).column; @@ -747,13 +747,13 @@ public: static IFunction * create(const Context & context) { return new FunctionIPv4StringToNum; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -796,7 +796,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column = block.getByPosition(arguments[0]).column; @@ -838,13 +838,13 @@ public: static IFunction * create(const Context & context) { return new FunctionIPv4NumToStringClassC; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -896,7 +896,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column = block.getByPosition(arguments[0]).column; @@ -945,9 +945,9 @@ public: static constexpr auto name = "IPv4ToIPv6"; static IFunction * create(const Context & context) { return new FunctionIPv4ToIPv6; } - String getName() const { return name; } + String getName() const override { return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1010,13 +1010,13 @@ public: static IFunction * create(const Context & context) { return new FunctionHex; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1231,7 +1231,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const IColumn * column = &*block.getByPosition(arguments[0]).column; ColumnPtr & res_column = block.getByPosition(result).column; @@ -1258,13 +1258,13 @@ public: static IFunction * create(const Context & context) { return new FunctionUnhex; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1303,7 +1303,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column = block.getByPosition(arguments[0]).column; @@ -1366,13 +1366,13 @@ public: static IFunction * create(const Context & context) { return new FunctionBitmaskToArray; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1450,7 +1450,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const IColumn * in_column = &*block.getByPosition(arguments[0]).column; ColumnPtr & out_column = block.getByPosition(result).column; @@ -1478,13 +1478,13 @@ public: static IFunction * create(const Context & context) { return new FunctionToStringCutToZero; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1600,7 +1600,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const IColumn * column = &*block.getByPosition(arguments[0]).column; ColumnPtr & res_column = block.getByPosition(result).column; diff --git a/dbms/include/DB/Functions/FunctionsComparison.h b/dbms/include/DB/Functions/FunctionsComparison.h index a901f363342..3c5c171b046 100644 --- a/dbms/include/DB/Functions/FunctionsComparison.h +++ b/dbms/include/DB/Functions/FunctionsComparison.h @@ -622,13 +622,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -672,7 +672,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const IColumn * col_left_untyped = block.getByPosition(arguments[0]).column.get(); const IColumn * col_right_untyped = block.getByPosition(arguments[1]).column.get(); diff --git a/dbms/include/DB/Functions/FunctionsConditional.h b/dbms/include/DB/Functions/FunctionsConditional.h index 31b1c50ebd8..47f67bd7304 100644 --- a/dbms/include/DB/Functions/FunctionsConditional.h +++ b/dbms/include/DB/Functions/FunctionsConditional.h @@ -1273,13 +1273,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 3) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1346,7 +1346,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnVector * cond_col = typeid_cast *>(&*block.getByPosition(arguments[0]).column); const ColumnConst * cond_const_col = typeid_cast *>(&*block.getByPosition(arguments[0]).column); diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index 6a6b3551686..ae7ade63bff 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -896,19 +896,19 @@ public: static IFunction * create(const Context & context) { return new FunctionConvert; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { return getReturnTypeImpl(arguments); } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { IDataType * from_type = &*block.getByPosition(arguments[0]).type; @@ -1009,7 +1009,7 @@ public: static IFunction * create(const Context & context) { return new FunctionToFixedString; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } @@ -1020,7 +1020,7 @@ public: */ void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, - std::vector & out_prerequisites) + std::vector & out_prerequisites) override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1038,7 +1038,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { ColumnPtr column = block.getByPosition(arguments[0]).column; size_t n = getSize(block.getByPosition(arguments[1])); diff --git a/dbms/include/DB/Functions/FunctionsDateTime.h b/dbms/include/DB/Functions/FunctionsDateTime.h index 6b30efd1e33..9d270044810 100644 --- a/dbms/include/DB/Functions/FunctionsDateTime.h +++ b/dbms/include/DB/Functions/FunctionsDateTime.h @@ -397,7 +397,7 @@ public: static IFunction * create(const Context & context) { return new FunctionDateOrDateTimeToSomething; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } @@ -408,7 +408,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { IDataType * from_type = &*block.getByPosition(arguments[0]).type; @@ -479,13 +479,13 @@ public: static IFunction * create(const Context & context) { return new FunctionNow; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -496,7 +496,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = new ColumnConstUInt32( block.rowsInFirstColumn(), @@ -512,13 +512,13 @@ public: static IFunction * create(const Context & context) { return new FunctionToday; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -529,7 +529,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = new ColumnConstUInt16( block.rowsInFirstColumn(), @@ -545,13 +545,13 @@ public: static IFunction * create(const Context & context) { return new FunctionYesterday; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -562,7 +562,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = new ColumnConstUInt16( block.rowsInFirstColumn(), @@ -578,13 +578,13 @@ public: static IFunction * create(const Context & context) { return new FunctionTimeSlot; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -599,7 +599,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (const ColumnUInt32 * times = typeid_cast(&*block.getByPosition(arguments[0]).column)) { @@ -714,13 +714,13 @@ public: static IFunction * create(const Context & context) { return new FunctionTimeSlots; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -739,7 +739,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnUInt32 * starts = typeid_cast(&*block.getByPosition(arguments[0]).column); const ColumnConstUInt32 * const_starts = typeid_cast(&*block.getByPosition(arguments[0]).column); diff --git a/dbms/include/DB/Functions/FunctionsDictionaries.h b/dbms/include/DB/Functions/FunctionsDictionaries.h index 1c60af47803..e897a13d222 100644 --- a/dbms/include/DB/Functions/FunctionsDictionaries.h +++ b/dbms/include/DB/Functions/FunctionsDictionaries.h @@ -182,13 +182,13 @@ public: } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -209,7 +209,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { /// Ключ словаря, определяющий "точку зрения". std::string dict_key; @@ -274,13 +274,13 @@ public: } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2 && arguments.size() != 3) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -306,7 +306,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { /// Ключ словаря, определяющий "точку зрения". std::string dict_key; @@ -407,13 +407,13 @@ public: } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -434,7 +434,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { /// Ключ словаря, определяющий "точку зрения". std::string dict_key; @@ -667,13 +667,13 @@ public: } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -694,7 +694,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { RegionsNames::Language language = RegionsNames::Language::RU; diff --git a/dbms/include/DB/Functions/FunctionsFormatting.h b/dbms/include/DB/Functions/FunctionsFormatting.h index 69ffea37349..d9c421966af 100644 --- a/dbms/include/DB/Functions/FunctionsFormatting.h +++ b/dbms/include/DB/Functions/FunctionsFormatting.h @@ -25,13 +25,13 @@ public: static IFunction * create(const Context & context) { return new FunctionBitmaskToList; } /// Получить основное имя функции. - virtual String getName() const + virtual String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - virtual DataTypePtr getReturnType(const DataTypes & arguments) const + virtual DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -54,7 +54,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!( executeType(block, arguments, result) || executeType(block, arguments, result) @@ -138,13 +138,13 @@ public: static IFunction * create(const Context & context) { return new FunctionFormatReadableSize; } /// Получить основное имя функции. - virtual String getName() const + virtual String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - virtual DataTypePtr getReturnType(const DataTypes & arguments) const + virtual DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -160,7 +160,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!( executeType(block, arguments, result) || executeType(block, arguments, result) diff --git a/dbms/include/DB/Functions/FunctionsHashing.h b/dbms/include/DB/Functions/FunctionsHashing.h index 2d9e077fc0c..b1bcccceb89 100644 --- a/dbms/include/DB/Functions/FunctionsHashing.h +++ b/dbms/include/DB/Functions/FunctionsHashing.h @@ -174,13 +174,13 @@ public: static IFunction * create(const Context & context) { return new FunctionStringHash64; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -195,7 +195,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (const ColumnString * col_from = typeid_cast(&*block.getByPosition(arguments[0]).column)) { @@ -235,13 +235,13 @@ public: static IFunction * create(const Context & context) { return new FunctionStringHashFixedString; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -256,7 +256,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (const ColumnString * col_from = typeid_cast(&*block.getByPosition(arguments[0]).column)) { @@ -334,13 +334,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -355,7 +355,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { IDataType * from_type = &*block.getByPosition(arguments[0]).type; @@ -567,19 +567,19 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { return new DataTypeUInt64; } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { size_t rows = block.rowsInFirstColumn(); ColumnUInt64 * col_to = new ColumnUInt64(rows); diff --git a/dbms/include/DB/Functions/FunctionsHigherOrder.h b/dbms/include/DB/Functions/FunctionsHigherOrder.h index d62fb7340f9..272714feb76 100644 --- a/dbms/include/DB/Functions/FunctionsHigherOrder.h +++ b/dbms/include/DB/Functions/FunctionsHigherOrder.h @@ -549,7 +549,7 @@ public: static IFunction * create(const Context & context) { return new FunctionArrayMapped; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } @@ -588,7 +588,7 @@ public: void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, - ExpressionActions::Actions & out_prerequisites) + ExpressionActions::Actions & out_prerequisites) override { size_t min_args = Impl::needExpression() ? 2 : 1; if (arguments.size() < min_args) diff --git a/dbms/include/DB/Functions/FunctionsLogical.h b/dbms/include/DB/Functions/FunctionsLogical.h index 28f8e9e5254..d3704c37a60 100644 --- a/dbms/include/DB/Functions/FunctionsLogical.h +++ b/dbms/include/DB/Functions/FunctionsLogical.h @@ -242,13 +242,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() < 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -268,7 +268,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { ColumnPlainPtrs in(arguments.size()); for (size_t i = 0; i < arguments.size(); ++i) @@ -396,13 +396,13 @@ private: public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -419,7 +419,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!( executeType(block, arguments, result) || executeType(block, arguments, result) diff --git a/dbms/include/DB/Functions/FunctionsMiscellaneous.h b/dbms/include/DB/Functions/FunctionsMiscellaneous.h index db841629d5e..9367488d180 100644 --- a/dbms/include/DB/Functions/FunctionsMiscellaneous.h +++ b/dbms/include/DB/Functions/FunctionsMiscellaneous.h @@ -94,11 +94,11 @@ public: explicit FunctionCurrentDatabase(const String & db_name) : db_name{db_name} {} - String getName() const { + String getName() const override { return name; } - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -125,13 +125,13 @@ public: static IFunction * create(const Context & context) { return new FunctionHostName; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -143,7 +143,7 @@ public: /** Выполнить функцию над блоком. convertToFullColumn вызывается для того, чтобы в случае * распределенного выполнения запроса каждый сервер возвращал свое имя хоста. */ - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = ColumnConstString( block.rowsInFirstColumn(), @@ -159,13 +159,13 @@ public: static IFunction * create(const Context & context) { return new FunctionVisibleWidth; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -176,7 +176,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result); + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override; }; @@ -187,13 +187,13 @@ public: static IFunction * create(const Context & context) { return new FunctionToTypeName; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -204,7 +204,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = new ColumnConstString(block.rowsInFirstColumn(), block.getByPosition(arguments[0]).type->getName()); } @@ -218,13 +218,13 @@ public: static IFunction * create(const Context & context) { return new FunctionBlockSize; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (!arguments.empty()) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -235,7 +235,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { size_t size = block.rowsInFirstColumn(); block.getByPosition(result).column = ColumnConstUInt64(size, size).convertToFullColumn(); @@ -250,13 +250,13 @@ public: static IFunction * create(const Context & context) { return new FunctionSleep; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -276,7 +276,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { IColumn * col = &*block.getByPosition(arguments[0]).column; double seconds; @@ -319,13 +319,13 @@ public: static IFunction * create(const Context & context) { return new FunctionMaterialize; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -336,7 +336,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const IColumn & argument = *block.getByPosition(arguments[0]).column; if (!argument.isConst()) @@ -360,13 +360,13 @@ public: static IFunction * create(const Context & context) { return new FunctionIn; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function '" + getName() + "' doesn't match: passed " @@ -377,7 +377,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { /// Второй аргумент - обязательно ColumnSet. ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; @@ -407,13 +407,13 @@ public: static IFunction * create(const Context & context) { return new FunctionTuple; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() < 1) throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -422,7 +422,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { Block tuple_block; @@ -441,14 +441,14 @@ public: static IFunction * create(const Context & context) { return new FunctionTupleElement; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, - ExpressionActions::Actions & out_prerequisites) + ExpressionActions::Actions & out_prerequisites) override { if (arguments.size() != 2) throw Exception("Function " + getName() + " requires exactly two arguments: tuple and element index.", @@ -476,7 +476,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnTuple * tuple_col = typeid_cast(&*block.getByPosition(arguments[0]).column); const ColumnConstUInt8 * index_col = typeid_cast(&*block.getByPosition(arguments[1]).column); @@ -508,19 +508,19 @@ public: static IFunction * create(const Context & context) { return new FunctionIgnore; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { return new DataTypeUInt8; } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = new ColumnConstUInt8(block.rowsInFirstColumn(), 0); } @@ -534,13 +534,13 @@ public: static IFunction * create(const Context & context) { return new FunctionIdentity; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Function " + getName() + " requires exactly one argument.", @@ -550,7 +550,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { block.getByPosition(result).column = block.getByPosition(arguments.front()).column; } @@ -565,13 +565,13 @@ public: /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -584,7 +584,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL); } @@ -601,13 +601,13 @@ public: static IFunction * create(const Context & context) { return new FunctionReplicate; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -622,7 +622,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { ColumnPtr first_column = block.getByPosition(arguments[0]).column; @@ -652,13 +652,13 @@ public: static IFunction * create(const Context & context) { return new FunctionBar; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 3 && arguments.size() != 4) throw Exception("Function " + getName() + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed " @@ -673,7 +673,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { Int64 min = extractConstant(block, arguments, 1, "Second"); /// Уровень значения, при котором полоска имеет нулевую длину. Int64 max = extractConstant(block, arguments, 2, "Third"); /// Уровень значения, при котором полоска имеет максимальную длину. diff --git a/dbms/include/DB/Functions/FunctionsRandom.h b/dbms/include/DB/Functions/FunctionsRandom.h index 92e97df6694..f0d2759063d 100644 --- a/dbms/include/DB/Functions/FunctionsRandom.h +++ b/dbms/include/DB/Functions/FunctionsRandom.h @@ -153,13 +153,13 @@ public: static IFunction * create(const Context & context) { return new FunctionRandom; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() > 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -170,7 +170,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { ColumnVector * col_to = new ColumnVector; block.getByPosition(result).column = col_to; @@ -199,13 +199,13 @@ public: static IFunction * create(const Context & context) { return new FunctionRandomConstant; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() > 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -216,7 +216,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!is_initialized) { diff --git a/dbms/include/DB/Functions/FunctionsReinterpret.h b/dbms/include/DB/Functions/FunctionsReinterpret.h index 5d76a9e6036..59488e3c28b 100644 --- a/dbms/include/DB/Functions/FunctionsReinterpret.h +++ b/dbms/include/DB/Functions/FunctionsReinterpret.h @@ -27,13 +27,13 @@ public: static IFunction * create(const Context & context) { return new FunctionReinterpretAsStringImpl; }; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -97,7 +97,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (!( executeType(block, arguments, result) || executeType(block, arguments, result) @@ -125,13 +125,13 @@ public: typedef typename ToDataType::FieldType ToFieldType; /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -147,7 +147,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { if (ColumnString * col_from = typeid_cast(&*block.getByPosition(arguments[0]).column)) { diff --git a/dbms/include/DB/Functions/FunctionsString.h b/dbms/include/DB/Functions/FunctionsString.h index 498221cfb8c..a807abc7301 100644 --- a/dbms/include/DB/Functions/FunctionsString.h +++ b/dbms/include/DB/Functions/FunctionsString.h @@ -770,13 +770,13 @@ public: static IFunction * create(const Context & context) { return new FunctionStringOrArrayToT; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -792,7 +792,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column = block.getByPosition(arguments[0]).column; if (const ColumnString * col = typeid_cast(&*column)) @@ -866,13 +866,13 @@ public: static IFunction * create(const Context & context) { return new FunctionStringToString; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -887,7 +887,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column = block.getByPosition(arguments[0]).column; if (const ColumnString * col = typeid_cast(&*column)) @@ -926,13 +926,13 @@ public: static IFunction * create(const Context & context) { return new FunctionConcat; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() < 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1397,13 +1397,13 @@ public: static IFunction * create(const Context & context) { return new FunctionStringNumNumToString; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 3) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1422,7 +1422,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column_string = block.getByPosition(arguments[0]).column; const ColumnPtr column_start = block.getByPosition(arguments[1]).column; @@ -1477,13 +1477,13 @@ public: static constexpr auto name = "appendTrailingCharIfAbsent"; static IFunction * create(const Context & context) { return new FunctionAppendTrailingCharIfAbsent; } - String getName() const + String getName() const override { return name; } private: - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception{ diff --git a/dbms/include/DB/Functions/FunctionsStringArray.h b/dbms/include/DB/Functions/FunctionsStringArray.h index 80f6a77b4c7..b9fbceceb37 100644 --- a/dbms/include/DB/Functions/FunctionsStringArray.h +++ b/dbms/include/DB/Functions/FunctionsStringArray.h @@ -316,13 +316,13 @@ public: static IFunction * create(const Context & context) { return new FunctionTokens; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { Generator::checkArguments(arguments); @@ -330,7 +330,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { Generator generator; generator.init(block, arguments); diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index 4da88809724..8f55cb3ebb5 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -1505,13 +1505,13 @@ public: static IFunction * create(const Context & context) { return new FunctionStringReplace; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 3) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1534,7 +1534,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column_src = block.getByPosition(arguments[0]).column; const ColumnPtr column_needle = block.getByPosition(arguments[1]).column; @@ -1592,13 +1592,13 @@ public: static IFunction * create(const Context & context) { return new FunctionsStringSearch; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1617,7 +1617,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { typedef typename Impl::ResultType ResultType; @@ -1661,13 +1661,13 @@ public: static IFunction * create(const Context & context) { return new FunctionsStringSearchToString; } /// Получить имя функции. - String getName() const + String getName() const override { return name; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. - DataTypePtr getReturnType(const DataTypes & arguments) const + DataTypePtr getReturnType(const DataTypes & arguments) const override { if (arguments.size() != 2) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " @@ -1686,7 +1686,7 @@ public: } /// Выполнить функцию над блоком. - void execute(Block & block, const ColumnNumbers & arguments, size_t result) + void execute(Block & block, const ColumnNumbers & arguments, size_t result) override { const ColumnPtr column = block.getByPosition(arguments[0]).column; const ColumnPtr column_needle = block.getByPosition(arguments[1]).column; From 4201a685b51e65f55dafa5ae3a67a7cda4c9c4c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Oct 2015 04:42:47 +0300 Subject: [PATCH 12/16] dbms: added support of tuple arguments to comparison functions [#METR-18149]. --- .../DB/Functions/FunctionsComparison.h | 206 ++++++++++++++++-- .../DB/Functions/FunctionsHigherOrder.h | 2 +- .../00250_tuple_comparison.reference | 15 ++ .../0_stateless/00250_tuple_comparison.sql | 105 +++++++++ 4 files changed, 312 insertions(+), 16 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00250_tuple_comparison.reference create mode 100644 dbms/tests/queries/0_stateless/00250_tuple_comparison.sql diff --git a/dbms/include/DB/Functions/FunctionsComparison.h b/dbms/include/DB/Functions/FunctionsComparison.h index 3c5c171b046..34f33ea75fc 100644 --- a/dbms/include/DB/Functions/FunctionsComparison.h +++ b/dbms/include/DB/Functions/FunctionsComparison.h @@ -10,7 +10,9 @@ #include #include #include +#include +#include #include @@ -384,6 +386,14 @@ template struct StringComparisonImpl> : StringEqualsImpl {}; +struct NameEquals { static constexpr auto name = "equals"; }; +struct NameNotEquals { static constexpr auto name = "notEquals"; }; +struct NameLess { static constexpr auto name = "less"; }; +struct NameGreater { static constexpr auto name = "greater"; }; +struct NameLessOrEquals { static constexpr auto name = "lessOrEquals"; }; +struct NameGreaterOrEquals { static constexpr auto name = "greaterOrEquals"; }; + + template < template class Op, typename Name> @@ -620,6 +630,108 @@ private: } } + void executeTuple(Block & block, size_t result, const IColumn * c0, const IColumn * c1) + { + /** Сравнивать кортежи будем лексикографически. Это делается следующим образом: + * x == y : x1 == y1 && x2 == y2 ... + * x != y : x1 != y1 || x2 != y2 ... + * + * x < y: x1 < y1 || (x1 == y1 && (x2 < y2 || (x2 == y2 ... && xn < yn)) + * x > y: x1 > y1 || (x1 == y1 && (x2 > y2 || (x2 == y2 ... && xn > yn)) + * x <= y: x1 < y1 || (x1 == y1 && (x2 < y2 || (x2 == y2 ... && xn <= yn)) + * + * Рекурсивная запись: + * x <= y: x1 < y1 || (x1 == y1 && x_tail <= y_tail) + * + * x >= y: x1 > y1 || (x1 == y1 && (x2 > y2 || (x2 == y2 ... && xn >= yn)) + */ + + const size_t tuple_size = x->getData().columns(); + + if (0 == tuple_size) + throw Exception("Comparison of zero-sized tuples is not implemented.", ErrorCodes::NOT_IMPLEMENTED); + + auto x = static_cast(c0); + auto y = static_cast(c1); + executeTupleImpl(block, result, x, y, tuple_size); + } + + void executeTupleImpl(Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size); + + template + void executeTupleEqualityImpl(Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) + { + ComparisonFunction func_compare; + ConvolutionFunction func_convolution; + + Block tmp_block; + for (size_t i = 0; i < tuple_size; ++i) + { + tmp_block.insert(x->getData().getByPosition(i)); + tmp_block.insert(y->getData().getByPosition(i)); + + /// Сравнение элементов. + tmp_block.insert({ nullptr, new DataTypeUInt8, "" }); + func_compare.execute(tmp_block, {i * 3, i * 3 + 1}, i * 3 + 2); + } + + /// Логическая свёртка. + tmp_block.insert({ nullptr, new DataTypeUInt8, "" }); + + ColumnNumbers convolution_args(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + convolution_args[i] = i * 3 + 2; + + func_convolution.execute(tmp_block, convolution_args, tuple_size * 3); + block.getByPosition(result).column = tmp_block.getByPosition(tuple_size * 3).column; + } + + template + void executeTupleLessGreaterImpl(Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) + { + HeadComparisonFunction func_compare_head; + TailComparisonFunction func_compare_tail; + FunctionAnd func_and; + FunctionOr func_or; + FunctionComparison func_equals; + + Block tmp_block; + + /// Попарное сравнение на неравенство всех элементов; на равенство всех элементов кроме последнего. + for (size_t i = 0; i < tuple_size; ++i) + { + tmp_block.insert(x->getData().getByPosition(i)); + tmp_block.insert(y->getData().getByPosition(i)); + + tmp_block.insert({ nullptr, new DataTypeUInt8, "" }); + + if (i + 1 != tuple_size) + { + func_compare_head.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2); + + tmp_block.insert({ nullptr, new DataTypeUInt8, "" }); + func_equals.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 3); + + } + else + func_compare_tail.execute(tmp_block, {i * 4, i * 4 + 1}, i * 4 + 2); + } + + /// Комбинирование. Сложный код - сделайте рисунок. Можно заменить на рекурсивное сравнение кортежей. + size_t i = tuple_size - 1; + while (i > 0) + { + tmp_block.insert({ nullptr, new DataTypeUInt8, "" }); + func_and.execute(tmp_block, { tmp_block.columns() - 2, (i - 1) * 4 + 3 }, tmp_block.columns() - 1); + tmp_block.insert({ nullptr, new DataTypeUInt8, "" }); + func_or.execute(tmp_block, { tmp_block.columns() - 2, (i - 1) * 4 + 2 }, tmp_block.columns() - 1); + --i; + } + + block.getByPosition(result).column = tmp_block.getByPosition(tmp_block.columns() - 1).column; + } + + public: /// Получить имя функции. String getName() const override @@ -639,23 +751,27 @@ public: bool left_is_date_time = false; bool left_is_string = false; bool left_is_fixed_string = false; + const DataTypeTuple * left_tuple = nullptr; false || (left_is_date = typeid_cast(arguments[0].get())) || (left_is_date_time = typeid_cast(arguments[0].get())) || (left_is_string = typeid_cast(arguments[0].get())) - || (left_is_fixed_string = typeid_cast(arguments[0].get())); + || (left_is_fixed_string = typeid_cast(arguments[0].get())) + || (left_tuple = typeid_cast(arguments[0].get())); bool right_is_date = false; bool right_is_date_time = false; bool right_is_string = false; bool right_is_fixed_string = false; + const DataTypeTuple * right_tuple = nullptr; false || (right_is_date = typeid_cast(arguments[1].get())) || (right_is_date_time = typeid_cast(arguments[1].get())) || (right_is_string = typeid_cast(arguments[1].get())) - || (right_is_fixed_string = typeid_cast(arguments[1].get())); + || (right_is_fixed_string = typeid_cast(arguments[1].get())) + || (right_tuple = typeid_cast(arguments[1].get())); if (!( (arguments[0]->behavesAsNumber() && arguments[1]->behavesAsNumber()) || ((left_is_string || left_is_fixed_string) && (right_is_string || right_is_fixed_string)) @@ -664,10 +780,18 @@ public: || (left_is_string && right_is_date) || (left_is_date_time && right_is_date_time) || (left_is_date_time && right_is_string) - || (left_is_string && right_is_date_time))) + || (left_is_string && right_is_date_time) + || (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size()))) throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")" " of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (left_tuple && right_tuple) + { + size_t size = left_tuple->getElements().size(); + for (size_t i = 0; i < size; ++i) + getReturnType({ left_tuple->getElements()[i], right_tuple->getElements()[i] }); + } + return new DataTypeUInt8; } @@ -696,25 +820,25 @@ public: + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } - else if (!left_is_num && !right_is_num) - { - executeString(block, result, col_left_untyped, col_right_untyped); - } else { - executeDateOrDateTimeWithConstString(block, result, col_left_untyped, col_right_untyped, left_is_num, right_is_num); + if (typeid_cast(col_left_untyped)) + { + executeTuple(block, result, col_left_untyped, col_right_untyped); + } + else if (!left_is_num && !right_is_num) + { + executeString(block, result, col_left_untyped, col_right_untyped); + } + else + { + executeDateOrDateTimeWithConstString(block, result, col_left_untyped, col_right_untyped, left_is_num, right_is_num); + } } } }; -struct NameEquals { static constexpr auto name = "equals"; }; -struct NameNotEquals { static constexpr auto name = "notEquals"; }; -struct NameLess { static constexpr auto name = "less"; }; -struct NameGreater { static constexpr auto name = "greater"; }; -struct NameLessOrEquals { static constexpr auto name = "lessOrEquals"; }; -struct NameGreaterOrEquals { static constexpr auto name = "greaterOrEquals"; }; - typedef FunctionComparison FunctionEquals; typedef FunctionComparison FunctionNotEquals; typedef FunctionComparison FunctionLess; @@ -722,4 +846,56 @@ typedef FunctionComparison FunctionGreater; typedef FunctionComparison FunctionLessOrEquals; typedef FunctionComparison FunctionGreaterOrEquals; + +template <> +void FunctionComparison::executeTupleImpl( + Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) +{ + return executeTupleEqualityImpl, FunctionAnd>(block, result, x, y, tuple_size); +} + +template <> +void FunctionComparison::executeTupleImpl( + Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) +{ + return executeTupleEqualityImpl, FunctionOr>(block, result, x, y, tuple_size); +} + +template <> +void FunctionComparison::executeTupleImpl( + Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) +{ + return executeTupleLessGreaterImpl< + FunctionComparison, + FunctionComparison>(block, result, x, y, tuple_size); +} + +template <> +void FunctionComparison::executeTupleImpl( + Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) +{ + return executeTupleLessGreaterImpl< + FunctionComparison, + FunctionComparison>(block, result, x, y, tuple_size); +} + +template <> +void FunctionComparison::executeTupleImpl( + Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) +{ + return executeTupleLessGreaterImpl< + FunctionComparison, + FunctionComparison>(block, result, x, y, tuple_size); +} + +template <> +void FunctionComparison::executeTupleImpl( + Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size) +{ + return executeTupleLessGreaterImpl< + FunctionComparison, + FunctionComparison>(block, result, x, y, tuple_size); +} + + } diff --git a/dbms/include/DB/Functions/FunctionsHigherOrder.h b/dbms/include/DB/Functions/FunctionsHigherOrder.h index 272714feb76..3a1857caef4 100644 --- a/dbms/include/DB/Functions/FunctionsHigherOrder.h +++ b/dbms/include/DB/Functions/FunctionsHigherOrder.h @@ -8,7 +8,7 @@ #include #include -#include "FunctionsMiscellaneous.h" +#include namespace DB diff --git a/dbms/tests/queries/0_stateless/00250_tuple_comparison.reference b/dbms/tests/queries/0_stateless/00250_tuple_comparison.reference new file mode 100644 index 00000000000..05547aaf6a4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00250_tuple_comparison.reference @@ -0,0 +1,15 @@ +1 0 0 0 1 1 +0 1 1 0 1 0 +0 1 1 0 1 0 +0 1 1 0 1 0 +0 1 0 1 0 1 +0 1 0 1 0 1 +0 1 0 1 0 1 +1 0 0 0 1 1 +0 1 1 0 1 0 +0 1 1 0 1 0 +0 1 0 1 0 1 +0 1 0 1 0 1 +1 0 0 0 1 1 +0 1 1 0 1 0 +0 1 0 1 0 1 diff --git a/dbms/tests/queries/0_stateless/00250_tuple_comparison.sql b/dbms/tests/queries/0_stateless/00250_tuple_comparison.sql new file mode 100644 index 00000000000..8f5763e2ebb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00250_tuple_comparison.sql @@ -0,0 +1,105 @@ +SELECT + (1, 'Hello', 23) = (1, 'Hello', 23), + (1, 'Hello', 23) != (1, 'Hello', 23), + (1, 'Hello', 23) < (1, 'Hello', 23), + (1, 'Hello', 23) > (1, 'Hello', 23), + (1, 'Hello', 23) <= (1, 'Hello', 23), + (1, 'Hello', 23) >= (1, 'Hello', 23); +SELECT + (1, 'Hello', 23) = (2, 'Hello', 23), + (1, 'Hello', 23) != (2, 'Hello', 23), + (1, 'Hello', 23) < (2, 'Hello', 23), + (1, 'Hello', 23) > (2, 'Hello', 23), + (1, 'Hello', 23) <= (2, 'Hello', 23), + (1, 'Hello', 23) >= (2, 'Hello', 23); +SELECT + (1, 'Hello', 23) = (1, 'World', 23), + (1, 'Hello', 23) != (1, 'World', 23), + (1, 'Hello', 23) < (1, 'World', 23), + (1, 'Hello', 23) > (1, 'World', 23), + (1, 'Hello', 23) <= (1, 'World', 23), + (1, 'Hello', 23) >= (1, 'World', 23); +SELECT + (1, 'Hello', 23) = (1, 'Hello', 24), + (1, 'Hello', 23) != (1, 'Hello', 24), + (1, 'Hello', 23) < (1, 'Hello', 24), + (1, 'Hello', 23) > (1, 'Hello', 24), + (1, 'Hello', 23) <= (1, 'Hello', 24), + (1, 'Hello', 23) >= (1, 'Hello', 24); +SELECT + (2, 'Hello', 23) = (1, 'Hello', 23), + (2, 'Hello', 23) != (1, 'Hello', 23), + (2, 'Hello', 23) < (1, 'Hello', 23), + (2, 'Hello', 23) > (1, 'Hello', 23), + (2, 'Hello', 23) <= (1, 'Hello', 23), + (2, 'Hello', 23) >= (1, 'Hello', 23); +SELECT + (1, 'World', 23) = (1, 'Hello', 23), + (1, 'World', 23) != (1, 'Hello', 23), + (1, 'World', 23) < (1, 'Hello', 23), + (1, 'World', 23) > (1, 'Hello', 23), + (1, 'World', 23) <= (1, 'Hello', 23), + (1, 'World', 23) >= (1, 'Hello', 23); +SELECT + (1, 'Hello', 24) = (1, 'Hello', 23), + (1, 'Hello', 24) != (1, 'Hello', 23), + (1, 'Hello', 24) < (1, 'Hello', 23), + (1, 'Hello', 24) > (1, 'Hello', 23), + (1, 'Hello', 24) <= (1, 'Hello', 23), + (1, 'Hello', 24) >= (1, 'Hello', 23); +SELECT + (1, 'Hello') = (1, 'Hello'), + (1, 'Hello') != (1, 'Hello'), + (1, 'Hello') < (1, 'Hello'), + (1, 'Hello') > (1, 'Hello'), + (1, 'Hello') <= (1, 'Hello'), + (1, 'Hello') >= (1, 'Hello'); +SELECT + (1, 'Hello') = (2, 'Hello'), + (1, 'Hello') != (2, 'Hello'), + (1, 'Hello') < (2, 'Hello'), + (1, 'Hello') > (2, 'Hello'), + (1, 'Hello') <= (2, 'Hello'), + (1, 'Hello') >= (2, 'Hello'); +SELECT + (1, 'Hello') = (1, 'World'), + (1, 'Hello') != (1, 'World'), + (1, 'Hello') < (1, 'World'), + (1, 'Hello') > (1, 'World'), + (1, 'Hello') <= (1, 'World'), + (1, 'Hello') >= (1, 'World'); +SELECT + (2, 'Hello') = (1, 'Hello'), + (2, 'Hello') != (1, 'Hello'), + (2, 'Hello') < (1, 'Hello'), + (2, 'Hello') > (1, 'Hello'), + (2, 'Hello') <= (1, 'Hello'), + (2, 'Hello') >= (1, 'Hello'); +SELECT + (1, 'World') = (1, 'Hello'), + (1, 'World') != (1, 'Hello'), + (1, 'World') < (1, 'Hello'), + (1, 'World') > (1, 'Hello'), + (1, 'World') <= (1, 'Hello'), + (1, 'World') >= (1, 'Hello'); +SELECT + tuple(1) = tuple(1), + tuple(1) != tuple(1), + tuple(1) < tuple(1), + tuple(1) > tuple(1), + tuple(1) <= tuple(1), + tuple(1) >= tuple(1); +SELECT + tuple(1) = tuple(2), + tuple(1) != tuple(2), + tuple(1) < tuple(2), + tuple(1) > tuple(2), + tuple(1) <= tuple(2), + tuple(1) >= tuple(2); +SELECT + tuple(2) = tuple(1), + tuple(2) != tuple(1), + tuple(2) < tuple(1), + tuple(2) > tuple(1), + tuple(2) <= tuple(1), + tuple(2) >= tuple(1); From cbaaa0a4d72e3801d5039c2742100fb7f9d043cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Oct 2015 04:43:58 +0300 Subject: [PATCH 13/16] dbms: addition to prev. revision [#METR-18149]. --- dbms/include/DB/Functions/FunctionsComparison.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsComparison.h b/dbms/include/DB/Functions/FunctionsComparison.h index 34f33ea75fc..a4678722614 100644 --- a/dbms/include/DB/Functions/FunctionsComparison.h +++ b/dbms/include/DB/Functions/FunctionsComparison.h @@ -646,13 +646,13 @@ private: * x >= y: x1 > y1 || (x1 == y1 && (x2 > y2 || (x2 == y2 ... && xn >= yn)) */ + auto x = static_cast(c0); + auto y = static_cast(c1); const size_t tuple_size = x->getData().columns(); if (0 == tuple_size) throw Exception("Comparison of zero-sized tuples is not implemented.", ErrorCodes::NOT_IMPLEMENTED); - auto x = static_cast(c0); - auto y = static_cast(c1); executeTupleImpl(block, result, x, y, tuple_size); } From d12532e7b1447f8f0035bf7fd7e6db012d5e1fb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Oct 2015 07:27:02 +0300 Subject: [PATCH 14/16] dbms: fixed error [#METR-18382]. --- dbms/src/Interpreters/Set.cpp | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index b87ff47b359..00e0cd2cbf6 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -239,26 +239,17 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set) * Если не попадает - возвращается Field(Null). */ -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wsign-compare" - template static Field convertNumericTypeImpl(const Field & from) { From value = from.get(); - if (value != To(value) - || (std::is_unsigned::value && std::is_unsigned::value && value > std::numeric_limits::max()) - || (std::is_signed::value && std::is_signed::value && (value < std::numeric_limits::min() || value > std::numeric_limits::max())) - || (std::is_signed::value && std::is_unsigned::value && (value < 0 || static_cast(value) > std::numeric_limits::max())) - || (std::is_unsigned::value && std::is_signed::value && value > static_cast(std::numeric_limits::max()))) + if (static_cast(value) != static_cast(To(value))) return {}; return Field(typename NearestFieldType::Type(value)); } -#pragma GCC diagnostic pop - template static Field convertNumericType(const Field & from, const IDataType & type) { @@ -270,7 +261,7 @@ static Field convertNumericType(const Field & from, const IDataType & type) return convertNumericTypeImpl(from); throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(from.getType()) + " at right"); + + Field::Types::toString(from.getType()) + " at right", ErrorCodes::TYPE_MISMATCH); } @@ -329,7 +320,7 @@ static Field convertToType(const Field & src, const IDataType & type) } throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " at right"); + + Field::Types::toString(src.getType()) + " at right", ErrorCodes::TYPE_MISMATCH); } else { @@ -343,7 +334,7 @@ static Field convertToType(const Field & src, const IDataType & type) || (src.getType() == Field::Types::Array && !typeid_cast(&type))) throw Exception("Type mismatch in IN section: " + type.getName() + " at left, " - + Field::Types::toString(src.getType()) + " at right"); + + Field::Types::toString(src.getType()) + " at right", ErrorCodes::TYPE_MISMATCH); } return src; From bb283e66025ca516999085b70ba6755920f16691 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Oct 2015 10:05:54 +0300 Subject: [PATCH 15/16] dbms: added support for different numeric types in function has [#METR-18382]. --- .../AggregateFunctionQuantile.h | 2 + .../AggregateFunctionQuantileDeterministic.h | 2 + .../AggregateFunctionQuantileTiming.h | 2 + .../AggregateFunctionUniqUpTo.h | 1 + .../DB/Columns/ColumnAggregateFunction.h | 3 + dbms/include/DB/Columns/ColumnVector.h | 3 + dbms/include/DB/Common/CounterInFile.h | 2 - dbms/include/DB/Core/Field.h | 416 +----------------- dbms/include/DB/Core/FieldVisitors.h | 344 +++++++++++++++ .../SummingSortedBlockInputStream.h | 1 + .../DB/DataTypes/DataTypeAggregateFunction.h | 24 +- .../DB/DataTypes/DataTypeFixedString.h | 5 +- dbms/include/DB/DataTypes/FieldToDataType.h | 2 + dbms/include/DB/DataTypes/IDataType.h | 2 - .../DB/Dictionaries/DictionaryStructure.h | 2 + dbms/include/DB/Functions/FunctionsArray.h | 45 +- .../DB/Functions/FunctionsMiscellaneous.h | 1 + dbms/include/DB/Functions/FunctionsString.h | 2 + .../include/DB/Functions/FunctionsTransform.h | 1 + dbms/include/DB/Interpreters/Set.h | 18 +- dbms/include/DB/Interpreters/SettingsCommon.h | 2 + dbms/include/DB/Parsers/ASTLiteral.h | 1 + dbms/include/DB/Parsers/ASTSetQuery.h | 1 + .../DB/Storages/MergeTree/PKCondition.h | 1 + .../AggregateFunctionFactory.cpp | 2 + dbms/src/Core/Field.cpp | 141 ++++++ dbms/src/Core/FieldVisitors.cpp | 88 ++++ dbms/src/Core/NamesAndTypes.cpp | 4 + dbms/src/Core/tests/field.cpp | 7 +- .../CollapsingSortedBlockInputStream.cpp | 1 + .../DataTypes/DataTypeAggregateFunction.cpp | 26 ++ dbms/src/DataTypes/DataTypeFixedString.cpp | 9 + dbms/src/DataTypes/FieldToDataType.cpp | 2 + dbms/src/Interpreters/Set.cpp | 20 + dbms/src/Parsers/ASTLiteral.cpp | 1 + dbms/src/Parsers/ASTSelectQuery.cpp | 1 + dbms/src/Storages/ColumnsDescription.cpp | 4 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 + dbms/src/Storages/MergeTree/PKCondition.cpp | 124 ------ dbms/src/Storages/StorageFactory.cpp | 2 + dbms/src/Storages/StorageMergeTree.cpp | 1 + .../Storages/StorageReplicatedMergeTree.cpp | 1 + 42 files changed, 729 insertions(+), 589 deletions(-) create mode 100644 dbms/include/DB/Core/FieldVisitors.h create mode 100644 dbms/src/Core/Field.cpp create mode 100644 dbms/src/Core/FieldVisitors.cpp diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h index c7bbcbe6abf..1171de2b119 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantile.h @@ -2,6 +2,8 @@ #include +#include + #include #include diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileDeterministic.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileDeterministic.h index a5d12dccc13..bf3236cc09d 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileDeterministic.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileDeterministic.h @@ -2,6 +2,8 @@ #include +#include + #include #include diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h index e61997c5d82..ea90765759a 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h @@ -4,6 +4,8 @@ #include +#include + #include #include diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h index df3911a355b..f1f2bd0c9bc 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include diff --git a/dbms/include/DB/Columns/ColumnAggregateFunction.h b/dbms/include/DB/Columns/ColumnAggregateFunction.h index e9e0522a115..eff61c1f145 100644 --- a/dbms/include/DB/Columns/ColumnAggregateFunction.h +++ b/dbms/include/DB/Columns/ColumnAggregateFunction.h @@ -9,6 +9,9 @@ #include #include +#include +#include + namespace DB { diff --git a/dbms/include/DB/Columns/ColumnVector.h b/dbms/include/DB/Columns/ColumnVector.h index c7b85aff703..8df53faad98 100644 --- a/dbms/include/DB/Columns/ColumnVector.h +++ b/dbms/include/DB/Columns/ColumnVector.h @@ -6,6 +6,9 @@ #include #include +#include +#include + #include diff --git a/dbms/include/DB/Common/CounterInFile.h b/dbms/include/DB/Common/CounterInFile.h index 23928c31386..f53caf68618 100644 --- a/dbms/include/DB/Common/CounterInFile.h +++ b/dbms/include/DB/Common/CounterInFile.h @@ -11,8 +11,6 @@ #include #include -#include - #include #include #include diff --git a/dbms/include/DB/Core/Field.h b/dbms/include/DB/Core/Field.h index a5cab3221fe..5bc1cfc8763 100644 --- a/dbms/include/DB/Core/Field.h +++ b/dbms/include/DB/Core/Field.h @@ -6,18 +6,9 @@ #include -#include -#include -#include - -#include #include #include -#include -#include -#include - -#include +#include namespace DB @@ -342,6 +333,8 @@ private: } }; +#undef DBMS_TOTAL_FIELD_SIZE + template <> struct Field::TypeToEnum { static const Types::Which value = Types::Null; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::UInt64; }; @@ -383,280 +376,9 @@ T safeGet(Field & field) } -/** StaticVisitor (его наследники) - класс с перегруженными для разных типов операторами (). - * Вызвать visitor для field можно с помощью функции apply_visitor. - * Также поддерживается visitor, в котором оператор () принимает два аргумента. - */ -template -struct StaticVisitor -{ - typedef R ResultType; -}; - - -template -typename Visitor::ResultType apply_visitor_impl(Visitor & visitor, F & field) -{ - switch (field.getType()) - { - case Field::Types::Null: return visitor(field.template get()); - case Field::Types::UInt64: return visitor(field.template get()); - case Field::Types::Int64: return visitor(field.template get()); - case Field::Types::Float64: return visitor(field.template get()); - case Field::Types::String: return visitor(field.template get()); - case Field::Types::Array: return visitor(field.template get()); - - default: - throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); - } -} - -/** Эти штуки нужны, чтобы принимать временный объект по константной ссылке. - * В шаблон выше, типы форвардятся уже с const-ом. - */ - -template -typename Visitor::ResultType apply_visitor(const Visitor & visitor, Field & field) -{ - return apply_visitor_impl(visitor, field); -} - -template -typename Visitor::ResultType apply_visitor(const Visitor & visitor, const Field & field) -{ - return apply_visitor_impl(visitor, field); -} - -template -typename Visitor::ResultType apply_visitor(Visitor & visitor, Field & field) -{ - return apply_visitor_impl(visitor, field); -} - -template -typename Visitor::ResultType apply_visitor(Visitor & visitor, const Field & field) -{ - return apply_visitor_impl(visitor, field); -} - - -template -typename Visitor::ResultType apply_binary_visitor_impl2(Visitor & visitor, F1 & field1, F2 & field2) -{ - switch (field2.getType()) - { - case Field::Types::Null: return visitor(field1, field2.template get()); - case Field::Types::UInt64: return visitor(field1, field2.template get()); - case Field::Types::Int64: return visitor(field1, field2.template get()); - case Field::Types::Float64: return visitor(field1, field2.template get()); - case Field::Types::String: return visitor(field1, field2.template get()); - case Field::Types::Array: return visitor(field1, field2.template get()); - - default: - throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); - } -} - -template -typename Visitor::ResultType apply_binary_visitor_impl1(Visitor & visitor, F1 & field1, F2 & field2) -{ - switch (field1.getType()) - { - case Field::Types::Null: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); - case Field::Types::UInt64: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); - case Field::Types::Int64: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); - case Field::Types::Float64: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); - case Field::Types::String: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); - case Field::Types::Array: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); - - default: - throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); - } -} - -template -typename Visitor::ResultType apply_visitor(Visitor & visitor, Field & field1, Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - -template -typename Visitor::ResultType apply_visitor(Visitor & visitor, Field & field1, const Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - -template -typename Visitor::ResultType apply_visitor(Visitor & visitor, const Field & field1, Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - -template -typename Visitor::ResultType apply_visitor(Visitor & visitor, const Field & field1, const Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - -template -typename Visitor::ResultType apply_visitor(const Visitor & visitor, Field & field1, Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - -template -typename Visitor::ResultType apply_visitor(const Visitor & visitor, Field & field1, const Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - -template -typename Visitor::ResultType apply_visitor(const Visitor & visitor, const Field & field1, Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - -template -typename Visitor::ResultType apply_visitor(const Visitor & visitor, const Field & field1, const Field & field2) -{ - return apply_binary_visitor_impl1(visitor, field1, field2); -} - - template <> struct TypeName { static std::string get() { return "Array"; } }; -/** Возвращает строковый дамп типа */ -class FieldVisitorDump : public StaticVisitor -{ -private: - template - static inline String formatQuotedWithPrefix(T x, const char * prefix) - { - String res; - WriteBufferFromString wb(res); - wb.write(prefix, strlen(prefix)); - writeQuoted(x, wb); - return res; - } -public: - String operator() (const Null & x) const { return "NULL"; } - String operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); } - String operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); } - String operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); } - - String operator() (const String & x) const - { - String res; - WriteBufferFromString wb(res); - writeQuoted(x, wb); - return res; - } - - String operator() (const Array & x) const - { - String res; - WriteBufferFromString wb(res); - FieldVisitorDump visitor; - - wb.write("Array_[", 7); - for (Array::const_iterator it = x.begin(); it != x.end(); ++it) - { - if (it != x.begin()) - wb.write(", ", 2); - writeString(apply_visitor(visitor, *it), wb); - } - writeChar(']', wb); - - return res; - } -}; - -/** Выводит текстовое представление типа, как литерала в SQL запросе */ -class FieldVisitorToString : public StaticVisitor -{ -private: - template - static inline String formatQuoted(T x) - { - String res; - WriteBufferFromString wb(res); - writeQuoted(x, wb); - return res; - } - - /** В отличие от writeFloatText (и writeQuoted), если число после форматирования выглядит целым, всё равно добавляет десятичную точку. - * - для того, чтобы это число могло обратно распарситься как Float64 парсером запроса (иначе распарсится как целое). - * - * При этом, не оставляет завершающие нули справа. - * - * NOTE: При таком roundtrip-е, точность может теряться. - */ - static String formatFloat(const Float64 x) - { - char tmp[25]; - double_conversion::StringBuilder builder{tmp, sizeof(tmp)}; - - const auto result = getDoubleToStringConverter().ToShortest(x, &builder); - - if (!result) - throw Exception("Cannot print float or double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER); - - return { tmp, tmp + builder.position() }; - } - -public: - String operator() (const Null & x) const { return "NULL"; } - String operator() (const UInt64 & x) const { return formatQuoted(x); } - String operator() (const Int64 & x) const { return formatQuoted(x); } - String operator() (const Float64 & x) const { return formatFloat(x); } - String operator() (const String & x) const { return formatQuoted(x); } - - String operator() (const Array & x) const - { - String res; - WriteBufferFromString wb(res); - FieldVisitorToString visitor; - - writeChar('[', wb); - for (Array::const_iterator it = x.begin(); it != x.end(); ++it) - { - if (it != x.begin()) - wb.write(", ", 2); - writeString(apply_visitor(visitor, *it), wb); - } - writeChar(']', wb); - - return res; - } -}; - -/** Числовой тип преобразует в указанный. */ -template -class FieldVisitorConvertToNumber : public StaticVisitor -{ -public: - T operator() (const Null & x) const - { - throw Exception("Cannot convert NULL to " + TypeName::get(), ErrorCodes::CANNOT_CONVERT_TYPE); - } - - T operator() (const String & x) const - { - throw Exception("Cannot convert String to " + TypeName::get(), ErrorCodes::CANNOT_CONVERT_TYPE); - } - - T operator() (const Array & x) const - { - throw Exception("Cannot convert Array to " + TypeName::get(), ErrorCodes::CANNOT_CONVERT_TYPE); - } - - T operator() (const UInt64 & x) const { return x; } - T operator() (const Int64 & x) const { return x; } - T operator() (const Float64 & x) const { return x; } -}; - - template struct NearestFieldType; template <> struct NearestFieldType { typedef UInt64 Type; }; @@ -684,27 +406,14 @@ typename NearestFieldType::Type nearestFieldType(const T & x) /// Заглушки, чтобы DBObject-ы с полем типа Array компилировались. +#include + namespace mysqlxx { - inline std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Array & value) - { - return res.ostr << apply_visitor(DB::FieldVisitorToString(), value); - } - - inline std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Array & value) - { - throw Poco::Exception("Cannot quote Array with mysqlxx::quote."); - } - - inline std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Array & value) - { - throw Poco::Exception("Cannot unescape Array with mysqlxx::unescape."); - } - - inline std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Array & value) - { - throw Poco::Exception("Cannot unquote Array with mysqlxx::unquote."); - } + std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Array & value); + std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Array & value); + std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Array & value); + std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Array & value); } @@ -714,116 +423,15 @@ namespace DB class WriteBuffer; /// Предполагается что у всех элементов массива одинаковый тип. - inline void readBinary(Array & x, ReadBuffer & buf) - { - size_t size; - UInt8 type; - DB::readBinary(type, buf); - DB::readBinary(size, buf); - - for (size_t index = 0; index < size; ++index) - { - switch (type) - { - case Field::Types::Null: - { - x.push_back(DB::Field()); - break; - } - case Field::Types::UInt64: - { - UInt64 value; - DB::readVarUInt(value, buf); - x.push_back(value); - break; - } - case Field::Types::Int64: - { - Int64 value; - DB::readVarInt(value, buf); - x.push_back(value); - break; - } - case Field::Types::Float64: - { - Float64 value; - DB::readFloatBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::String: - { - std::string value; - DB::readStringBinary(value, buf); - x.push_back(value); - break; - } - case Field::Types::Array: - { - Array value; - DB::readBinary(value, buf); - x.push_back(value); - break; - } - }; - } - } + void readBinary(Array & x, ReadBuffer & buf); inline void readText(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); } inline void readQuoted(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); } /// Предполагается что у всех элементов массива одинаковый тип. - inline void writeBinary(const Array & x, WriteBuffer & buf) - { - UInt8 type = Field::Types::Null; - size_t size = x.size(); - if (size) - type = x.front().getType(); - DB::writeBinary(type, buf); - DB::writeBinary(size, buf); + void writeBinary(const Array & x, WriteBuffer & buf); - for (Array::const_iterator it = x.begin(); it != x.end(); ++it) - { - switch (type) - { - case Field::Types::Null: break; - case Field::Types::UInt64: - { - DB::writeVarUInt(get(*it), buf); - break; - } - case Field::Types::Int64: - { - DB::writeVarInt(get(*it), buf); - break; - } - case Field::Types::Float64: - { - DB::writeFloatBinary(get(*it), buf); - break; - } - case Field::Types::String: - { - DB::writeStringBinary(get(*it), buf); - break; - } - case Field::Types::Array: - { - DB::writeBinary(get(*it), buf); - break; - } - }; - } - } - - inline void writeText(const Array & x, WriteBuffer & buf) - { - DB::String res = apply_visitor(DB::FieldVisitorToString(), DB::Field(x)); - buf.write(res.data(), res.size()); - } + void writeText(const Array & x, WriteBuffer & buf); inline void writeQuoted(const Array & x, WriteBuffer & buf) { throw Exception("Cannot write Array quoted.", ErrorCodes::NOT_IMPLEMENTED); } } - - -#undef DBMS_TOTAL_FIELD_SIZE diff --git a/dbms/include/DB/Core/FieldVisitors.h b/dbms/include/DB/Core/FieldVisitors.h new file mode 100644 index 00000000000..ed6d481d602 --- /dev/null +++ b/dbms/include/DB/Core/FieldVisitors.h @@ -0,0 +1,344 @@ +#pragma once + +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +/** StaticVisitor (его наследники) - класс с перегруженными для разных типов операторами (). + * Вызвать visitor для field можно с помощью функции apply_visitor. + * Также поддерживается visitor, в котором оператор () принимает два аргумента. + */ +template +struct StaticVisitor +{ + typedef R ResultType; +}; + + +template +typename Visitor::ResultType apply_visitor_impl(Visitor & visitor, F & field) +{ + switch (field.getType()) + { + case Field::Types::Null: return visitor(field.template get()); + case Field::Types::UInt64: return visitor(field.template get()); + case Field::Types::Int64: return visitor(field.template get()); + case Field::Types::Float64: return visitor(field.template get()); + case Field::Types::String: return visitor(field.template get()); + case Field::Types::Array: return visitor(field.template get()); + + default: + throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + } +} + +/** Эти штуки нужны, чтобы принимать временный объект по константной ссылке. + * В шаблон выше, типы форвардятся уже с const-ом. + */ + +template +typename Visitor::ResultType apply_visitor(const Visitor & visitor, Field & field) +{ + return apply_visitor_impl(visitor, field); +} + +template +typename Visitor::ResultType apply_visitor(const Visitor & visitor, const Field & field) +{ + return apply_visitor_impl(visitor, field); +} + +template +typename Visitor::ResultType apply_visitor(Visitor & visitor, Field & field) +{ + return apply_visitor_impl(visitor, field); +} + +template +typename Visitor::ResultType apply_visitor(Visitor & visitor, const Field & field) +{ + return apply_visitor_impl(visitor, field); +} + + +template +typename Visitor::ResultType apply_binary_visitor_impl2(Visitor & visitor, F1 & field1, F2 & field2) +{ + switch (field2.getType()) + { + case Field::Types::Null: return visitor(field1, field2.template get()); + case Field::Types::UInt64: return visitor(field1, field2.template get()); + case Field::Types::Int64: return visitor(field1, field2.template get()); + case Field::Types::Float64: return visitor(field1, field2.template get()); + case Field::Types::String: return visitor(field1, field2.template get()); + case Field::Types::Array: return visitor(field1, field2.template get()); + + default: + throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + } +} + +template +typename Visitor::ResultType apply_binary_visitor_impl1(Visitor & visitor, F1 & field1, F2 & field2) +{ + switch (field1.getType()) + { + case Field::Types::Null: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); + case Field::Types::UInt64: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); + case Field::Types::Int64: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); + case Field::Types::Float64: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); + case Field::Types::String: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); + case Field::Types::Array: return apply_binary_visitor_impl2(visitor, field1.template get(), field2); + + default: + throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + } +} + +template +typename Visitor::ResultType apply_visitor(Visitor & visitor, Field & field1, Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + +template +typename Visitor::ResultType apply_visitor(Visitor & visitor, Field & field1, const Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + +template +typename Visitor::ResultType apply_visitor(Visitor & visitor, const Field & field1, Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + +template +typename Visitor::ResultType apply_visitor(Visitor & visitor, const Field & field1, const Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + +template +typename Visitor::ResultType apply_visitor(const Visitor & visitor, Field & field1, Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + +template +typename Visitor::ResultType apply_visitor(const Visitor & visitor, Field & field1, const Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + +template +typename Visitor::ResultType apply_visitor(const Visitor & visitor, const Field & field1, Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + +template +typename Visitor::ResultType apply_visitor(const Visitor & visitor, const Field & field1, const Field & field2) +{ + return apply_binary_visitor_impl1(visitor, field1, field2); +} + + +/** Возвращает строковый дамп типа */ +class FieldVisitorDump : public StaticVisitor +{ +private: + template + static inline String formatQuotedWithPrefix(T x, const char * prefix) + { + String res; + WriteBufferFromString wb(res); + wb.write(prefix, strlen(prefix)); + writeQuoted(x, wb); + return res; + } +public: + String operator() (const Null & x) const { return "NULL"; } + String operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); } + String operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); } + String operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); } + String operator() (const String & x) const; + String operator() (const Array & x) const; +}; + + +/** Выводит текстовое представление типа, как литерала в SQL запросе */ +class FieldVisitorToString : public StaticVisitor +{ +private: + template + static inline String formatQuoted(T x) + { + String res; + WriteBufferFromString wb(res); + writeQuoted(x, wb); + return res; + } + + /** В отличие от writeFloatText (и writeQuoted), если число после форматирования выглядит целым, всё равно добавляет десятичную точку. + * - для того, чтобы это число могло обратно распарситься как Float64 парсером запроса (иначе распарсится как целое). + * + * При этом, не оставляет завершающие нули справа. + * + * NOTE: При таком roundtrip-е, точность может теряться. + */ + static String formatFloat(const Float64 x); + +public: + String operator() (const Null & x) const { return "NULL"; } + String operator() (const UInt64 & x) const { return formatQuoted(x); } + String operator() (const Int64 & x) const { return formatQuoted(x); } + String operator() (const Float64 & x) const { return formatFloat(x); } + String operator() (const String & x) const { return formatQuoted(x); } + String operator() (const Array & x) const; +}; + + +/** Числовой тип преобразует в указанный. */ +template +class FieldVisitorConvertToNumber : public StaticVisitor +{ +public: + T operator() (const Null & x) const + { + throw Exception("Cannot convert NULL to " + TypeName::get(), ErrorCodes::CANNOT_CONVERT_TYPE); + } + + T operator() (const String & x) const + { + throw Exception("Cannot convert String to " + TypeName::get(), ErrorCodes::CANNOT_CONVERT_TYPE); + } + + T operator() (const Array & x) const + { + throw Exception("Cannot convert Array to " + TypeName::get(), ErrorCodes::CANNOT_CONVERT_TYPE); + } + + T operator() (const UInt64 & x) const { return x; } + T operator() (const Int64 & x) const { return x; } + T operator() (const Float64 & x) const { return x; } +}; + + +/// Преобразование строки с датой или датой-с-временем в UInt64, содержащим числовое значение даты или даты-с-временем. +UInt64 stringToDateOrDateTime(const String & s); + + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" + +/** Более точное сравнение. + * Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой. + * Правила сравнения - такие же, что и в FunctionsComparison. + * В том числе, сравнение знаковых и беззнаковых оставляем UB. + */ +class FieldVisitorAccurateEquals : public StaticVisitor +{ +public: + bool operator() (const Null & l, const Null & r) const { return true; } + bool operator() (const Null & l, const UInt64 & r) const { return false; } + bool operator() (const Null & l, const Int64 & r) const { return false; } + bool operator() (const Null & l, const Float64 & r) const { return false; } + bool operator() (const Null & l, const String & r) const { return false; } + bool operator() (const Null & l, const Array & r) const { return false; } + + bool operator() (const UInt64 & l, const Null & r) const { return false; } + bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; } + bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; } + bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; } + bool operator() (const UInt64 & l, const String & r) const { return l == stringToDateOrDateTime(r); } + bool operator() (const UInt64 & l, const Array & r) const { return false; } + + bool operator() (const Int64 & l, const Null & r) const { return false; } + bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; } + bool operator() (const Int64 & l, const Int64 & r) const { return l == r; } + bool operator() (const Int64 & l, const Float64 & r) const { return l == r; } + bool operator() (const Int64 & l, const String & r) const { return false; } + bool operator() (const Int64 & l, const Array & r) const { return false; } + + bool operator() (const Float64 & l, const Null & r) const { return false; } + bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; } + bool operator() (const Float64 & l, const Int64 & r) const { return l == r; } + bool operator() (const Float64 & l, const Float64 & r) const { return l == r; } + bool operator() (const Float64 & l, const String & r) const { return false; } + bool operator() (const Float64 & l, const Array & r) const { return false; } + + bool operator() (const String & l, const Null & r) const { return false; } + bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) == r; } + bool operator() (const String & l, const Int64 & r) const { return false; } + bool operator() (const String & l, const Float64 & r) const { return false; } + bool operator() (const String & l, const String & r) const { return l == r; } + bool operator() (const String & l, const Array & r) const { return false; } + + bool operator() (const Array & l, const Null & r) const { return false; } + bool operator() (const Array & l, const UInt64 & r) const { return false; } + bool operator() (const Array & l, const Int64 & r) const { return false; } + bool operator() (const Array & l, const Float64 & r) const { return false; } + bool operator() (const Array & l, const String & r) const { return false; } + bool operator() (const Array & l, const Array & r) const { return l == r; } +}; + +class FieldVisitorAccurateLess : public StaticVisitor +{ +public: + bool operator() (const Null & l, const Null & r) const { return false; } + bool operator() (const Null & l, const UInt64 & r) const { return true; } + bool operator() (const Null & l, const Int64 & r) const { return true; } + bool operator() (const Null & l, const Float64 & r) const { return true; } + bool operator() (const Null & l, const String & r) const { return true; } + bool operator() (const Null & l, const Array & r) const { return true; } + + bool operator() (const UInt64 & l, const Null & r) const { return false; } + bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; } + bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; } + bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; } + bool operator() (const UInt64 & l, const String & r) const { return l < stringToDateOrDateTime(r); } + bool operator() (const UInt64 & l, const Array & r) const { return true; } + + bool operator() (const Int64 & l, const Null & r) const { return false; } + bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; } + bool operator() (const Int64 & l, const Int64 & r) const { return l < r; } + bool operator() (const Int64 & l, const Float64 & r) const { return l < r; } + bool operator() (const Int64 & l, const String & r) const { return true; } + bool operator() (const Int64 & l, const Array & r) const { return true; } + + bool operator() (const Float64 & l, const Null & r) const { return false; } + bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; } + bool operator() (const Float64 & l, const Int64 & r) const { return l < r; } + bool operator() (const Float64 & l, const Float64 & r) const { return l < r; } + bool operator() (const Float64 & l, const String & r) const { return true; } + bool operator() (const Float64 & l, const Array & r) const { return true; } + + bool operator() (const String & l, const Null & r) const { return false; } + bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) < r; } + bool operator() (const String & l, const Int64 & r) const { return false; } + bool operator() (const String & l, const Float64 & r) const { return false; } + bool operator() (const String & l, const String & r) const { return l < r; } + bool operator() (const String & l, const Array & r) const { return true; } + + bool operator() (const Array & l, const Null & r) const { return false; } + bool operator() (const Array & l, const UInt64 & r) const { return false; } + bool operator() (const Array & l, const Int64 & r) const { return false; } + bool operator() (const Array & l, const Float64 & r) const { return false; } + bool operator() (const Array & l, const String & r) const { return false; } + bool operator() (const Array & l, const Array & r) const { return l < r; } +}; + +#pragma GCC diagnostic pop + +} diff --git a/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h b/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h index 25377433496..9ea2252a86a 100644 --- a/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h @@ -2,6 +2,7 @@ #include +#include #include #include #include diff --git a/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h b/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h index 72e3e8f36cc..2a0cf70d57d 100644 --- a/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h +++ b/dbms/include/DB/DataTypes/DataTypeAggregateFunction.h @@ -28,29 +28,7 @@ public: std::string getFunctionName() const { return function->getName(); } - std::string getName() const override - { - std::stringstream stream; - stream << "AggregateFunction(" << function->getName(); - - if (!parameters.empty()) - { - stream << "("; - for (size_t i = 0; i < parameters.size(); ++i) - { - if (i) - stream << ", "; - stream << apply_visitor(DB::FieldVisitorToString(), parameters[i]); - } - stream << ")"; - } - - for (DataTypes::const_iterator it = argument_types.begin(); it != argument_types.end(); ++it) - stream << ", " << (*it)->getName(); - - stream << ")"; - return stream.str(); - } + std::string getName() const override; DataTypePtr getReturnType() const { return function->getReturnType(); }; DataTypes getArgumentsDataTypes() const { return argument_types; } diff --git a/dbms/include/DB/DataTypes/DataTypeFixedString.h b/dbms/include/DB/DataTypes/DataTypeFixedString.h index 01d05f9c116..a5affe409aa 100644 --- a/dbms/include/DB/DataTypes/DataTypeFixedString.h +++ b/dbms/include/DB/DataTypes/DataTypeFixedString.h @@ -25,10 +25,7 @@ public: throw Exception("FixedString size must be positive", ErrorCodes::ARGUMENT_OUT_OF_BOUND); } - std::string getName() const override - { - return "FixedString(" + toString(n) + ")"; - } + std::string getName() const override; DataTypePtr clone() const override { diff --git a/dbms/include/DB/DataTypes/FieldToDataType.h b/dbms/include/DB/DataTypes/FieldToDataType.h index d0ec05b7891..0531fc2eb9d 100644 --- a/dbms/include/DB/DataTypes/FieldToDataType.h +++ b/dbms/include/DB/DataTypes/FieldToDataType.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include diff --git a/dbms/include/DB/DataTypes/IDataType.h b/dbms/include/DB/DataTypes/IDataType.h index 831048551cf..346eeca7631 100644 --- a/dbms/include/DB/DataTypes/IDataType.h +++ b/dbms/include/DB/DataTypes/IDataType.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include diff --git a/dbms/include/DB/Dictionaries/DictionaryStructure.h b/dbms/include/DB/Dictionaries/DictionaryStructure.h index 0b38b1e442d..2afce2f1dc0 100644 --- a/dbms/include/DB/Dictionaries/DictionaryStructure.h +++ b/dbms/include/DB/Dictionaries/DictionaryStructure.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include diff --git a/dbms/include/DB/Functions/FunctionsArray.h b/dbms/include/DB/Functions/FunctionsArray.h index 6751522cab8..bcf35d06791 100644 --- a/dbms/include/DB/Functions/FunctionsArray.h +++ b/dbms/include/DB/Functions/FunctionsArray.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -795,13 +797,18 @@ struct IndexCount }; -template +template struct ArrayIndexNumImpl { +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" + /// compares `lhs` against `i`-th element of `rhs` - static bool compare(const T & lhs, const PODArray & rhs, const std::size_t i ) { return lhs == rhs[i]; } + static bool compare(const T & lhs, const PODArray & rhs, const std::size_t i ) { return lhs == rhs[i]; } /// compares `lhs against `rhs`, third argument unused - static bool compare(const T & lhs, const T & rhs, std::size_t) { return lhs == rhs; } + static bool compare(const T & lhs, const U & rhs, std::size_t) { return lhs == rhs; } + +#pragma GCC diagnostic pop template static void vector( @@ -921,6 +928,21 @@ private: template bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result) + { + return executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result); + } + + template + bool executeNumberNumber(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnArray * col_array = typeid_cast(&*block.getByPosition(arguments[0]).column); @@ -934,24 +956,26 @@ private: const auto item_arg = block.getByPosition(arguments[1]).column.get(); - if (const auto item_arg_const = typeid_cast *>(item_arg)) + if (const auto item_arg_const = typeid_cast *>(item_arg)) { const auto col_res = new ResultColumnType; ColumnPtr col_ptr{col_res}; block.getByPosition(result).column = col_ptr; - ArrayIndexNumImpl::vector(col_nested->getData(), col_array->getOffsets(), + ArrayIndexNumImpl::vector(col_nested->getData(), col_array->getOffsets(), item_arg_const->getData(), col_res->getData()); } - else if (const auto item_arg_vector = typeid_cast *>(item_arg)) + else if (const auto item_arg_vector = typeid_cast *>(item_arg)) { const auto col_res = new ResultColumnType; ColumnPtr col_ptr{col_res}; block.getByPosition(result).column = col_ptr; - ArrayIndexNumImpl::vector(col_nested->getData(), col_array->getOffsets(), + ArrayIndexNumImpl::vector(col_nested->getData(), col_array->getOffsets(), item_arg_vector->getData(), col_res->getData()); } + else + return false; return true; } @@ -1010,7 +1034,7 @@ private: for (size_t i = 0, size = arr.size(); i < size; ++i) { - if (arr[i] == value) + if (apply_visitor(FieldVisitorAccurateEquals(), arr[i], value)) { if (!IndexConv::apply(i, current)) break; @@ -1035,7 +1059,7 @@ private: const auto & value = (*item_arg)[row]; for (size_t i = 0, size = arr.size(); i < size; ++i) { - if (arr[i] == value) + if (apply_visitor(FieldVisitorAccurateEquals(), arr[i], value)) { if (!IndexConv::apply(i, data[row])) break; @@ -1067,7 +1091,8 @@ public: if (!array_type) throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (array_type->getNestedType()->getName() != arguments[1]->getName()) + if (!(array_type->getNestedType()->behavesAsNumber() && arguments[1]->behavesAsNumber()) + && array_type->getNestedType()->getName() != arguments[1]->getName()) throw Exception("Type of array elements and second argument for function " + getName() + " must be same." " Passed: " + arguments[0]->getName() + " and " + arguments[1]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/include/DB/Functions/FunctionsMiscellaneous.h b/dbms/include/DB/Functions/FunctionsMiscellaneous.h index 9367488d180..3ad4b88e833 100644 --- a/dbms/include/DB/Functions/FunctionsMiscellaneous.h +++ b/dbms/include/DB/Functions/FunctionsMiscellaneous.h @@ -4,6 +4,7 @@ #include #include +#include #include #include #include diff --git a/dbms/include/DB/Functions/FunctionsString.h b/dbms/include/DB/Functions/FunctionsString.h index a807abc7301..695a7b4f6b2 100644 --- a/dbms/include/DB/Functions/FunctionsString.h +++ b/dbms/include/DB/Functions/FunctionsString.h @@ -3,6 +3,8 @@ #include #include +#include + #include #include #include diff --git a/dbms/include/DB/Functions/FunctionsTransform.h b/dbms/include/DB/Functions/FunctionsTransform.h index 87419d67bf7..62650874989 100644 --- a/dbms/include/DB/Functions/FunctionsTransform.h +++ b/dbms/include/DB/Functions/FunctionsTransform.h @@ -2,6 +2,7 @@ #include +#include #include #include #include diff --git a/dbms/include/DB/Interpreters/Set.h b/dbms/include/DB/Interpreters/Set.h index c8376892655..6150d250ba4 100644 --- a/dbms/include/DB/Interpreters/Set.h +++ b/dbms/include/DB/Interpreters/Set.h @@ -293,23 +293,7 @@ public: */ ColumnPtr execute(const Block & block, bool negative) const; - std::string describe() const - { - if (!ordered_set_elements) - return "{}"; - - bool first = true; - std::stringstream ss; - - ss << "{"; - for (const Field & f : *ordered_set_elements) - { - ss << (first ? "" : ", ") << apply_visitor(FieldVisitorToString(), f); - first = false; - } - ss << "}"; - return ss.str(); - } + std::string describe() const; /// проверяет есть ли в Set элементы для заданного диапазона индекса BoolMask mayBeTrueInRange(const Range & range) const; diff --git a/dbms/include/DB/Interpreters/SettingsCommon.h b/dbms/include/DB/Interpreters/SettingsCommon.h index c318478d62b..67066741543 100644 --- a/dbms/include/DB/Interpreters/SettingsCommon.h +++ b/dbms/include/DB/Interpreters/SettingsCommon.h @@ -5,6 +5,8 @@ #include #include #include +#include + namespace DB { diff --git a/dbms/include/DB/Parsers/ASTLiteral.h b/dbms/include/DB/Parsers/ASTLiteral.h index 17c5d87fd4e..852c193258d 100644 --- a/dbms/include/DB/Parsers/ASTLiteral.h +++ b/dbms/include/DB/Parsers/ASTLiteral.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include diff --git a/dbms/include/DB/Parsers/ASTSetQuery.h b/dbms/include/DB/Parsers/ASTSetQuery.h index 471c76d2855..beea91e13c8 100644 --- a/dbms/include/DB/Parsers/ASTSetQuery.h +++ b/dbms/include/DB/Parsers/ASTSetQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include diff --git a/dbms/include/DB/Storages/MergeTree/PKCondition.h b/dbms/include/DB/Storages/MergeTree/PKCondition.h index ead8617919c..42e6dcd07ca 100644 --- a/dbms/include/DB/Storages/MergeTree/PKCondition.h +++ b/dbms/include/DB/Storages/MergeTree/PKCondition.h @@ -2,6 +2,7 @@ #include +#include #include #include #include diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index d07a39d4485..eeeff670644 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include diff --git a/dbms/src/Core/Field.cpp b/dbms/src/Core/Field.cpp new file mode 100644 index 00000000000..7b11caa8398 --- /dev/null +++ b/dbms/src/Core/Field.cpp @@ -0,0 +1,141 @@ +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace mysqlxx +{ + std::ostream & operator<< (mysqlxx::EscapeManipResult res, const DB::Array & value) + { + return res.ostr << apply_visitor(DB::FieldVisitorToString(), value); + } + + std::ostream & operator<< (mysqlxx::QuoteManipResult res, const DB::Array & value) + { + throw Poco::Exception("Cannot quote Array with mysqlxx::quote."); + } + + std::istream & operator>> (mysqlxx::UnEscapeManipResult res, DB::Array & value) + { + throw Poco::Exception("Cannot unescape Array with mysqlxx::unescape."); + } + + std::istream & operator>> (mysqlxx::UnQuoteManipResult res, DB::Array & value) + { + throw Poco::Exception("Cannot unquote Array with mysqlxx::unquote."); + } +} + + +namespace DB +{ + inline void readBinary(Array & x, ReadBuffer & buf) + { + size_t size; + UInt8 type; + DB::readBinary(type, buf); + DB::readBinary(size, buf); + + for (size_t index = 0; index < size; ++index) + { + switch (type) + { + case Field::Types::Null: + { + x.push_back(DB::Field()); + break; + } + case Field::Types::UInt64: + { + UInt64 value; + DB::readVarUInt(value, buf); + x.push_back(value); + break; + } + case Field::Types::Int64: + { + Int64 value; + DB::readVarInt(value, buf); + x.push_back(value); + break; + } + case Field::Types::Float64: + { + Float64 value; + DB::readFloatBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::String: + { + std::string value; + DB::readStringBinary(value, buf); + x.push_back(value); + break; + } + case Field::Types::Array: + { + Array value; + DB::readBinary(value, buf); + x.push_back(value); + break; + } + }; + } + } + + void writeBinary(const Array & x, WriteBuffer & buf) + { + UInt8 type = Field::Types::Null; + size_t size = x.size(); + if (size) + type = x.front().getType(); + DB::writeBinary(type, buf); + DB::writeBinary(size, buf); + + for (Array::const_iterator it = x.begin(); it != x.end(); ++it) + { + switch (type) + { + case Field::Types::Null: break; + case Field::Types::UInt64: + { + DB::writeVarUInt(get(*it), buf); + break; + } + case Field::Types::Int64: + { + DB::writeVarInt(get(*it), buf); + break; + } + case Field::Types::Float64: + { + DB::writeFloatBinary(get(*it), buf); + break; + } + case Field::Types::String: + { + DB::writeStringBinary(get(*it), buf); + break; + } + case Field::Types::Array: + { + DB::writeBinary(get(*it), buf); + break; + } + }; + } + } + + void writeText(const Array & x, WriteBuffer & buf) + { + DB::String res = apply_visitor(DB::FieldVisitorToString(), DB::Field(x)); + buf.write(res.data(), res.size()); + } +} diff --git a/dbms/src/Core/FieldVisitors.cpp b/dbms/src/Core/FieldVisitors.cpp new file mode 100644 index 00000000000..b9b7c1cb417 --- /dev/null +++ b/dbms/src/Core/FieldVisitors.cpp @@ -0,0 +1,88 @@ +#include + + +namespace DB +{ + + +String FieldVisitorDump::operator() (const String & x) const +{ + String res; + WriteBufferFromString wb(res); + writeQuoted(x, wb); + return res; +} + +String FieldVisitorDump::operator() (const Array & x) const +{ + String res; + WriteBufferFromString wb(res); + FieldVisitorDump visitor; + + wb.write("Array_[", 7); + for (Array::const_iterator it = x.begin(); it != x.end(); ++it) + { + if (it != x.begin()) + wb.write(", ", 2); + writeString(apply_visitor(visitor, *it), wb); + } + writeChar(']', wb); + + return res; +} + + +String FieldVisitorToString::formatFloat(const Float64 x) +{ + char tmp[25]; + double_conversion::StringBuilder builder{tmp, sizeof(tmp)}; + + const auto result = getDoubleToStringConverter().ToShortest(x, &builder); + + if (!result) + throw Exception("Cannot print float or double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER); + + return { tmp, tmp + builder.position() }; +} + +String FieldVisitorToString::operator() (const Array & x) const +{ + String res; + WriteBufferFromString wb(res); + FieldVisitorToString visitor; + + writeChar('[', wb); + for (Array::const_iterator it = x.begin(); it != x.end(); ++it) + { + if (it != x.begin()) + wb.write(", ", 2); + writeString(apply_visitor(visitor, *it), wb); + } + writeChar(']', wb); + + return res; +} + + +UInt64 stringToDateOrDateTime(const String & s) +{ + ReadBufferFromString in(s); + + if (s.size() == strlen("YYYY-MM-DD")) + { + DayNum_t date{}; + readDateText(date, in); + return UInt64(date); + } + else + { + time_t date_time{}; + readDateTimeText(date_time, in); + if (!in.eof()) + throw Exception("String is too long for DateTime: " + s); + return UInt64(date_time); + } +} + + +} diff --git a/dbms/src/Core/NamesAndTypes.cpp b/dbms/src/Core/NamesAndTypes.cpp index 57dfe42dd0b..b093325410c 100644 --- a/dbms/src/Core/NamesAndTypes.cpp +++ b/dbms/src/Core/NamesAndTypes.cpp @@ -1,5 +1,9 @@ #include #include +#include +#include +#include +#include namespace DB diff --git a/dbms/src/Core/tests/field.cpp b/dbms/src/Core/tests/field.cpp index bf091ba6cef..857700166cb 100644 --- a/dbms/src/Core/tests/field.cpp +++ b/dbms/src/Core/tests/field.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -13,7 +14,7 @@ int main(int argc, char ** argv) { DB::FieldVisitorToString to_string; - + DB::Field field = DB::UInt64(0); std::cerr << DB::apply_visitor(to_string, field) << std::endl; @@ -94,8 +95,8 @@ int main(int argc, char ** argv) std::cerr << e.what() << ", " << e.displayText() << std::endl; return 1; } - + std::cerr << "sizeof(Field) = " << sizeof(DB::Field) << std::endl; - + return 0; } diff --git a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp index c90bea9636e..368996fb65d 100644 --- a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp @@ -1,3 +1,4 @@ +#include #include /// Максимальное количество сообщений о некорректных данных в логе. diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp index 080baed8276..9c35af9d1d3 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp @@ -1,3 +1,5 @@ +#include + #include #include @@ -13,6 +15,30 @@ namespace DB using Poco::SharedPtr; +std::string DataTypeAggregateFunction::getName() const +{ + std::stringstream stream; + stream << "AggregateFunction(" << function->getName(); + + if (!parameters.empty()) + { + stream << "("; + for (size_t i = 0; i < parameters.size(); ++i) + { + if (i) + stream << ", "; + stream << apply_visitor(DB::FieldVisitorToString(), parameters[i]); + } + stream << ")"; + } + + for (DataTypes::const_iterator it = argument_types.begin(); it != argument_types.end(); ++it) + stream << ", " << (*it)->getName(); + + stream << ")"; + return stream.str(); +} + void DataTypeAggregateFunction::serializeBinary(const Field & field, WriteBuffer & ostr) const { const String & s = get(field); diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index 6e5a86463ae..ff24ba020b2 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -1,5 +1,8 @@ #include +#include +#include + #include #include #include @@ -17,6 +20,12 @@ namespace DB using Poco::SharedPtr; +std::string DataTypeFixedString::getName() const +{ + return "FixedString(" + toString(n) + ")"; +} + + void DataTypeFixedString::serializeBinary(const Field & field, WriteBuffer & ostr) const { const String & s = get(field); diff --git a/dbms/src/DataTypes/FieldToDataType.cpp b/dbms/src/DataTypes/FieldToDataType.cpp index 52e0d611b8f..c151f6cad7b 100644 --- a/dbms/src/DataTypes/FieldToDataType.cpp +++ b/dbms/src/DataTypes/FieldToDataType.cpp @@ -1,5 +1,7 @@ +#include #include + namespace DB { diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 00e0cd2cbf6..3665ca857db 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -678,4 +679,23 @@ BoolMask Set::mayBeTrueInRange(const Range & range) const return BoolMask(can_be_true, can_be_false); } + +std::string Set::describe() const +{ + if (!ordered_set_elements) + return "{}"; + + bool first = true; + std::stringstream ss; + + ss << "{"; + for (const Field & f : *ordered_set_elements) + { + ss << (first ? "" : ", ") << apply_visitor(FieldVisitorToString(), f); + first = false; + } + ss << "}"; + return ss.str(); +} + } diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp index 9c6392b1e0f..0bbcbaecad0 100644 --- a/dbms/src/Parsers/ASTLiteral.cpp +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -1,4 +1,5 @@ #include +#include #include diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 882876b7697..9c3f1df2579 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 49a50d3c2b2..989c629a4b8 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -1,4 +1,8 @@ #include +#include +#include +#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 14725490205..b86d5336fe8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1,3 +1,4 @@ +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/PKCondition.cpp b/dbms/src/Storages/MergeTree/PKCondition.cpp index 422c5b2245c..77c65fc3b8e 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.cpp +++ b/dbms/src/Storages/MergeTree/PKCondition.cpp @@ -63,130 +63,6 @@ const PKCondition::AtomMap PKCondition::atom_map{ } }; -/// Преобразование строки с датой или датой-с-временем в UInt64, содержащим числовое значение даты или даты-с-временем. -UInt64 stringToDateOrDateTime(const String & s) -{ - ReadBufferFromString in(s); - - if (s.size() == strlen("YYYY-MM-DD")) - { - DayNum_t date{}; - readDateText(date, in); - return UInt64(date); - } - else - { - time_t date_time{}; - readDateTimeText(date_time, in); - if (!in.eof()) - throw Exception("String is too long for DateTime: " + s); - return UInt64(date_time); - } -} - - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wsign-compare" - -/** Более точное сравнение. - * Отличается от Field::operator< и Field::operator== тем, что сравнивает значения разных числовых типов между собой. - * Правила сравнения - такие же, что и в FunctionsComparison. - * В том числе, сравнение знаковых и беззнаковых оставляем UB. - */ -class FieldVisitorAccurateEquals : public StaticVisitor -{ -public: - bool operator() (const Null & l, const Null & r) const { return true; } - bool operator() (const Null & l, const UInt64 & r) const { return false; } - bool operator() (const Null & l, const Int64 & r) const { return false; } - bool operator() (const Null & l, const Float64 & r) const { return false; } - bool operator() (const Null & l, const String & r) const { return false; } - bool operator() (const Null & l, const Array & r) const { return false; } - - bool operator() (const UInt64 & l, const Null & r) const { return false; } - bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const UInt64 & l, const Int64 & r) const { return l == r; } - bool operator() (const UInt64 & l, const Float64 & r) const { return l == r; } - bool operator() (const UInt64 & l, const String & r) const { return l == stringToDateOrDateTime(r); } - bool operator() (const UInt64 & l, const Array & r) const { return false; } - - bool operator() (const Int64 & l, const Null & r) const { return false; } - bool operator() (const Int64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const Int64 & l, const Int64 & r) const { return l == r; } - bool operator() (const Int64 & l, const Float64 & r) const { return l == r; } - bool operator() (const Int64 & l, const String & r) const { return false; } - bool operator() (const Int64 & l, const Array & r) const { return false; } - - bool operator() (const Float64 & l, const Null & r) const { return false; } - bool operator() (const Float64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const Float64 & l, const Int64 & r) const { return l == r; } - bool operator() (const Float64 & l, const Float64 & r) const { return l == r; } - bool operator() (const Float64 & l, const String & r) const { return false; } - bool operator() (const Float64 & l, const Array & r) const { return false; } - - bool operator() (const String & l, const Null & r) const { return false; } - bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) == r; } - bool operator() (const String & l, const Int64 & r) const { return false; } - bool operator() (const String & l, const Float64 & r) const { return false; } - bool operator() (const String & l, const String & r) const { return l == r; } - bool operator() (const String & l, const Array & r) const { return false; } - - bool operator() (const Array & l, const Null & r) const { return false; } - bool operator() (const Array & l, const UInt64 & r) const { return false; } - bool operator() (const Array & l, const Int64 & r) const { return false; } - bool operator() (const Array & l, const Float64 & r) const { return false; } - bool operator() (const Array & l, const String & r) const { return false; } - bool operator() (const Array & l, const Array & r) const { return l == r; } -}; - -class FieldVisitorAccurateLess : public StaticVisitor -{ -public: - bool operator() (const Null & l, const Null & r) const { return false; } - bool operator() (const Null & l, const UInt64 & r) const { return true; } - bool operator() (const Null & l, const Int64 & r) const { return true; } - bool operator() (const Null & l, const Float64 & r) const { return true; } - bool operator() (const Null & l, const String & r) const { return true; } - bool operator() (const Null & l, const Array & r) const { return true; } - - bool operator() (const UInt64 & l, const Null & r) const { return false; } - bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const UInt64 & l, const Int64 & r) const { return l < r; } - bool operator() (const UInt64 & l, const Float64 & r) const { return l < r; } - bool operator() (const UInt64 & l, const String & r) const { return l < stringToDateOrDateTime(r); } - bool operator() (const UInt64 & l, const Array & r) const { return true; } - - bool operator() (const Int64 & l, const Null & r) const { return false; } - bool operator() (const Int64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const Int64 & l, const Int64 & r) const { return l < r; } - bool operator() (const Int64 & l, const Float64 & r) const { return l < r; } - bool operator() (const Int64 & l, const String & r) const { return true; } - bool operator() (const Int64 & l, const Array & r) const { return true; } - - bool operator() (const Float64 & l, const Null & r) const { return false; } - bool operator() (const Float64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const Float64 & l, const Int64 & r) const { return l < r; } - bool operator() (const Float64 & l, const Float64 & r) const { return l < r; } - bool operator() (const Float64 & l, const String & r) const { return true; } - bool operator() (const Float64 & l, const Array & r) const { return true; } - - bool operator() (const String & l, const Null & r) const { return false; } - bool operator() (const String & l, const UInt64 & r) const { return stringToDateOrDateTime(l) < r; } - bool operator() (const String & l, const Int64 & r) const { return false; } - bool operator() (const String & l, const Float64 & r) const { return false; } - bool operator() (const String & l, const String & r) const { return l < r; } - bool operator() (const String & l, const Array & r) const { return true; } - - bool operator() (const Array & l, const Null & r) const { return false; } - bool operator() (const Array & l, const UInt64 & r) const { return false; } - bool operator() (const Array & l, const Int64 & r) const { return false; } - bool operator() (const Array & l, const Float64 & r) const { return false; } - bool operator() (const Array & l, const String & r) const { return false; } - bool operator() (const Array & l, const Array & r) const { return l < r; } -}; - -#pragma GCC diagnostic pop - inline bool Range::equals(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateEquals(), lhs, rhs); } inline bool Range::less(const Field & lhs, const Field & rhs) { return apply_visitor(FieldVisitorAccurateLess(), lhs, rhs); } diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 34345c03632..50ee571929f 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -1,6 +1,8 @@ #include #include +#include + #include #include #include diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 09aa2a03b16..ea6fd49bed8 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -1,3 +1,4 @@ +#include #include #include #include diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index fcc51f7bd6c..14a6a53620a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include From 7f06d70d45da13e1402b349563626d681e29a558 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 12 Oct 2015 10:15:57 +0300 Subject: [PATCH 16/16] dbms: added test [#METR-18382]. --- .../0_stateless/00251_has_types.reference | 20 ++++++++++++++++ .../queries/0_stateless/00251_has_types.sql | 23 +++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00251_has_types.reference create mode 100644 dbms/tests/queries/0_stateless/00251_has_types.sql diff --git a/dbms/tests/queries/0_stateless/00251_has_types.reference b/dbms/tests/queries/0_stateless/00251_has_types.reference new file mode 100644 index 00000000000..e22d7202705 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00251_has_types.reference @@ -0,0 +1,20 @@ +1 +1 +0 +1 +0 +1 +1 +0 +1 +0 +1 +1 +0 +1 +0 +1 +1 +0 +1 +0 diff --git a/dbms/tests/queries/0_stateless/00251_has_types.sql b/dbms/tests/queries/0_stateless/00251_has_types.sql new file mode 100644 index 00000000000..f64354b15eb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00251_has_types.sql @@ -0,0 +1,23 @@ +SELECT has([1, 2, 3], 3.0); +SELECT has([1, 2.0, 3], 2); +SELECT has([1, 2.1, 3], 2); +SELECT has([1, -1], 1); +SELECT has([1, -1], 1000); + +SELECT has(materialize([1, 2, 3]), 3.0); +SELECT has(materialize([1, 2.0, 3]), 2); +SELECT has(materialize([1, 2.1, 3]), 2); +SELECT has(materialize([1, -1]), 1); +SELECT has(materialize([1, -1]), 1000); + +SELECT has([1, 2, 3], materialize(3.0)); +SELECT has([1, 2.0, 3], materialize(2)); +SELECT has([1, 2.1, 3], materialize(2)); +SELECT has([1, -1], materialize(1)); +SELECT has([1, -1], materialize(1000)); + +SELECT has(materialize([1, 2, 3]), materialize(3.0)); +SELECT has(materialize([1, 2.0, 3]), materialize(2)); +SELECT has(materialize([1, 2.1, 3]), materialize(2)); +SELECT has(materialize([1, -1]), materialize(1)); +SELECT has(materialize([1, -1]), materialize(1000));