From 9d9d16e1ea74652557d29513469dfe9c8bee131b Mon Sep 17 00:00:00 2001 From: bgranvea Date: Fri, 8 Mar 2019 17:49:10 +0100 Subject: [PATCH 01/58] support for SimpleAggregateFunction data type --- .../AggregatingSortedBlockInputStream.cpp | 44 +++++- .../AggregatingSortedBlockInputStream.h | 51 ++++++ .../DataTypes/DataTypeDomainIPv4AndIPv6.cpp | 18 +-- .../DataTypeDomainSimpleAggregateFunction.cpp | 149 ++++++++++++++++++ .../DataTypeDomainSimpleAggregateFunction.h | 45 ++++++ .../DataTypeDomainWithSimpleSerialization.h | 2 +- dbms/src/DataTypes/DataTypeFactory.cpp | 31 ++-- dbms/src/DataTypes/DataTypeFactory.h | 13 +- dbms/src/DataTypes/IDataType.cpp | 53 +++---- dbms/src/DataTypes/IDataType.h | 11 +- dbms/src/DataTypes/IDataTypeDomain.h | 41 +++-- .../00915_simple_aggregate_function.reference | 43 +++++ .../00915_simple_aggregate_function.sql | 27 ++++ 13 files changed, 454 insertions(+), 74 deletions(-) create mode 100644 dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp create mode 100644 dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h create mode 100644 dbms/tests/queries/0_stateless/00915_simple_aggregate_function.reference create mode 100644 dbms/tests/queries/0_stateless/00915_simple_aggregate_function.sql diff --git a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp index 0697ec8167c..34fb19b2688 100644 --- a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include namespace DB @@ -22,7 +24,7 @@ AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream( ColumnWithTypeAndName & column = header.safeGetByPosition(i); /// We leave only states of aggregate functions. - if (!startsWith(column.type->getName(), "AggregateFunction")) + if (!dynamic_cast(column.type.get()) && !findSimpleAggregateFunction(column.type)) { column_numbers_not_to_aggregate.push_back(i); continue; @@ -40,7 +42,14 @@ AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream( continue; } - column_numbers_to_aggregate.push_back(i); + if (auto simple_aggr = findSimpleAggregateFunction(column.type)) { + // simple aggregate function + SimpleAggregateDescription desc{simple_aggr->getFunction(), i}; + columns_to_simple_aggregate.emplace_back(std::move(desc)); + } else { + // standard aggregate function + column_numbers_to_aggregate.push_back(i); + } } } @@ -90,8 +99,11 @@ void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, s key_differs = next_key != current_key; /// if there are enough rows accumulated and the last one is calculated completely - if (key_differs && merged_rows >= max_block_size) + if (key_differs && merged_rows >= max_block_size) { + /// Write the simple aggregation result for the previous group. + insertSimpleAggregationResult(merged_columns); return; + } queue.pop(); @@ -110,6 +122,14 @@ void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, s for (auto & column_to_aggregate : columns_to_aggregate) column_to_aggregate->insertDefault(); + /// Write the simple aggregation result for the previous group. + if (merged_rows > 0) + insertSimpleAggregationResult(merged_columns); + + /// Reset simple aggregation states for next row + for (auto & desc : columns_to_simple_aggregate) + desc.createState(); + ++merged_rows; } @@ -127,6 +147,9 @@ void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, s } } + /// Write the simple aggregation result for the previous group. + insertSimpleAggregationResult(merged_columns); + finished = true; } @@ -138,6 +161,21 @@ void AggregatingSortedBlockInputStream::addRow(SortCursor & cursor) size_t j = column_numbers_to_aggregate[i]; columns_to_aggregate[i]->insertMergeFrom(*cursor->all_columns[j], cursor->pos); } + + for (auto & desc : columns_to_simple_aggregate) + { + auto & col = cursor->all_columns[desc.column_number]; + desc.add_function(desc.function.get(), desc.state.data(), &col, cursor->pos, nullptr); + } +} + +void AggregatingSortedBlockInputStream::insertSimpleAggregationResult(MutableColumns & merged_columns) +{ + for (auto & desc : columns_to_simple_aggregate) + { + desc.function->insertResultInto(desc.state.data(), *merged_columns[desc.column_number]); + desc.destroyState(); + } } } diff --git a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h index 522b54aeaec..97a579e31a6 100644 --- a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -38,10 +39,13 @@ private: /// Read finished. bool finished = false; + struct SimpleAggregateDescription; + /// Columns with which numbers should be aggregated. ColumnNumbers column_numbers_to_aggregate; ColumnNumbers column_numbers_not_to_aggregate; std::vector columns_to_aggregate; + std::vector columns_to_simple_aggregate; RowRef current_key; /// The current primary key. RowRef next_key; /// The primary key of the next row. @@ -54,6 +58,53 @@ private: /** Extract all states of aggregate functions and merge them with the current group. */ void addRow(SortCursor & cursor); + + /** Insert all values of current row for simple aggregate functions + */ + void insertSimpleAggregationResult(MutableColumns & merged_columns); + + /// Stores information for aggregation of SimpleAggregateFunction columns + struct SimpleAggregateDescription + { + /// An aggregate function 'anyLast', 'sum'... + AggregateFunctionPtr function; + IAggregateFunction::AddFunc add_function; + size_t column_number; + AlignedBuffer state; + bool created = false; + + SimpleAggregateDescription(const AggregateFunctionPtr & function_, const size_t column_number_) : function(function_), column_number(column_number_) + { + add_function = function->getAddressOfAddFunction(); + state.reset(function->sizeOfData(), function->alignOfData()); + } + + void createState() + { + if (created) + return; + function->create(state.data()); + created = true; + } + + void destroyState() + { + if (!created) + return; + function->destroy(state.data()); + created = false; + } + + /// Explicitly destroy aggregation state if the stream is terminated + ~SimpleAggregateDescription() + { + destroyState(); + } + + SimpleAggregateDescription() = default; + SimpleAggregateDescription(SimpleAggregateDescription &&) = default; + SimpleAggregateDescription(const SimpleAggregateDescription &) = delete; + }; }; } diff --git a/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp b/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp index 873dbde506b..f57a6167d3d 100644 --- a/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp +++ b/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp @@ -23,7 +23,7 @@ namespace class DataTypeDomainIPv4 : public DataTypeDomainWithSimpleSerialization { public: - const char * getName() const override + String doGetName() const override { return "IPv4"; } @@ -33,7 +33,7 @@ public: const auto col = checkAndGetColumn(&column); if (!col) { - throw Exception(String(getName()) + " domain can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(getName() + " domain can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -48,7 +48,7 @@ public: ColumnUInt32 * col = typeid_cast(&column); if (!col) { - throw Exception(String(getName()) + " domain can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(getName() + " domain can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -66,7 +66,7 @@ public: class DataTypeDomainIPv6 : public DataTypeDomainWithSimpleSerialization { public: - const char * getName() const override + String doGetName() const override { return "IPv6"; } @@ -76,7 +76,7 @@ public: const auto col = checkAndGetColumn(&column); if (!col) { - throw Exception(String(getName()) + " domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(getName() + " domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -91,7 +91,7 @@ public: ColumnFixedString * col = typeid_cast(&column); if (!col) { - throw Exception(String(getName()) + " domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(getName() + " domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -100,7 +100,7 @@ public: std::string ipv6_value(IPV6_BINARY_LENGTH, '\0'); if (!parseIPv6(buffer, reinterpret_cast(ipv6_value.data()))) { - throw Exception(String("Invalid ") + getName() + " value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + throw Exception("Invalid " + getName() + " value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); } col->insertString(ipv6_value); @@ -111,8 +111,8 @@ public: void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory) { - factory.registerDataTypeDomain("UInt32", std::make_unique()); - factory.registerDataTypeDomain("FixedString(16)", std::make_unique()); + factory.registerDataTypeDomain("IPv4", [] { return std::make_pair(DataTypeFactory::instance().get("UInt32"), std::make_unique()); }); + factory.registerDataTypeDomain("IPv6", [] { return std::make_pair(DataTypeFactory::instance().get("FixedString(16)"), std::make_unique()); }); } } // namespace DB diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp new file mode 100644 index 00000000000..402ce86ad62 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp @@ -0,0 +1,149 @@ +#include +#include + +#include + +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace DB { + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int BAD_ARGUMENTS; + extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; +} + +const std::vector supported_functions = std::vector( + {"any", "anyLast", "min", "max", "sum"}); + + +String DataTypeDomainSimpleAggregateFunction::doGetName() const { + std::stringstream stream; + stream << "SimpleAggregateFunction(" << function->getName(); + + if (!parameters.empty()) { + stream << "("; + for (size_t i = 0; i < parameters.size(); ++i) { + if (i) + stream << ", "; + stream << applyVisitor(DB::FieldVisitorToString(), parameters[i]); + } + stream << ")"; + } + + for (const auto &argument_type : argument_types) + stream << ", " << argument_type->getName(); + + stream << ")"; + return stream.str(); +} + + +static std::pair create(const ASTPtr & arguments) +{ + String function_name; + AggregateFunctionPtr function; + DataTypes argument_types; + Array params_row; + + if (!arguments || arguments->children.empty()) + throw Exception("Data type SimpleAggregateFunction requires parameters: " + "name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (const ASTFunction * parametric = typeid_cast(arguments->children[0].get())) + { + if (parametric->parameters) + throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR); + function_name = parametric->name; + + const ASTs & parameters = typeid_cast(*parametric->arguments).children; + params_row.resize(parameters.size()); + + for (size_t i = 0; i < parameters.size(); ++i) + { + const ASTLiteral * lit = typeid_cast(parameters[i].get()); + if (!lit) + throw Exception("Parameters to aggregate functions must be literals", + ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS); + + params_row[i] = lit->value; + } + } + else if (auto opt_name = getIdentifierName(arguments->children[0])) + { + function_name = *opt_name; + } + else if (typeid_cast(arguments->children[0].get())) + { + throw Exception("Aggregate function name for data type SimpleAggregateFunction must be passed as identifier (without quotes) or function", + ErrorCodes::BAD_ARGUMENTS); + } + else + throw Exception("Unexpected AST element passed as aggregate function name for data type SimpleAggregateFunction. Must be identifier or function.", + ErrorCodes::BAD_ARGUMENTS); + + for (size_t i = 1; i < arguments->children.size(); ++i) + argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i])); + + if (function_name.empty()) + throw Exception("Logical error: empty name of aggregate function passed", ErrorCodes::LOGICAL_ERROR); + + function = AggregateFunctionFactory::instance().get(function_name, argument_types, params_row); + + // check function + if (std::find(std::begin(supported_functions), std::end(supported_functions), function->getName()) == std::end(supported_functions)) { + throw Exception("Unsupported aggregate function " + function->getName() + ", supported functions are " + boost::algorithm::join(supported_functions, ","), + ErrorCodes::BAD_ARGUMENTS); + } + + DataTypePtr storage_type = DataTypeFactory::instance().get(argument_types[0]->getName()); + DataTypeDomainPtr domain = std::make_unique(storage_type, function, argument_types, params_row); + + if (!function->getReturnType()->equals(*removeLowCardinality(storage_type))) { + throw Exception("Incompatible data types between aggregate function '" + function->getName() + "' which returns " + function->getReturnType()->getName() + " and column storage type " + storage_type->getName(), + ErrorCodes::BAD_ARGUMENTS); + } + + return std::make_pair(storage_type, std::move(domain)); +} + +static const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(const IDataTypeDomain * domain) { + if (domain == nullptr) + return nullptr; + + if (auto simple_aggr = dynamic_cast(domain)) + return simple_aggr; + + if (domain->getDomain() != nullptr) + return findSimpleAggregateFunction(domain->getDomain()); + + return nullptr; +} + +const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(DataTypePtr dataType) { + return findSimpleAggregateFunction(dataType->getDomain()); +} + + +void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory) +{ + factory.registerDataTypeDomain("SimpleAggregateFunction", create); +} + +} diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h new file mode 100644 index 00000000000..70e94b1a652 --- /dev/null +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +/** The type SimpleAggregateFunction(fct, type) is meant to be used in an AggregatingMergeTree. It behaves like a standard + * data type but when rows are merged, an aggregation function is applied. + * + * The aggregation function is limited to simple functions whose merge state is the final result: + * any, anyLast, min, max, sum + * + * Examples: + * + * SimpleAggregateFunction(sum, Nullable(Float64)) + * SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) + * SimpleAggregateFunction(anyLast, IPv4) + * + * Technically, a standard IDataType is instanciated and a DataTypeDomainSimpleAggregateFunction is added as domain. + */ + +class DataTypeDomainSimpleAggregateFunction : public IDataTypeDomain { +private: + const DataTypePtr storage_type; + const AggregateFunctionPtr function; + const DataTypes argument_types; + const Array parameters; + +public: + DataTypeDomainSimpleAggregateFunction(const DataTypePtr storage_type_, const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_) + : storage_type(storage_type_), function(function_), argument_types(argument_types_), parameters(parameters_) {} + + const AggregateFunctionPtr getFunction() const { return function; } + String doGetName() const override; +}; + +/// recursively follow data type domain to find a DataTypeDomainSimpleAggregateFunction +const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(DataTypePtr dataType); + +} diff --git a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h b/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h index 7834e9235d2..3ccb4091636 100644 --- a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h +++ b/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h @@ -12,7 +12,7 @@ class IColumn; /** Simple DataTypeDomain that uses serializeText/deserializeText * for all serialization and deserialization. */ -class DataTypeDomainWithSimpleSerialization : public IDataTypeDomain +class DataTypeDomainWithSimpleSerialization : public IDataTypeDomainCustomSerialization { public: virtual ~DataTypeDomainWithSimpleSerialization() override; diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index a0afab890e9..a405075e884 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -115,19 +115,23 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator }, case_sensitiveness); } -void DataTypeFactory::registerDataTypeDomain(const String & type_name, DataTypeDomainPtr domain, CaseSensitiveness case_sensitiveness) +void DataTypeFactory::registerDataTypeDomain(const String & family_name, CreatorWithDomain creator, CaseSensitiveness case_sensitiveness) { - all_domains.reserve(all_domains.size() + 1); - - auto data_type = get(type_name); - setDataTypeDomain(*data_type, *domain); - - registerDataType(domain->getName(), [data_type](const ASTPtr & /*ast*/) + registerDataType(family_name, [creator](const ASTPtr & ast) { - return data_type; - }, case_sensitiveness); + auto res = creator(ast); + res.first->appendDomain(std::move(res.second)); - all_domains.emplace_back(std::move(domain)); + return res.first; + }, case_sensitiveness); +} + +void DataTypeFactory::registerDataTypeDomain(const String & name, SimpleCreatorWithDomain creator, CaseSensitiveness case_sensitiveness) +{ + registerDataTypeDomain(name, [creator](const ASTPtr & /*ast*/) + { + return creator(); + }, case_sensitiveness); } const DataTypeFactory::Creator& DataTypeFactory::findCreatorByName(const String & family_name) const @@ -153,11 +157,6 @@ const DataTypeFactory::Creator& DataTypeFactory::findCreatorByName(const String throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE); } -void DataTypeFactory::setDataTypeDomain(const IDataType & data_type, const IDataTypeDomain & domain) -{ - data_type.setDomain(&domain); -} - void registerDataTypeNumbers(DataTypeFactory & factory); void registerDataTypeDecimal(DataTypeFactory & factory); void registerDataTypeDate(DataTypeFactory & factory); @@ -175,6 +174,7 @@ void registerDataTypeNested(DataTypeFactory & factory); void registerDataTypeInterval(DataTypeFactory & factory); void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); +void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); DataTypeFactory::DataTypeFactory() @@ -196,6 +196,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeInterval(*this); registerDataTypeLowCardinality(*this); registerDataTypeDomainIPv4AndIPv6(*this); + registerDataTypeDomainSimpleAggregateFunction(*this); } DataTypeFactory::~DataTypeFactory() diff --git a/dbms/src/DataTypes/DataTypeFactory.h b/dbms/src/DataTypes/DataTypeFactory.h index c6ef100bbb7..e4a82b342d1 100644 --- a/dbms/src/DataTypes/DataTypeFactory.h +++ b/dbms/src/DataTypes/DataTypeFactory.h @@ -28,6 +28,8 @@ class DataTypeFactory final : public ext::singleton, public IFa private: using SimpleCreator = std::function; using DataTypesDictionary = std::unordered_map; + using CreatorWithDomain = std::function(const ASTPtr & parameters)>; + using SimpleCreatorWithDomain = std::function()>; public: DataTypePtr get(const String & full_name) const; @@ -40,11 +42,13 @@ public: /// Register a simple data type, that have no parameters. void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - // Register a domain - a refinement of existing type. - void registerDataTypeDomain(const String & type_name, DataTypeDomainPtr domain, CaseSensitiveness case_sensitiveness = CaseSensitive); + /// Register a type family with a dynamic domain + void registerDataTypeDomain(const String & family_name, CreatorWithDomain creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + + /// Register a simple data type domain + void registerDataTypeDomain(const String & name, SimpleCreatorWithDomain creator, CaseSensitiveness case_sensitiveness = CaseSensitive); private: - static void setDataTypeDomain(const IDataType & data_type, const IDataTypeDomain & domain); const Creator& findCreatorByName(const String & family_name) const; private: @@ -53,9 +57,6 @@ private: /// Case insensitive data types will be additionally added here with lowercased name. DataTypesDictionary case_insensitive_data_types; - // All domains are owned by factory and shared amongst DataType instances. - std::vector all_domains; - DataTypeFactory(); ~DataTypeFactory() override; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index 679871dba71..0270f1d7923 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -142,9 +142,9 @@ void IDataType::insertDefaultInto(IColumn & column) const void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->serializeTextEscaped(column, row_num, ostr, settings); + ser_domain->serializeTextEscaped(column, row_num, ostr, settings); } else { @@ -154,9 +154,9 @@ void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, W void IDataType::deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->deserializeTextEscaped(column, istr, settings); + ser_domain->deserializeTextEscaped(column, istr, settings); } else { @@ -166,9 +166,9 @@ void IDataType::deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, co void IDataType::serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->serializeTextQuoted(column, row_num, ostr, settings); + ser_domain->serializeTextQuoted(column, row_num, ostr, settings); } else { @@ -178,9 +178,9 @@ void IDataType::serializeAsTextQuoted(const IColumn & column, size_t row_num, Wr void IDataType::deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->deserializeTextQuoted(column, istr, settings); + ser_domain->deserializeTextQuoted(column, istr, settings); } else { @@ -190,9 +190,9 @@ void IDataType::deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, con void IDataType::serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (domain) - { - domain->serializeTextCSV(column, row_num, ostr, settings); + if (auto ser_domain = dynamic_cast(domain.get())) + { + ser_domain->serializeTextCSV(column, row_num, ostr, settings); } else { @@ -202,9 +202,9 @@ void IDataType::serializeAsTextCSV(const IColumn & column, size_t row_num, Write void IDataType::deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->deserializeTextCSV(column, istr, settings); + ser_domain->deserializeTextCSV(column, istr, settings); } else { @@ -214,9 +214,9 @@ void IDataType::deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const void IDataType::serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->serializeText(column, row_num, ostr, settings); + ser_domain->serializeText(column, row_num, ostr, settings); } else { @@ -226,9 +226,9 @@ void IDataType::serializeAsText(const IColumn & column, size_t row_num, WriteBuf void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->serializeTextJSON(column, row_num, ostr, settings); + ser_domain->serializeTextJSON(column, row_num, ostr, settings); } else { @@ -238,9 +238,9 @@ void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, Writ void IDataType::deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->deserializeTextJSON(column, istr, settings); + ser_domain->deserializeTextJSON(column, istr, settings); } else { @@ -250,9 +250,9 @@ void IDataType::deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const void IDataType::serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (domain) + if (auto ser_domain = dynamic_cast(domain.get())) { - domain->serializeTextXML(column, row_num, ostr, settings); + ser_domain->serializeTextXML(column, row_num, ostr, settings); } else { @@ -260,13 +260,12 @@ void IDataType::serializeAsTextXML(const IColumn & column, size_t row_num, Write } } -void IDataType::setDomain(const IDataTypeDomain* const new_domain) const +void IDataType::appendDomain(DataTypeDomainPtr new_domain) const { - if (domain != nullptr) - { - throw Exception("Type " + getName() + " already has a domain.", ErrorCodes::LOGICAL_ERROR); - } - domain = new_domain; + if (domain == nullptr) + domain = std::move(new_domain); + else + domain->appendDomain(std::move(new_domain)); } } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index aa253fbdc08..a95402bf20a 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -13,6 +13,8 @@ class ReadBuffer; class WriteBuffer; class IDataTypeDomain; +using DataTypeDomainPtr = std::unique_ptr; + class IDataType; struct FormatSettings; @@ -459,18 +461,19 @@ public: private: friend class DataTypeFactory; - /** Sets domain on existing DataType, can be considered as second phase + /** Sets domain on existing DataType or append it to existing domain, can be considered as second phase * of construction explicitly done by DataTypeFactory. - * Will throw an exception if domain is already set. */ - void setDomain(const IDataTypeDomain* newDomain) const; + void appendDomain(DataTypeDomainPtr new_domain) const; private: /** This is mutable to allow setting domain on `const IDataType` post construction, * simplifying creation of domains for all types, without them even knowing * of domain existence. */ - mutable IDataTypeDomain const* domain; + mutable DataTypeDomainPtr domain; +public: + const IDataTypeDomain * getDomain() const { return domain.get(); } }; diff --git a/dbms/src/DataTypes/IDataTypeDomain.h b/dbms/src/DataTypes/IDataTypeDomain.h index ad38e88a213..1eed8afd808 100644 --- a/dbms/src/DataTypes/IDataTypeDomain.h +++ b/dbms/src/DataTypes/IDataTypeDomain.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include namespace DB { @@ -10,21 +12,42 @@ class WriteBuffer; struct FormatSettings; class IColumn; -/** Further refinment of the properties of data type. - * - * Contains methods for serialization/deserialization. - * Implementations of this interface represent a data type domain (example: IPv4) - * which is a refinement of the exsitgin type with a name and specific text - * representation. - * - * IDataTypeDomain is totally immutable object. You can always share them. +/** Allow to customize an existing data type and set a different name. Derived class IDataTypeDomainCustomSerialization allows + * further customization of serialization/deserialization methods. See use in IPv4 and IPv6 data type domains. + * + * IDataTypeDomain can be chained for further delegation (only for getName for the moment). */ class IDataTypeDomain { +private: + mutable DataTypeDomainPtr delegate; + public: virtual ~IDataTypeDomain() {} - virtual const char* getName() const = 0; + String getName() const { + if (delegate) + return delegate->getName(); + else + return doGetName(); + } + + void appendDomain(DataTypeDomainPtr delegate_) const { + if (delegate == nullptr) + delegate = std::move(delegate_); + else + delegate->appendDomain(std::move(delegate_)); + } + + const IDataTypeDomain * getDomain() const { return delegate.get(); } + +protected: + virtual String doGetName() const = 0; +}; + +class IDataTypeDomainCustomSerialization : public IDataTypeDomain { +public: + virtual ~IDataTypeDomainCustomSerialization() {} /** Text serialization for displaying on a terminal or saving into a text file, and the like. * Without escaping or quoting. diff --git a/dbms/tests/queries/0_stateless/00915_simple_aggregate_function.reference b/dbms/tests/queries/0_stateless/00915_simple_aggregate_function.reference new file mode 100644 index 00000000000..fbb3d60638e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00915_simple_aggregate_function.reference @@ -0,0 +1,43 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +SimpleAggregateFunction(sum, Float64) +0 0 +1 2 +2 4 +3 6 +4 8 +5 10 +6 12 +7 14 +8 16 +9 18 +0 0 +1 2 +2 4 +3 6 +4 8 +5 10 +6 12 +7 14 +8 16 +9 18 +1 1 2 2.2.2.2 +SimpleAggregateFunction(anyLast, Nullable(String)) SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) SimpleAggregateFunction(anyLast, IPv4) diff --git a/dbms/tests/queries/0_stateless/00915_simple_aggregate_function.sql b/dbms/tests/queries/0_stateless/00915_simple_aggregate_function.sql new file mode 100644 index 00000000000..f4f80033eaa --- /dev/null +++ b/dbms/tests/queries/0_stateless/00915_simple_aggregate_function.sql @@ -0,0 +1,27 @@ +-- basic test +drop table if exists test.simple; + +create table test.simple (id UInt64,val SimpleAggregateFunction(sum,Double)) engine=AggregatingMergeTree order by id; +insert into test.simple select number,number from system.numbers limit 10; + +select * from test.simple; +select * from test.simple final; +select toTypeName(val) from test.simple limit 1; + +-- merge +insert into test.simple select number,number from system.numbers limit 10; + +select * from test.simple final; + +optimize table test.simple final; +select * from test.simple; + +-- complex types +drop table if exists test.simple; + +create table test.simple (id UInt64,nullable_str SimpleAggregateFunction(anyLast,Nullable(String)),low_str SimpleAggregateFunction(anyLast,LowCardinality(Nullable(String))),ip SimpleAggregateFunction(anyLast,IPv4)) engine=AggregatingMergeTree order by id; +insert into test.simple values(1,'1','1','1.1.1.1'); +insert into test.simple values(1,null,'2','2.2.2.2'); + +select * from test.simple final; +select toTypeName(nullable_str),toTypeName(low_str),toTypeName(ip) from test.simple limit 1; From faa7d38cb5b8ba1d5fcb97c8442c02eaadc49bd7 Mon Sep 17 00:00:00 2001 From: bgranvea Date: Mon, 11 Mar 2019 09:24:52 +0100 Subject: [PATCH 02/58] fix for style --- .../AggregatingSortedBlockInputStream.cpp | 10 +++++--- .../DataTypeDomainSimpleAggregateFunction.cpp | 24 ++++++++++++------- .../DataTypeDomainSimpleAggregateFunction.h | 3 ++- dbms/src/DataTypes/IDataTypeDomain.h | 9 ++++--- 4 files changed, 31 insertions(+), 15 deletions(-) diff --git a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp index 34fb19b2688..1be85f7e1b8 100644 --- a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp @@ -42,11 +42,14 @@ AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream( continue; } - if (auto simple_aggr = findSimpleAggregateFunction(column.type)) { + if (auto simple_aggr = findSimpleAggregateFunction(column.type)) + { // simple aggregate function SimpleAggregateDescription desc{simple_aggr->getFunction(), i}; columns_to_simple_aggregate.emplace_back(std::move(desc)); - } else { + } + else + { // standard aggregate function column_numbers_to_aggregate.push_back(i); } @@ -99,7 +102,8 @@ void AggregatingSortedBlockInputStream::merge(MutableColumns & merged_columns, s key_differs = next_key != current_key; /// if there are enough rows accumulated and the last one is calculated completely - if (key_differs && merged_rows >= max_block_size) { + if (key_differs && merged_rows >= max_block_size) + { /// Write the simple aggregation result for the previous group. insertSimpleAggregationResult(merged_columns); return; diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp index 402ce86ad62..65bef22ce28 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp @@ -18,7 +18,8 @@ #include -namespace DB { +namespace DB +{ namespace ErrorCodes { @@ -33,13 +34,16 @@ const std::vector supported_functions = std::vector( {"any", "anyLast", "min", "max", "sum"}); -String DataTypeDomainSimpleAggregateFunction::doGetName() const { +String DataTypeDomainSimpleAggregateFunction::doGetName() const +{ std::stringstream stream; stream << "SimpleAggregateFunction(" << function->getName(); - if (!parameters.empty()) { + if (!parameters.empty()) + { stream << "("; - for (size_t i = 0; i < parameters.size(); ++i) { + for (size_t i = 0; i < parameters.size(); ++i) + { if (i) stream << ", "; stream << applyVisitor(DB::FieldVisitorToString(), parameters[i]); @@ -107,7 +111,8 @@ static std::pair create(const ASTPtr & arguments function = AggregateFunctionFactory::instance().get(function_name, argument_types, params_row); // check function - if (std::find(std::begin(supported_functions), std::end(supported_functions), function->getName()) == std::end(supported_functions)) { + if (std::find(std::begin(supported_functions), std::end(supported_functions), function->getName()) == std::end(supported_functions)) + { throw Exception("Unsupported aggregate function " + function->getName() + ", supported functions are " + boost::algorithm::join(supported_functions, ","), ErrorCodes::BAD_ARGUMENTS); } @@ -115,7 +120,8 @@ static std::pair create(const ASTPtr & arguments DataTypePtr storage_type = DataTypeFactory::instance().get(argument_types[0]->getName()); DataTypeDomainPtr domain = std::make_unique(storage_type, function, argument_types, params_row); - if (!function->getReturnType()->equals(*removeLowCardinality(storage_type))) { + if (!function->getReturnType()->equals(*removeLowCardinality(storage_type))) + { throw Exception("Incompatible data types between aggregate function '" + function->getName() + "' which returns " + function->getReturnType()->getName() + " and column storage type " + storage_type->getName(), ErrorCodes::BAD_ARGUMENTS); } @@ -123,7 +129,8 @@ static std::pair create(const ASTPtr & arguments return std::make_pair(storage_type, std::move(domain)); } -static const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(const IDataTypeDomain * domain) { +static const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(const IDataTypeDomain * domain) +{ if (domain == nullptr) return nullptr; @@ -136,7 +143,8 @@ static const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction return nullptr; } -const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(DataTypePtr dataType) { +const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(DataTypePtr dataType) +{ return findSimpleAggregateFunction(dataType->getDomain()); } diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h index 70e94b1a652..6573f1ae5d0 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h @@ -24,7 +24,8 @@ namespace DB * Technically, a standard IDataType is instanciated and a DataTypeDomainSimpleAggregateFunction is added as domain. */ -class DataTypeDomainSimpleAggregateFunction : public IDataTypeDomain { +class DataTypeDomainSimpleAggregateFunction : public IDataTypeDomain +{ private: const DataTypePtr storage_type; const AggregateFunctionPtr function; diff --git a/dbms/src/DataTypes/IDataTypeDomain.h b/dbms/src/DataTypes/IDataTypeDomain.h index 1eed8afd808..a840964d28a 100644 --- a/dbms/src/DataTypes/IDataTypeDomain.h +++ b/dbms/src/DataTypes/IDataTypeDomain.h @@ -25,14 +25,16 @@ private: public: virtual ~IDataTypeDomain() {} - String getName() const { + String getName() const + { if (delegate) return delegate->getName(); else return doGetName(); } - void appendDomain(DataTypeDomainPtr delegate_) const { + void appendDomain(DataTypeDomainPtr delegate_) const + { if (delegate == nullptr) delegate = std::move(delegate_); else @@ -45,7 +47,8 @@ protected: virtual String doGetName() const = 0; }; -class IDataTypeDomainCustomSerialization : public IDataTypeDomain { +class IDataTypeDomainCustomSerialization : public IDataTypeDomain +{ public: virtual ~IDataTypeDomainCustomSerialization() {} From ee5a88c15f979dd8782a2a82ce1ad5f434f57ad3 Mon Sep 17 00:00:00 2001 From: bgranvea Date: Mon, 11 Mar 2019 11:46:14 +0100 Subject: [PATCH 03/58] fix memory leak --- dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp | 2 +- dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp index 65bef22ce28..5daa886df43 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp @@ -118,7 +118,7 @@ static std::pair create(const ASTPtr & arguments } DataTypePtr storage_type = DataTypeFactory::instance().get(argument_types[0]->getName()); - DataTypeDomainPtr domain = std::make_unique(storage_type, function, argument_types, params_row); + DataTypeDomainPtr domain = std::make_unique(function, argument_types, params_row); if (!function->getReturnType()->equals(*removeLowCardinality(storage_type))) { diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h index 6573f1ae5d0..98989eeac11 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h @@ -27,14 +27,13 @@ namespace DB class DataTypeDomainSimpleAggregateFunction : public IDataTypeDomain { private: - const DataTypePtr storage_type; const AggregateFunctionPtr function; const DataTypes argument_types; const Array parameters; public: - DataTypeDomainSimpleAggregateFunction(const DataTypePtr storage_type_, const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_) - : storage_type(storage_type_), function(function_), argument_types(argument_types_), parameters(parameters_) {} + DataTypeDomainSimpleAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_) + : function(function_), argument_types(argument_types_), parameters(parameters_) {} const AggregateFunctionPtr getFunction() const { return function; } String doGetName() const override; From a3020f2d22d7bd8cf180afd2ae79ab250c2f9ccf Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 26 Mar 2019 00:51:54 +0300 Subject: [PATCH 04/58] Update DataTypeDomainSimpleAggregateFunction.cpp --- dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp index 5daa886df43..570310c1312 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp @@ -51,7 +51,7 @@ String DataTypeDomainSimpleAggregateFunction::doGetName() const stream << ")"; } - for (const auto &argument_type : argument_types) + for (const auto & argument_type : argument_types) stream << ", " << argument_type->getName(); stream << ")"; From c1ea15f0bb04c5b722947d6c688e3a0871d07493 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 26 Mar 2019 00:54:19 +0300 Subject: [PATCH 05/58] Update DataTypeDomainSimpleAggregateFunction.cpp --- .../DataTypes/DataTypeDomainSimpleAggregateFunction.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp index 570310c1312..ee524f76ec9 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp @@ -70,18 +70,18 @@ static std::pair create(const ASTPtr & arguments throw Exception("Data type SimpleAggregateFunction requires parameters: " "name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (const ASTFunction * parametric = typeid_cast(arguments->children[0].get())) + if (const ASTFunction * parametric = arguments->children[0]->as()) { if (parametric->parameters) throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR); function_name = parametric->name; - const ASTs & parameters = typeid_cast(*parametric->arguments).children; + const ASTs & parameters = parametric->arguments->as().children; params_row.resize(parameters.size()); for (size_t i = 0; i < parameters.size(); ++i) { - const ASTLiteral * lit = typeid_cast(parameters[i].get()); + const ASTLiteral * lit = parameters[i]->as(); if (!lit) throw Exception("Parameters to aggregate functions must be literals", ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS); @@ -93,7 +93,7 @@ static std::pair create(const ASTPtr & arguments { function_name = *opt_name; } - else if (typeid_cast(arguments->children[0].get())) + else if (arguments->children[0]->as()) { throw Exception("Aggregate function name for data type SimpleAggregateFunction must be passed as identifier (without quotes) or function", ErrorCodes::BAD_ARGUMENTS); From e4b93f092b6328a097bda6fba9a0d9805d14d0a7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 26 Mar 2019 00:57:34 +0300 Subject: [PATCH 06/58] Update DataTypeDomainSimpleAggregateFunction.cpp --- dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp index ee524f76ec9..dfcd7604aeb 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp @@ -30,8 +30,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -const std::vector supported_functions = std::vector( - {"any", "anyLast", "min", "max", "sum"}); +static const std::initializer_list supported_functions = std::vector( + {"any", "anyLast", "min", "max", "sum"}); String DataTypeDomainSimpleAggregateFunction::doGetName() const From caa096a3d0e7340d5cf0d4cecf33f552c06bbd9d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 26 Mar 2019 00:57:58 +0300 Subject: [PATCH 07/58] Update DataTypeDomainSimpleAggregateFunction.cpp --- dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp index dfcd7604aeb..82e3b873f5e 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp @@ -30,8 +30,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static const std::initializer_list supported_functions = std::vector( - {"any", "anyLast", "min", "max", "sum"}); +static const std::vector supported_functions{"any", "anyLast", "min", "max", "sum"}; String DataTypeDomainSimpleAggregateFunction::doGetName() const From 42b07c5ee980d51b9780aadd30821a174ec9d990 Mon Sep 17 00:00:00 2001 From: bgranvea Date: Fri, 29 Mar 2019 21:04:04 +0100 Subject: [PATCH 08/58] refactor to avoid dynamic_cast in data type serialization --- .../AggregatingSortedBlockInputStream.cpp | 6 +- .../{IDataTypeDomain.h => DataTypeCustom.h} | 67 ++++++++++--------- ...IPv6.cpp => DataTypeCustomIPv4AndIPv6.cpp} | 40 +++++------ ...DataTypeCustomSimpleAggregateFunction.cpp} | 33 ++------- ...> DataTypeCustomSimpleAggregateFunction.h} | 13 ++-- ...DataTypeCustomSimpleTextSerialization.cpp} | 26 +++---- ...> DataTypeCustomSimpleTextSerialization.h} | 8 +-- dbms/src/DataTypes/DataTypeFactory.cpp | 10 +-- dbms/src/DataTypes/DataTypeFactory.h | 15 ++--- dbms/src/DataTypes/IDataType.cpp | 63 ++++++++--------- dbms/src/DataTypes/IDataType.h | 20 +++--- 11 files changed, 139 insertions(+), 162 deletions(-) rename dbms/src/DataTypes/{IDataTypeDomain.h => DataTypeCustom.h} (62%) rename dbms/src/DataTypes/{DataTypeDomainIPv4AndIPv6.cpp => DataTypeCustomIPv4AndIPv6.cpp} (62%) rename dbms/src/DataTypes/{DataTypeDomainSimpleAggregateFunction.cpp => DataTypeCustomSimpleAggregateFunction.cpp} (81%) rename dbms/src/DataTypes/{DataTypeDomainSimpleAggregateFunction.h => DataTypeCustomSimpleAggregateFunction.h} (67%) rename dbms/src/DataTypes/{DataTypeDomainWithSimpleSerialization.cpp => DataTypeCustomSimpleTextSerialization.cpp} (73%) rename dbms/src/DataTypes/{DataTypeDomainWithSimpleSerialization.h => DataTypeCustomSimpleTextSerialization.h} (89%) diff --git a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp index 1be85f7e1b8..f093e47e640 100644 --- a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include namespace DB @@ -24,7 +24,7 @@ AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream( ColumnWithTypeAndName & column = header.safeGetByPosition(i); /// We leave only states of aggregate functions. - if (!dynamic_cast(column.type.get()) && !findSimpleAggregateFunction(column.type)) + if (!dynamic_cast(column.type.get()) && !dynamic_cast(column.type->getCustomName())) { column_numbers_not_to_aggregate.push_back(i); continue; @@ -42,7 +42,7 @@ AggregatingSortedBlockInputStream::AggregatingSortedBlockInputStream( continue; } - if (auto simple_aggr = findSimpleAggregateFunction(column.type)) + if (auto simple_aggr = dynamic_cast(column.type->getCustomName())) { // simple aggregate function SimpleAggregateDescription desc{simple_aggr->getFunction(), i}; diff --git a/dbms/src/DataTypes/IDataTypeDomain.h b/dbms/src/DataTypes/DataTypeCustom.h similarity index 62% rename from dbms/src/DataTypes/IDataTypeDomain.h rename to dbms/src/DataTypes/DataTypeCustom.h index a840964d28a..93882361e20 100644 --- a/dbms/src/DataTypes/IDataTypeDomain.h +++ b/dbms/src/DataTypes/DataTypeCustom.h @@ -1,8 +1,8 @@ #pragma once +#include #include #include -#include namespace DB { @@ -12,45 +12,21 @@ class WriteBuffer; struct FormatSettings; class IColumn; -/** Allow to customize an existing data type and set a different name. Derived class IDataTypeDomainCustomSerialization allows - * further customization of serialization/deserialization methods. See use in IPv4 and IPv6 data type domains. - * - * IDataTypeDomain can be chained for further delegation (only for getName for the moment). +/** Allow to customize an existing data type and set a different name and/or text serialization/deserialization methods. + * See use in IPv4 and IPv6 data types, and also in SimpleAggregateFunction. */ -class IDataTypeDomain +class IDataTypeCustomName { -private: - mutable DataTypeDomainPtr delegate; - public: - virtual ~IDataTypeDomain() {} + virtual ~IDataTypeCustomName() {} - String getName() const - { - if (delegate) - return delegate->getName(); - else - return doGetName(); - } - - void appendDomain(DataTypeDomainPtr delegate_) const - { - if (delegate == nullptr) - delegate = std::move(delegate_); - else - delegate->appendDomain(std::move(delegate_)); - } - - const IDataTypeDomain * getDomain() const { return delegate.get(); } - -protected: - virtual String doGetName() const = 0; + virtual String getName() const = 0; }; -class IDataTypeDomainCustomSerialization : public IDataTypeDomain +class IDataTypeCustomTextSerialization { public: - virtual ~IDataTypeDomainCustomSerialization() {} + virtual ~IDataTypeCustomTextSerialization() {} /** Text serialization for displaying on a terminal or saving into a text file, and the like. * Without escaping or quoting. @@ -82,4 +58,31 @@ public: virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const = 0; }; +using DataTypeCustomNamePtr = std::unique_ptr; +using DataTypeCustomTextSerializationPtr = std::unique_ptr; + +/** Describe a data type customization + */ +struct DataTypeCustomDesc +{ + DataTypeCustomNamePtr name; + DataTypeCustomTextSerializationPtr text_serialization; + + DataTypeCustomDesc(DataTypeCustomNamePtr name_, DataTypeCustomTextSerializationPtr text_serialization_) + : name(std::move(name_)), text_serialization(std::move(text_serialization_)) {} +}; + +using DataTypeCustomDescPtr = std::unique_ptr; + +/** A simple implementation of IDataTypeCustomName + */ +class DataTypeCustomFixedName : public IDataTypeCustomName +{ +private: + String name; +public: + DataTypeCustomFixedName(String name_) : name(name_) {} + String getName() const override { return name; } +}; + } // namespace DB diff --git a/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp b/dbms/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp similarity index 62% rename from dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp rename to dbms/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp index f57a6167d3d..8d12a9847db 100644 --- a/dbms/src/DataTypes/DataTypeDomainIPv4AndIPv6.cpp +++ b/dbms/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp @@ -1,9 +1,9 @@ #include #include #include -#include +#include #include -#include +#include #include #include @@ -20,20 +20,15 @@ namespace ErrorCodes namespace { -class DataTypeDomainIPv4 : public DataTypeDomainWithSimpleSerialization +class DataTypeCustomIPv4Serialization : public DataTypeCustomSimpleTextSerialization { public: - String doGetName() const override - { - return "IPv4"; - } - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override { const auto col = checkAndGetColumn(&column); if (!col) { - throw Exception(getName() + " domain can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("IPv4 type can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -48,7 +43,7 @@ public: ColumnUInt32 * col = typeid_cast(&column); if (!col) { - throw Exception(getName() + " domain can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("IPv4 type can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -63,20 +58,16 @@ public: } }; -class DataTypeDomainIPv6 : public DataTypeDomainWithSimpleSerialization +class DataTypeCustomIPv6Serialization : public DataTypeCustomSimpleTextSerialization { public: - String doGetName() const override - { - return "IPv6"; - } void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override { const auto col = checkAndGetColumn(&column); if (!col) { - throw Exception(getName() + " domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("IPv6 type domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -91,7 +82,7 @@ public: ColumnFixedString * col = typeid_cast(&column); if (!col) { - throw Exception(getName() + " domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception("IPv6 type domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'}; @@ -100,7 +91,7 @@ public: std::string ipv6_value(IPV6_BINARY_LENGTH, '\0'); if (!parseIPv6(buffer, reinterpret_cast(ipv6_value.data()))) { - throw Exception("Invalid " + getName() + " value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + throw Exception("Invalid IPv6 value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); } col->insertString(ipv6_value); @@ -111,8 +102,17 @@ public: void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory) { - factory.registerDataTypeDomain("IPv4", [] { return std::make_pair(DataTypeFactory::instance().get("UInt32"), std::make_unique()); }); - factory.registerDataTypeDomain("IPv6", [] { return std::make_pair(DataTypeFactory::instance().get("FixedString(16)"), std::make_unique()); }); + factory.registerSimpleDataTypeCustom("IPv4", [] + { + return std::make_pair(DataTypeFactory::instance().get("UInt32"), + std::make_unique(std::make_unique("IPv4"), std::make_unique())); + }); + + factory.registerSimpleDataTypeCustom("IPv6", [] + { + return std::make_pair(DataTypeFactory::instance().get("FixedString(16)"), + std::make_unique(std::make_unique("IPv6"), std::make_unique())); + }); } } // namespace DB diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp similarity index 81% rename from dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp rename to dbms/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index 82e3b873f5e..2cb0f87facd 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -5,7 +5,7 @@ #include -#include +#include #include #include #include @@ -33,7 +33,7 @@ namespace ErrorCodes static const std::vector supported_functions{"any", "anyLast", "min", "max", "sum"}; -String DataTypeDomainSimpleAggregateFunction::doGetName() const +String DataTypeCustomSimpleAggregateFunction::getName() const { std::stringstream stream; stream << "SimpleAggregateFunction(" << function->getName(); @@ -58,7 +58,7 @@ String DataTypeDomainSimpleAggregateFunction::doGetName() const } -static std::pair create(const ASTPtr & arguments) +static std::pair create(const ASTPtr & arguments) { String function_name; AggregateFunctionPtr function; @@ -117,7 +117,6 @@ static std::pair create(const ASTPtr & arguments } DataTypePtr storage_type = DataTypeFactory::instance().get(argument_types[0]->getName()); - DataTypeDomainPtr domain = std::make_unique(function, argument_types, params_row); if (!function->getReturnType()->equals(*removeLowCardinality(storage_type))) { @@ -125,32 +124,14 @@ static std::pair create(const ASTPtr & arguments ErrorCodes::BAD_ARGUMENTS); } - return std::make_pair(storage_type, std::move(domain)); + DataTypeCustomNamePtr custom_name = std::make_unique(function, argument_types, params_row); + + return std::make_pair(storage_type, std::make_unique(std::move(custom_name), nullptr)); } -static const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(const IDataTypeDomain * domain) -{ - if (domain == nullptr) - return nullptr; - - if (auto simple_aggr = dynamic_cast(domain)) - return simple_aggr; - - if (domain->getDomain() != nullptr) - return findSimpleAggregateFunction(domain->getDomain()); - - return nullptr; -} - -const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(DataTypePtr dataType) -{ - return findSimpleAggregateFunction(dataType->getDomain()); -} - - void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory) { - factory.registerDataTypeDomain("SimpleAggregateFunction", create); + factory.registerDataTypeCustom("SimpleAggregateFunction", create); } } diff --git a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h b/dbms/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h similarity index 67% rename from dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h rename to dbms/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h index 98989eeac11..3e82b546903 100644 --- a/dbms/src/DataTypes/DataTypeDomainSimpleAggregateFunction.h +++ b/dbms/src/DataTypes/DataTypeCustomSimpleAggregateFunction.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -21,10 +21,10 @@ namespace DB * SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) * SimpleAggregateFunction(anyLast, IPv4) * - * Technically, a standard IDataType is instanciated and a DataTypeDomainSimpleAggregateFunction is added as domain. + * Technically, a standard IDataType is instanciated and customized with IDataTypeCustomName and DataTypeCustomDesc. */ -class DataTypeDomainSimpleAggregateFunction : public IDataTypeDomain +class DataTypeCustomSimpleAggregateFunction : public IDataTypeCustomName { private: const AggregateFunctionPtr function; @@ -32,14 +32,11 @@ private: const Array parameters; public: - DataTypeDomainSimpleAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_) + DataTypeCustomSimpleAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_) : function(function_), argument_types(argument_types_), parameters(parameters_) {} const AggregateFunctionPtr getFunction() const { return function; } - String doGetName() const override; + String getName() const override; }; -/// recursively follow data type domain to find a DataTypeDomainSimpleAggregateFunction -const DataTypeDomainSimpleAggregateFunction * findSimpleAggregateFunction(DataTypePtr dataType); - } diff --git a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.cpp b/dbms/src/DataTypes/DataTypeCustomSimpleTextSerialization.cpp similarity index 73% rename from dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.cpp rename to dbms/src/DataTypes/DataTypeCustomSimpleTextSerialization.cpp index 12b1837be1f..44ce27a6e88 100644 --- a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.cpp +++ b/dbms/src/DataTypes/DataTypeCustomSimpleTextSerialization.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -9,7 +9,7 @@ namespace { using namespace DB; -static String serializeToString(const DataTypeDomainWithSimpleSerialization & domain, const IColumn & column, size_t row_num, const FormatSettings & settings) +static String serializeToString(const DataTypeCustomSimpleTextSerialization & domain, const IColumn & column, size_t row_num, const FormatSettings & settings) { WriteBufferFromOwnString buffer; domain.serializeText(column, row_num, buffer, settings); @@ -17,7 +17,7 @@ static String serializeToString(const DataTypeDomainWithSimpleSerialization & do return buffer.str(); } -static void deserializeFromString(const DataTypeDomainWithSimpleSerialization & domain, IColumn & column, const String & s, const FormatSettings & settings) +static void deserializeFromString(const DataTypeCustomSimpleTextSerialization & domain, IColumn & column, const String & s, const FormatSettings & settings) { ReadBufferFromString istr(s); domain.deserializeText(column, istr, settings); @@ -28,59 +28,59 @@ static void deserializeFromString(const DataTypeDomainWithSimpleSerialization & namespace DB { -DataTypeDomainWithSimpleSerialization::~DataTypeDomainWithSimpleSerialization() +DataTypeCustomSimpleTextSerialization::~DataTypeCustomSimpleTextSerialization() { } -void DataTypeDomainWithSimpleSerialization::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeEscapedString(serializeToString(*this, column, row_num, settings), ostr); } -void DataTypeDomainWithSimpleSerialization::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { String str; readEscapedString(str, istr); deserializeFromString(*this, column, str, settings); } -void DataTypeDomainWithSimpleSerialization::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeQuotedString(serializeToString(*this, column, row_num, settings), ostr); } -void DataTypeDomainWithSimpleSerialization::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { String str; readQuotedString(str, istr); deserializeFromString(*this, column, str, settings); } -void DataTypeDomainWithSimpleSerialization::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeCSVString(serializeToString(*this, column, row_num, settings), ostr); } -void DataTypeDomainWithSimpleSerialization::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { String str; readCSVString(str, istr, settings.csv); deserializeFromString(*this, column, str, settings); } -void DataTypeDomainWithSimpleSerialization::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeJSONString(serializeToString(*this, column, row_num, settings), ostr, settings); } -void DataTypeDomainWithSimpleSerialization::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { String str; readJSONString(str, istr); deserializeFromString(*this, column, str, settings); } -void DataTypeDomainWithSimpleSerialization::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeCustomSimpleTextSerialization::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeXMLString(serializeToString(*this, column, row_num, settings), ostr); } diff --git a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h b/dbms/src/DataTypes/DataTypeCustomSimpleTextSerialization.h similarity index 89% rename from dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h rename to dbms/src/DataTypes/DataTypeCustomSimpleTextSerialization.h index 3ccb4091636..fb9be86d95f 100644 --- a/dbms/src/DataTypes/DataTypeDomainWithSimpleSerialization.h +++ b/dbms/src/DataTypes/DataTypeCustomSimpleTextSerialization.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { @@ -10,12 +10,12 @@ class WriteBuffer; struct FormatSettings; class IColumn; -/** Simple DataTypeDomain that uses serializeText/deserializeText +/** Simple IDataTypeCustomTextSerialization that uses serializeText/deserializeText * for all serialization and deserialization. */ -class DataTypeDomainWithSimpleSerialization : public IDataTypeDomainCustomSerialization +class DataTypeCustomSimpleTextSerialization : public IDataTypeCustomTextSerialization { public: - virtual ~DataTypeDomainWithSimpleSerialization() override; + virtual ~DataTypeCustomSimpleTextSerialization() override; // Methods that subclasses must override in order to get full serialization/deserialization support. virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override = 0; diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index a405075e884..8c4c899516a 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -115,20 +115,20 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator }, case_sensitiveness); } -void DataTypeFactory::registerDataTypeDomain(const String & family_name, CreatorWithDomain creator, CaseSensitiveness case_sensitiveness) +void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness) { registerDataType(family_name, [creator](const ASTPtr & ast) { auto res = creator(ast); - res.first->appendDomain(std::move(res.second)); + res.first->setCustomization(std::move(res.second)); return res.first; }, case_sensitiveness); } -void DataTypeFactory::registerDataTypeDomain(const String & name, SimpleCreatorWithDomain creator, CaseSensitiveness case_sensitiveness) +void DataTypeFactory::registerSimpleDataTypeCustom(const String &name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness) { - registerDataTypeDomain(name, [creator](const ASTPtr & /*ast*/) + registerDataTypeCustom(name, [creator](const ASTPtr & /*ast*/) { return creator(); }, case_sensitiveness); diff --git a/dbms/src/DataTypes/DataTypeFactory.h b/dbms/src/DataTypes/DataTypeFactory.h index e4a82b342d1..a6c714c1a0e 100644 --- a/dbms/src/DataTypes/DataTypeFactory.h +++ b/dbms/src/DataTypes/DataTypeFactory.h @@ -17,9 +17,6 @@ namespace DB class IDataType; using DataTypePtr = std::shared_ptr; -class IDataTypeDomain; -using DataTypeDomainPtr = std::unique_ptr; - /** Creates a data type by name of data type family and parameters. */ @@ -28,8 +25,8 @@ class DataTypeFactory final : public ext::singleton, public IFa private: using SimpleCreator = std::function; using DataTypesDictionary = std::unordered_map; - using CreatorWithDomain = std::function(const ASTPtr & parameters)>; - using SimpleCreatorWithDomain = std::function()>; + using CreatorWithCustom = std::function(const ASTPtr & parameters)>; + using SimpleCreatorWithCustom = std::function()>; public: DataTypePtr get(const String & full_name) const; @@ -42,11 +39,11 @@ public: /// Register a simple data type, that have no parameters. void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - /// Register a type family with a dynamic domain - void registerDataTypeDomain(const String & family_name, CreatorWithDomain creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + /// Register a customized type family + void registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - /// Register a simple data type domain - void registerDataTypeDomain(const String & name, SimpleCreatorWithDomain creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + /// Register a simple customized data type + void registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive); private: const Creator& findCreatorByName(const String & family_name) const; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index 0270f1d7923..09c080f56cc 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include @@ -23,8 +23,7 @@ namespace ErrorCodes extern const int DATA_TYPE_CANNOT_BE_PROMOTED; } -IDataType::IDataType() - : domain(nullptr) +IDataType::IDataType() : custom_name(nullptr), custom_text_serialization(nullptr) { } @@ -34,9 +33,9 @@ IDataType::~IDataType() String IDataType::getName() const { - if (domain) + if (custom_name) { - return domain->getName(); + return custom_name->getName(); } else { @@ -142,9 +141,9 @@ void IDataType::insertDefaultInto(IColumn & column) const void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->serializeTextEscaped(column, row_num, ostr, settings); + custom_text_serialization->serializeTextEscaped(column, row_num, ostr, settings); } else { @@ -154,9 +153,9 @@ void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, W void IDataType::deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->deserializeTextEscaped(column, istr, settings); + custom_text_serialization->deserializeTextEscaped(column, istr, settings); } else { @@ -166,9 +165,9 @@ void IDataType::deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, co void IDataType::serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->serializeTextQuoted(column, row_num, ostr, settings); + custom_text_serialization->serializeTextQuoted(column, row_num, ostr, settings); } else { @@ -178,9 +177,9 @@ void IDataType::serializeAsTextQuoted(const IColumn & column, size_t row_num, Wr void IDataType::deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->deserializeTextQuoted(column, istr, settings); + custom_text_serialization->deserializeTextQuoted(column, istr, settings); } else { @@ -190,9 +189,9 @@ void IDataType::deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, con void IDataType::serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) - { - ser_domain->serializeTextCSV(column, row_num, ostr, settings); + if (custom_text_serialization) + { + custom_text_serialization->serializeTextCSV(column, row_num, ostr, settings); } else { @@ -202,9 +201,9 @@ void IDataType::serializeAsTextCSV(const IColumn & column, size_t row_num, Write void IDataType::deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->deserializeTextCSV(column, istr, settings); + custom_text_serialization->deserializeTextCSV(column, istr, settings); } else { @@ -214,9 +213,9 @@ void IDataType::deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const void IDataType::serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->serializeText(column, row_num, ostr, settings); + custom_text_serialization->serializeText(column, row_num, ostr, settings); } else { @@ -226,9 +225,9 @@ void IDataType::serializeAsText(const IColumn & column, size_t row_num, WriteBuf void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->serializeTextJSON(column, row_num, ostr, settings); + custom_text_serialization->serializeTextJSON(column, row_num, ostr, settings); } else { @@ -238,9 +237,9 @@ void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, Writ void IDataType::deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->deserializeTextJSON(column, istr, settings); + custom_text_serialization->deserializeTextJSON(column, istr, settings); } else { @@ -250,9 +249,9 @@ void IDataType::deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const void IDataType::serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - if (auto ser_domain = dynamic_cast(domain.get())) + if (custom_text_serialization) { - ser_domain->serializeTextXML(column, row_num, ostr, settings); + custom_text_serialization->serializeTextXML(column, row_num, ostr, settings); } else { @@ -260,12 +259,14 @@ void IDataType::serializeAsTextXML(const IColumn & column, size_t row_num, Write } } -void IDataType::appendDomain(DataTypeDomainPtr new_domain) const +void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const { - if (domain == nullptr) - domain = std::move(new_domain); - else - domain->appendDomain(std::move(new_domain)); + /// replace only if not null + if (custom_desc_->name) + custom_name = std::move(custom_desc_->name); + + if (custom_desc_->text_serialization) + custom_text_serialization = std::move(custom_desc_->text_serialization); } } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index a95402bf20a..6446f8ada43 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -12,9 +13,6 @@ namespace DB class ReadBuffer; class WriteBuffer; -class IDataTypeDomain; -using DataTypeDomainPtr = std::unique_ptr; - class IDataType; struct FormatSettings; @@ -461,19 +459,19 @@ public: private: friend class DataTypeFactory; - /** Sets domain on existing DataType or append it to existing domain, can be considered as second phase - * of construction explicitly done by DataTypeFactory. + /** Customize this DataType */ - void appendDomain(DataTypeDomainPtr new_domain) const; + void setCustomization(DataTypeCustomDescPtr custom_desc_) const; private: - /** This is mutable to allow setting domain on `const IDataType` post construction, - * simplifying creation of domains for all types, without them even knowing - * of domain existence. + /** This is mutable to allow setting custom name and serialization on `const IDataType` post construction. */ - mutable DataTypeDomainPtr domain; + mutable DataTypeCustomNamePtr custom_name; + mutable DataTypeCustomTextSerializationPtr custom_text_serialization; + public: - const IDataTypeDomain * getDomain() const { return domain.get(); } + const IDataTypeCustomName * getCustomName() const { return custom_name.get(); } + const IDataTypeCustomTextSerialization * getCustomTextSerialization() const { return custom_text_serialization.get(); } }; From 8668d22f4b869e4d916d44b1b075804686041e5a Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 22 Apr 2019 22:03:11 +0300 Subject: [PATCH 09/58] Freebsd fixes --- cmake/find_execinfo.cmake | 7 +++---- dbms/CMakeLists.txt | 3 +-- dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt | 2 +- dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt | 2 +- dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt | 2 +- dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt | 2 +- dbms/tests/clickhouse-test | 8 ++++---- libs/libcommon/CMakeLists.txt | 6 +++++- libs/libdaemon/CMakeLists.txt | 2 +- 9 files changed, 18 insertions(+), 16 deletions(-) diff --git a/cmake/find_execinfo.cmake b/cmake/find_execinfo.cmake index 650d279983c..85cc5cf951a 100644 --- a/cmake/find_execinfo.cmake +++ b/cmake/find_execinfo.cmake @@ -1,9 +1,8 @@ if (OS_FREEBSD) find_library (EXECINFO_LIBRARY execinfo) find_library (ELF_LIBRARY elf) - message (STATUS "Using execinfo: ${EXECINFO_LIBRARY}") - message (STATUS "Using elf: ${ELF_LIBRARY}") + set (EXECINFO_LIBRARIES ${EXECINFO_LIBRARY} ${ELF_LIBRARY}) + message (STATUS "Using execinfo: ${EXECINFO_LIBRARIES}") else () - set (EXECINFO_LIBRARY "") - set (ELF_LIBRARY "") + set (EXECINFO_LIBRARIES "") endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 2c9bfa48605..63e97f4e061 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -194,8 +194,7 @@ target_link_libraries (clickhouse_common_io ${CITYHASH_LIBRARIES} PRIVATE ${ZLIB_LIBRARIES} - ${EXECINFO_LIBRARY} - ${ELF_LIBRARY} + ${EXECINFO_LIBRARIES} PUBLIC ${Boost_SYSTEM_LIBRARY} PRIVATE diff --git a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt index 7fe0cd92ef7..e0171630bf2 100644 --- a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt @@ -46,7 +46,7 @@ LLVMSupport #PollyISL #PollyPPCG -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt index b96bdb0647a..bac622ab09e 100644 --- a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt @@ -46,7 +46,7 @@ ${REQUIRED_LLVM_LIBRARIES} #PollyISL #PollyPPCG -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt index 8b6ba6be994..35e23cc6b46 100644 --- a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt @@ -42,7 +42,7 @@ lldCore ${REQUIRED_LLVM_LIBRARIES} -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt index d0ccc8d672c..d03052ffc28 100644 --- a/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-7.0.0bundled/CMakeLists.txt @@ -42,7 +42,7 @@ lldCore ${REQUIRED_LLVM_LIBRARIES} -PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads +PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARIES} Threads::Threads ${MALLOC_LIBRARIES} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES} diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 6b11331f451..7a99be3ea48 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -19,7 +19,7 @@ from time import sleep from errno import ESRCH import termcolor from random import random -import commands +import subprocess MESSAGES_TO_RETRY = [ @@ -145,7 +145,7 @@ def main(args): os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level) if args.zookeeper is None: - code, out = commands.getstatusoutput(args.extract_from_config +" --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') + code, out = subprocess.getstatusoutput(args.extract_from_config +" --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') try: if int(out) > 0: args.zookeeper = True @@ -155,7 +155,7 @@ def main(args): args.zookeeper = False if args.shard is None: - code, out = commands.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"') + code, out = subprocess.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"') if out: args.shard = True else: @@ -166,7 +166,7 @@ def main(args): failures_total = 0 clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database) + clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + args.database).encode()) if args.database != "test": clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test") diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 999290996a9..02199c21a4f 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -91,9 +91,13 @@ target_include_directories (common BEFORE PRIVATE ${CCTZ_INCLUDE_DIR}) target_include_directories (common PUBLIC ${COMMON_INCLUDE_DIR}) if (NOT USE_INTERNAL_BOOST_LIBRARY) - target_include_directories (common BEFORE PUBLIC ${Boost_INCLUDE_DIRS}) + target_include_directories (common SYSTEM BEFORE PUBLIC ${Boost_INCLUDE_DIRS}) endif () +if(NOT USE_INTERNAL_POCO_LIBRARY) + target_include_directories (common SYSTEM BEFORE PUBLIC ${Poco_Foundation_INCLUDE_DIR}) +endif() + target_link_libraries (common PUBLIC ${Poco_Foundation_LIBRARY} diff --git a/libs/libdaemon/CMakeLists.txt b/libs/libdaemon/CMakeLists.txt index eb73f1cda33..181030009b0 100644 --- a/libs/libdaemon/CMakeLists.txt +++ b/libs/libdaemon/CMakeLists.txt @@ -22,4 +22,4 @@ endif () target_include_directories (daemon PUBLIC include) -target_link_libraries (daemon PRIVATE clickhouse_common_io clickhouse_common_config common ${Poco_Net_LIBRARY} ${Poco_Util_LIBRARY} ${EXECINFO_LIBRARY} ${ELF_LIBRARY}) +target_link_libraries (daemon PRIVATE clickhouse_common_io clickhouse_common_config common ${Poco_Net_LIBRARY} ${Poco_Util_LIBRARY} ${EXECINFO_LIBRARIES}) From 37df95a437f70b9b2b7f3bb83568013b2230cf68 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 22 Apr 2019 23:09:14 +0300 Subject: [PATCH 10/58] fix --- libs/libmysqlxx/cmake/find_mysqlclient.cmake | 22 ++++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/libs/libmysqlxx/cmake/find_mysqlclient.cmake b/libs/libmysqlxx/cmake/find_mysqlclient.cmake index 71cb2bfeb1d..98b42a0a9b4 100644 --- a/libs/libmysqlxx/cmake/find_mysqlclient.cmake +++ b/libs/libmysqlxx/cmake/find_mysqlclient.cmake @@ -1,17 +1,17 @@ -option (ENABLE_MYSQL "Enable MySQL" ${OS_LINUX}) -if (OS_LINUX) - option (USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) -else () - option (USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" OFF) -endif () +option(ENABLE_MYSQL "Enable MySQL" 1) +if(ENABLE_MYSQL) + if(OS_LINUX) + option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" ${NOT_UNBUNDLED}) + else() + option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient library instead of bundled" OFF) + endif() -if (USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README.md") - message (WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_MYSQL_LIBRARY 0) -endif () + if(USE_INTERNAL_MYSQL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README.md") + message(WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") + set(USE_INTERNAL_MYSQL_LIBRARY 0) + endif() -if (ENABLE_MYSQL) if (USE_INTERNAL_MYSQL_LIBRARY) set (MYSQLCLIENT_LIBRARIES mysqlclient) set (USE_MYSQL 1) From e899d7b71f4c898dcc9b804db35ad37c84bbc261 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 23 Apr 2019 15:50:56 +0300 Subject: [PATCH 11/58] fix --- dbms/tests/clickhouse-test | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index f1819d12d9a..7612c415373 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -116,6 +116,7 @@ exit_code = 0 #def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total): def run_tests_array(all_tests_with_params): all_tests, suite, suite_dir, suite_tmp_dir, run_total = all_tests_with_params + global exit_code global SERVER_DIED OP_SQUARE_BRACKET = colored("[", args, attrs=['bold']) @@ -393,7 +394,7 @@ def main(args): suite = suite_re_obj.group(1) if os.path.isdir(suite_dir): - if 'stateful' in suite and not is_data_present(): + if 'stateful' in suite and not args.no_stateful and not is_data_present(): print("Won't run stateful tests because test data wasn't loaded.") continue if 'stateless' in suite and args.no_stateless: From 38f5ebf000a50e7e08ffadaff7465cd053b21d77 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 23 Apr 2019 16:58:43 +0300 Subject: [PATCH 12/58] Fix segfault in performance-test when no options specified --- dbms/programs/performance-test/PerformanceTestSuite.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/programs/performance-test/PerformanceTestSuite.cpp b/dbms/programs/performance-test/PerformanceTestSuite.cpp index 7a373d7bfba..9daf1b86ce7 100644 --- a/dbms/programs/performance-test/PerformanceTestSuite.cpp +++ b/dbms/programs/performance-test/PerformanceTestSuite.cpp @@ -298,6 +298,8 @@ std::unordered_map> getTestQueryIndexes(co { std::unordered_map> result; const auto & options = parsed_opts.options; + if (options.empty()) + return result; for (size_t i = 0; i < options.size() - 1; ++i) { const auto & opt = options[i]; From acbeedad3856447262c8ce7f21b9aa9a61e56cb9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=90=B4=E5=81=A5?= Date: Tue, 30 Apr 2019 11:50:30 +0800 Subject: [PATCH 13/58] Fix null problem in FunctionIn --- dbms/src/Functions/in.cpp | 5 ----- .../queries/0_stateless/00939_test_null_in.reference | 9 +++++++++ dbms/tests/queries/0_stateless/00939_test_null_in.sql | 11 +++++++++++ 3 files changed, 20 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00939_test_null_in.reference create mode 100644 dbms/tests/queries/0_stateless/00939_test_null_in.sql diff --git a/dbms/src/Functions/in.cpp b/dbms/src/Functions/in.cpp index 9267a42082c..c7c6a1f6c15 100644 --- a/dbms/src/Functions/in.cpp +++ b/dbms/src/Functions/in.cpp @@ -73,11 +73,6 @@ public: return std::make_shared(); } - bool useDefaultImplementationForNulls() const override - { - return false; - } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { /// Second argument must be ColumnSet. diff --git a/dbms/tests/queries/0_stateless/00939_test_null_in.reference b/dbms/tests/queries/0_stateless/00939_test_null_in.reference new file mode 100644 index 00000000000..4e012f99783 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00939_test_null_in.reference @@ -0,0 +1,9 @@ +1 +\N +\N +1 +\N +\N +1 +\N +\N diff --git a/dbms/tests/queries/0_stateless/00939_test_null_in.sql b/dbms/tests/queries/0_stateless/00939_test_null_in.sql new file mode 100644 index 00000000000..973ba13be5e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00939_test_null_in.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS test.nullt; + +CREATE TABLE test.nullt (c1 Nullable(UInt32), c2 Nullable(String))ENGINE = Log; +INSERT INTO test.nullt VALUES (1, 'abc'), (2, NULL), (NULL, NULL); + +SELECT c2 = ('abc') FROM test.nullt; +SELECT c2 IN ('abc') FROM test.nullt; + +SELECT c2 IN ('abc', NULL) FROM test.nullt; + +DROP TABLE IF EXISTS test.nullt; From 35471f423410d5466ece25470dde4ab1ac604c26 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 30 Apr 2019 20:26:56 +0300 Subject: [PATCH 14/58] Update in.cpp --- dbms/src/Functions/in.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/in.cpp b/dbms/src/Functions/in.cpp index c7c6a1f6c15..2911f743538 100644 --- a/dbms/src/Functions/in.cpp +++ b/dbms/src/Functions/in.cpp @@ -84,7 +84,7 @@ public: Block block_of_key_columns; - /// First argument may be tuple or single column. + /// First argument may be a tuple or a single column. const ColumnWithTypeAndName & left_arg = block.getByPosition(arguments[0]); const ColumnTuple * tuple = typeid_cast(left_arg.column.get()); const ColumnConst * const_tuple = checkAndGetColumnConst(left_arg.column.get()); From 173884c04f63f88ae0958b929ee723006c13f23f Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Sun, 5 May 2019 12:27:18 +0300 Subject: [PATCH 15/58] Perf test for upcoming review for regexp matching --- dbms/tests/performance/constant_column_search.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/performance/constant_column_search.xml b/dbms/tests/performance/constant_column_search.xml index df5c8f5886c..da1c94230dd 100644 --- a/dbms/tests/performance/constant_column_search.xml +++ b/dbms/tests/performance/constant_column_search.xml @@ -22,6 +22,8 @@ + + From 451407a82973f868445966ebf933a120920278d5 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Sun, 5 May 2019 12:32:26 +0300 Subject: [PATCH 16/58] Fix word extraction if prefix candidate is small --- .../src/Common/OptimizedRegularExpression.cpp | 26 ++++++++++++++----- 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/dbms/src/Common/OptimizedRegularExpression.cpp b/dbms/src/Common/OptimizedRegularExpression.cpp index 0b5f5c42b14..e8132fc1cd6 100644 --- a/dbms/src/Common/OptimizedRegularExpression.cpp +++ b/dbms/src/Common/OptimizedRegularExpression.cpp @@ -1,7 +1,6 @@ #include #include - #define MIN_LENGTH_FOR_STRSTR 3 #define MAX_SUBPATTERNS 5 @@ -214,23 +213,38 @@ void OptimizedRegularExpressionImpl::analyze( /** We choose the non-alternative substring of the maximum length, among the prefixes, * or a non-alternative substring of maximum length. */ + + /// Tuning for typical usage domain + auto tuning_strings_condition = [](const std::string & str) + { + return str != "://" && str != "http://" && str != "www" && str != "Windows "; + }; size_t max_length = 0; Substrings::const_iterator candidate_it = trivial_substrings.begin(); for (Substrings::const_iterator it = trivial_substrings.begin(); it != trivial_substrings.end(); ++it) { if (((it->second == 0 && candidate_it->second != 0) || ((it->second == 0) == (candidate_it->second == 0) && it->first.size() > max_length)) - /// Tuning for typical usage domain - && (it->first.size() > strlen("://") || strncmp(it->first.data(), "://", strlen("://"))) - && (it->first.size() > strlen("http://") || strncmp(it->first.data(), "http", strlen("http"))) - && (it->first.size() > strlen("www.") || strncmp(it->first.data(), "www", strlen("www"))) - && (it->first.size() > strlen("Windows ") || strncmp(it->first.data(), "Windows ", strlen("Windows ")))) + && tuning_strings_condition(it->first)) { max_length = it->first.size(); candidate_it = it; } } + /// If prefix is small, it won't be chosen + if (max_length == 0) + { + for (Substrings::const_iterator it = trivial_substrings.begin(); it != trivial_substrings.end(); ++it) + { + if (it->first.size() > max_length && tuning_strings_condition(it->first)) + { + max_length = it->first.size(); + candidate_it = it; + } + } + } + if (max_length >= MIN_LENGTH_FOR_STRSTR) { required_substring = candidate_it->first; From e531348e12d810418b66cae71cdab00a0e324d58 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Sun, 5 May 2019 12:34:43 +0300 Subject: [PATCH 17/58] Better condition --- dbms/src/Common/OptimizedRegularExpression.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/OptimizedRegularExpression.cpp b/dbms/src/Common/OptimizedRegularExpression.cpp index e8132fc1cd6..e202718d9a7 100644 --- a/dbms/src/Common/OptimizedRegularExpression.cpp +++ b/dbms/src/Common/OptimizedRegularExpression.cpp @@ -233,7 +233,7 @@ void OptimizedRegularExpressionImpl::analyze( } /// If prefix is small, it won't be chosen - if (max_length == 0) + if (max_length < MIN_LENGTH_FOR_STRSTR) { for (Substrings::const_iterator it = trivial_substrings.begin(); it != trivial_substrings.end(); ++it) { From 359e4c33bda646adc3f3d5795a66daf5bea0a2fb Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 5 May 2019 18:49:12 +0800 Subject: [PATCH 18/58] try fix push require columns with join --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 06a8b93f8e2..5607733d553 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -513,13 +513,14 @@ void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only columns_added_by_join_list)); } -static void appendRequiredColumns(NameSet & required_columns, const Block & sample, const AnalyzedJoin & analyzed_join) +static void appendRequiredColumns( + NameSet & required_columns, const Block & sample, const Names & key_names_right, const JoinedColumnsList & columns_added_by_join) { - for (auto & column : analyzed_join.key_names_right) + for (auto & column : key_names_right) if (!sample.has(column)) required_columns.insert(column); - for (auto & column : analyzed_join.columns_from_joined_table) + for (auto & column : columns_added_by_join) if (!sample.has(column.name_and_type.name)) required_columns.insert(column.name_and_type.name); } @@ -606,7 +607,8 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty Names action_columns = joined_block_actions->getRequiredColumns(); NameSet required_columns(action_columns.begin(), action_columns.end()); - appendRequiredColumns(required_columns, joined_block_actions->getSampleBlock(), analyzed_join); + appendRequiredColumns( + required_columns, joined_block_actions->getSampleBlock(), analyzed_join.key_names_right, columns_added_by_join); Names original_columns = analyzed_join.getOriginalColumnNames(required_columns); From ced9805cc1caf05f4de15e0ea43092701649fe64 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Sun, 5 May 2019 15:26:20 +0300 Subject: [PATCH 19/58] Remove condition of prefixes first --- .../src/Common/OptimizedRegularExpression.cpp | 21 ++----------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/dbms/src/Common/OptimizedRegularExpression.cpp b/dbms/src/Common/OptimizedRegularExpression.cpp index e202718d9a7..c87d87fc2df 100644 --- a/dbms/src/Common/OptimizedRegularExpression.cpp +++ b/dbms/src/Common/OptimizedRegularExpression.cpp @@ -210,9 +210,7 @@ void OptimizedRegularExpressionImpl::analyze( { if (!has_alternative_on_depth_0) { - /** We choose the non-alternative substring of the maximum length, among the prefixes, - * or a non-alternative substring of maximum length. - */ + /// We choose the non-alternative substring of the maximum length for first search. /// Tuning for typical usage domain auto tuning_strings_condition = [](const std::string & str) @@ -223,28 +221,13 @@ void OptimizedRegularExpressionImpl::analyze( Substrings::const_iterator candidate_it = trivial_substrings.begin(); for (Substrings::const_iterator it = trivial_substrings.begin(); it != trivial_substrings.end(); ++it) { - if (((it->second == 0 && candidate_it->second != 0) - || ((it->second == 0) == (candidate_it->second == 0) && it->first.size() > max_length)) - && tuning_strings_condition(it->first)) + if (it->first.size() > max_length && tuning_strings_condition(it->first)) { max_length = it->first.size(); candidate_it = it; } } - /// If prefix is small, it won't be chosen - if (max_length < MIN_LENGTH_FOR_STRSTR) - { - for (Substrings::const_iterator it = trivial_substrings.begin(); it != trivial_substrings.end(); ++it) - { - if (it->first.size() > max_length && tuning_strings_condition(it->first)) - { - max_length = it->first.size(); - candidate_it = it; - } - } - } - if (max_length >= MIN_LENGTH_FOR_STRSTR) { required_substring = candidate_it->first; From 7c54df39b84a2ff986ca2058ebc53ddf7a101131 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 May 2019 00:20:06 +0800 Subject: [PATCH 20/58] ISSUES-5190 add performance test from Denny Crane --- dbms/tests/performance/simple_join_query.xml | 30 ++++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 dbms/tests/performance/simple_join_query.xml diff --git a/dbms/tests/performance/simple_join_query.xml b/dbms/tests/performance/simple_join_query.xml new file mode 100644 index 00000000000..3830690fea3 --- /dev/null +++ b/dbms/tests/performance/simple_join_query.xml @@ -0,0 +1,30 @@ + + Simple Join Query + + once + + + + 30000 + + + 5000 + 60000 + + + + + + + + CREATE TABLE join_table(A Int64, S0 String, S1 String, S2 String, S3 String)ENGINE = MergeTree ORDER BY A + + INSERT INTO join_table SELECT number AS A, toString(arrayMap(x->x, range(100))) S0, S0 AS S1, S0 AS S2, S0 AS S3 from numbers(500000) + + SELECT COUNT() FROM join_table LEFT JOIN join_table USING A + SELECT COUNT() FROM join_table LEFT JOIN (SELECT A FROM join_table) USING A + SELECT COUNT() FROM join_table AS left LEFT JOIN join_table AS right ON left.A = right.A + SELECT COUNT() FROM join_table AS left LEFT JOIN (SELECT A FROM join_table) AS right ON left.A = right.A + + DROP TABLE IF EXISTS join_table + From f613c06c479f234d7894ef5cfe83a81317fc5c50 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Sun, 5 May 2019 23:48:46 +0300 Subject: [PATCH 21/58] ngram vector_vector distance added --- .../Functions/FunctionsStringSimilarity.cpp | 137 +++++-- .../src/Functions/FunctionsStringSimilarity.h | 29 +- dbms/tests/performance/ngram_distance.xml | 5 +- .../00909_ngram_distance.reference | 346 ++++++++++++++++++ .../0_stateless/00909_ngram_distance.sql | 74 ++++ 5 files changed, 554 insertions(+), 37 deletions(-) diff --git a/dbms/src/Functions/FunctionsStringSimilarity.cpp b/dbms/src/Functions/FunctionsStringSimilarity.cpp index 255301c00ed..d640cbf27e4 100644 --- a/dbms/src/Functions/FunctionsStringSimilarity.cpp +++ b/dbms/src/Functions/FunctionsStringSimilarity.cpp @@ -164,43 +164,46 @@ struct NgramDistanceImpl return num; } + template static ALWAYS_INLINE inline size_t calculateNeedleStats( const char * data, const size_t size, NgramStats & ngram_stats, + [[maybe_unused]] UInt16 * ngram_storage, size_t (*read_code_points)(CodePoint *, const char *&, const char *), UInt16 (*hash_functor)(const CodePoint *)) { - // To prevent size_t overflow below. - if (size < N) - return 0; - const char * start = data; const char * end = data + size; CodePoint cp[simultaneously_codepoints_num] = {}; - /// read_code_points returns the position of cp where it stopped reading codepoints. size_t found = read_code_points(cp, start, end); /// We need to start for the first time here, because first N - 1 codepoints mean nothing. size_t i = N - 1; - /// Initialize with this value because for the first time `found` does not initialize first N - 1 codepoints. - size_t len = -N + 1; + size_t len = 0; do { - len += found - N + 1; for (; i + N <= found; ++i) - ++ngram_stats[hash_functor(cp + i)]; + { + ++len; + UInt16 hash = hash_functor(cp + i); + if constexpr (SaveNgrams) + *ngram_storage++ = hash; + ++ngram_stats[hash]; + } i = 0; } while (start < end && (found = read_code_points(cp, start, end))); return len; } + template static ALWAYS_INLINE inline UInt64 calculateHaystackStatsAndMetric( const char * data, const size_t size, NgramStats & ngram_stats, size_t & distance, + [[maybe_unused]] UInt16 * ngram_storage, size_t (*read_code_points)(CodePoint *, const char *&, const char *), UInt16 (*hash_functor)(const CodePoint *)) { @@ -209,18 +212,6 @@ struct NgramDistanceImpl const char * end = data + size; CodePoint cp[simultaneously_codepoints_num] = {}; - /// allocation tricks, most strings are relatively small - static constexpr size_t small_buffer_size = 256; - std::unique_ptr big_buffer; - UInt16 small_buffer[small_buffer_size]; - UInt16 * ngram_storage = small_buffer; - - if (size > small_buffer_size) - { - ngram_storage = new UInt16[size]; - big_buffer.reset(ngram_storage); - } - /// read_code_points returns the position of cp where it stopped reading codepoints. size_t found = read_code_points(cp, start, end); /// We need to start for the first time here, because first N - 1 codepoints mean nothing. @@ -235,21 +226,25 @@ struct NgramDistanceImpl --distance; else ++distance; - - ngram_storage[ngram_cnt++] = hash; + if constexpr (ReuseStats) + ngram_storage[ngram_cnt] = hash; + ++ngram_cnt; --ngram_stats[hash]; } iter = 0; } while (start < end && (found = read_code_points(cp, start, end))); /// Return the state of hash map to its initial. - for (size_t i = 0; i < ngram_cnt; ++i) - ++ngram_stats[ngram_storage[i]]; + if constexpr (ReuseStats) + { + for (size_t i = 0; i < ngram_cnt; ++i) + ++ngram_stats[ngram_storage[i]]; + } return ngram_cnt; } template - static inline size_t dispatchSearcher(Callback callback, Args &&... args) + static inline auto dispatchSearcher(Callback callback, Args &&... args) { if constexpr (!UTF8) return callback(std::forward(args)..., readASCIICodePoints, ASCIIHash); @@ -268,11 +263,11 @@ struct NgramDistanceImpl needle.resize(needle_size + default_padding); data.resize(data_size + default_padding); - size_t second_size = dispatchSearcher(calculateNeedleStats, needle.data(), needle_size, common_stats); + size_t second_size = dispatchSearcher(calculateNeedleStats, needle.data(), needle_size, common_stats, nullptr); size_t distance = second_size; if (data_size <= max_string_size) { - size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric, data.data(), data_size, common_stats, distance); + size_t first_size = dispatchSearcher(calculateHaystackStatsAndMetric, data.data(), data_size, common_stats, distance, nullptr); res = distance * 1.f / std::max(first_size + second_size, size_t(1)); } else @@ -281,18 +276,94 @@ struct NgramDistanceImpl } } + static void vector_vector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + PaddedPODArray & res) + { + const size_t haystack_offsets_size = haystack_offsets.size(); + size_t prev_haystack_offset = 0; + size_t prev_needle_offset = 0; + + NgramStats common_stats; + memset(common_stats, 0, sizeof(common_stats)); + + /// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb). + /// And we can reuse these storages in one thread because we care only about what was written to first places. + std::unique_ptr needle_ngram_storage; + std::unique_ptr haystack_ngram_storage; + needle_ngram_storage.reset(new UInt16[max_string_size]); + haystack_ngram_storage.reset(new UInt16[max_string_size]); + + for (size_t i = 0; i < haystack_offsets_size; ++i) + { + const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); + const size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); + const size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + + if (needle_size <= max_string_size && haystack_size <= max_string_size) + { + /// Get needle stats. + const size_t needle_stats_size = dispatchSearcher( + calculateNeedleStats, + needle, + needle_size, + common_stats, + needle_ngram_storage.get()); + + size_t distance = needle_stats_size; + + /// Combine with haystack stats, return to initial needle stats. + const size_t haystack_stats_size = dispatchSearcher( + calculateHaystackStatsAndMetric, + haystack, + haystack_size, + common_stats, + distance, + haystack_ngram_storage.get()); + + /// Return to zero array stats. + for (size_t j = 0; j < needle_stats_size; ++j) + --common_stats[needle_ngram_storage[j]]; + + /// For now, common stats is a zero array. + res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, size_t(1)); + } + else + { + /// String are too big, we are assuming they are not the same. This is done because of limiting number + /// of bigrams added and not allocating too much memory. + res[i] = 1.f; + } + + prev_needle_offset = needle_offsets[i]; + prev_haystack_offset = haystack_offsets[i]; + } + } + static void vector_constant( - const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string needle, PaddedPODArray & res) + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + std::string needle, + PaddedPODArray & res) { /// zeroing our map NgramStats common_stats; memset(common_stats, 0, sizeof(common_stats)); + /// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb). + /// And we can reuse these storages in one thread because we care only about what was written to first places. + std::unique_ptr ngram_storage; + ngram_storage.reset(new UInt16[max_string_size]); + /// We use unsafe versions of getting ngrams, so I decided to use padded_data even in needle case. const size_t needle_size = needle.size(); needle.resize(needle_size + default_padding); - const size_t needle_stats_size = dispatchSearcher(calculateNeedleStats, needle.data(), needle_size, common_stats); + const size_t needle_stats_size = dispatchSearcher(calculateNeedleStats, needle.data(), needle_size, common_stats, nullptr); size_t distance = needle_stats_size; size_t prev_offset = 0; @@ -303,7 +374,11 @@ struct NgramDistanceImpl if (haystack_size <= max_string_size) { size_t haystack_stats_size = dispatchSearcher( - calculateHaystackStatsAndMetric, reinterpret_cast(haystack), haystack_size, common_stats, distance); + calculateHaystackStatsAndMetric, + reinterpret_cast(haystack), + haystack_size, common_stats, + distance, + ngram_storage.get()); res[i] = distance * 1.f / std::max(haystack_stats_size + needle_stats_size, size_t(1)); } else diff --git a/dbms/src/Functions/FunctionsStringSimilarity.h b/dbms/src/Functions/FunctionsStringSimilarity.h index 7a106980722..9fd11e47622 100644 --- a/dbms/src/Functions/FunctionsStringSimilarity.h +++ b/dbms/src/Functions/FunctionsStringSimilarity.h @@ -62,10 +62,7 @@ public: const ColumnConst * col_haystack_const = typeid_cast(&*column_haystack); const ColumnConst * col_needle_const = typeid_cast(&*column_needle); - if (!col_needle_const) - throw Exception("Second argument of function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_COLUMN); - - if (col_haystack_const) + if (col_haystack_const && col_needle_const) { ResultType res{}; const String & needle = col_needle_const->getValue(); @@ -88,8 +85,9 @@ public: vec_res.resize(column_haystack->size()); const ColumnString * col_haystack_vector = checkAndGetColumn(&*column_haystack); + const ColumnString * col_needle_vector = checkAndGetColumn(&*column_needle); - if (col_haystack_vector) + if (col_haystack_vector && col_needle_const) { const String & needle = col_needle_const->getValue(); if (needle.size() > Impl::max_string_size) @@ -101,6 +99,27 @@ public: } Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), needle, vec_res); } + else if (col_haystack_vector && col_needle_vector) + { + Impl::vector_vector( + col_haystack_vector->getChars(), + col_haystack_vector->getOffsets(), + col_needle_vector->getChars(), + col_needle_vector->getOffsets(), + vec_res); + } + else if (col_haystack_const && col_needle_vector) + { + const String & needle = col_haystack_const->getValue(); + if (needle.size() > Impl::max_string_size) + { + throw Exception( + "String size of needle is too big for function " + getName() + ". Should be at most " + + std::to_string(Impl::max_string_size), + ErrorCodes::TOO_LARGE_STRING_SIZE); + } + Impl::vector_constant(col_needle_vector->getChars(), col_needle_vector->getOffsets(), needle, vec_res); + } else { throw Exception( diff --git a/dbms/tests/performance/ngram_distance.xml b/dbms/tests/performance/ngram_distance.xml index 16960811067..af1d81e6f61 100644 --- a/dbms/tests/performance/ngram_distance.xml +++ b/dbms/tests/performance/ngram_distance.xml @@ -21,13 +21,16 @@ 60000 - + SELECT DISTINCT URL,Title, ngramDistance(Title, URL) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 + SELECT DISTINCT SearchPhrase,Title, ngramDistance(Title, SearchPhrase) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT Title, ngramDistance(Title, 'what is love') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT Title, ngramDistance(Title, 'baby dont hurt me') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT Title, ngramDistance(Title, 'no more') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'wHAt Is lovE') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'BABY DonT hUrT me') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT Title, ngramDistanceCaseInsensitive(Title, 'nO MOrE') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 + SELECT DISTINCT URL,Title, ngramDistanceUTF8(Title, URL) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 + SELECT DISTINCT SearchPhrase,Title, ngramDistanceUTF8(Title, SearchPhrase) AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT Title, ngramDistanceUTF8(Title, 'метрика') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'как дела') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 SELECT DISTINCT URL, ngramDistanceUTF8(URL, 'чем занимаешься') AS distance FROM hits_100m_single ORDER BY distance ASC LIMIT 50 diff --git a/dbms/tests/queries/0_stateless/00909_ngram_distance.reference b/dbms/tests/queries/0_stateless/00909_ngram_distance.reference index 356cc5db466..4fc02992bdf 100644 --- a/dbms/tests/queries/0_stateless/00909_ngram_distance.reference +++ b/dbms/tests/queries/0_stateless/00909_ngram_distance.reference @@ -33,6 +33,76 @@ 1000 1000 1000 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 0 1000 1000 @@ -40,6 +110,39 @@ 77 636 1000 +привет как дела?... Херсон 0 +привет как дела клип - Яндекс.Видео 0 +привет 0 +пап привет как дела - Яндекс.Видео 0 +привет братан как дела - Яндекс.Видео 0 +http://metric.ru/ 0 +http://autometric.ru/ 0 +http://metrica.yandex.com/ 0 +http://metris.ru/ 0 +http://metrika.ru/ 0 + 0 + 0 +привет как дела?... Херсон 600 +пап привет как дела - Яндекс.Видео 684 +привет как дела клип - Яндекс.Видео 692 +привет братан как дела - Яндекс.Видео 707 +привет 1000 +http://metric.ru/ 1000 +http://autometric.ru/ 1000 +http://metrica.yandex.com/ 1000 +http://metris.ru/ 1000 +http://metrika.ru/ 1000 + 0 +http://metric.ru/ 765 +http://metris.ru/ 765 +http://metrika.ru/ 778 +http://autometric.ru/ 810 +http://metrica.yandex.com/ 846 +привет как дела?... Херсон 1000 +привет как дела клип - Яндекс.Видео 1000 +привет 1000 +пап привет как дела - Яндекс.Видео 1000 +привет братан как дела - Яндекс.Видео 1000 привет как дела?... Херсон 297 пап привет как дела - Яндекс.Видео 422 привет как дела клип - Яндекс.Видео 435 @@ -152,6 +255,76 @@ http://metrika.ru/ 1000 1000 1000 1000 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 0 1000 1000 @@ -159,6 +332,39 @@ http://metrika.ru/ 1000 77 636 1000 +привет как дела?... Херсон 0 +привет как дела клип - Яндекс.Видео 0 +привет 0 +пап привет как дела - Яндекс.Видео 0 +привет братан как дела - Яндекс.Видео 0 +http://metric.ru/ 0 +http://autometric.ru/ 0 +http://metrica.yandex.com/ 0 +http://metris.ru/ 0 +http://metrika.ru/ 0 + 0 + 0 +привет как дела?... Херсон 600 +пап привет как дела - Яндекс.Видео 684 +привет как дела клип - Яндекс.Видео 692 +привет братан как дела - Яндекс.Видео 707 +привет 1000 +http://metric.ru/ 1000 +http://autometric.ru/ 1000 +http://metrica.yandex.com/ 1000 +http://metris.ru/ 1000 +http://metrika.ru/ 1000 + 0 +http://metric.ru/ 765 +http://metris.ru/ 765 +http://metrika.ru/ 778 +http://autometric.ru/ 810 +http://metrica.yandex.com/ 846 +привет как дела?... Херсон 1000 +привет как дела клип - Яндекс.Видео 1000 +привет 1000 +пап привет как дела - Яндекс.Видео 1000 +привет братан как дела - Яндекс.Видео 1000 привет как дела?... Херсон 297 пап привет как дела - Яндекс.Видео 422 привет как дела клип - Яндекс.Видео 435 @@ -293,6 +499,76 @@ http://metrika.ru/ 1000 1000 1000 1000 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 0 0 0 @@ -412,6 +688,76 @@ http://metrika.ru/ 1000 1000 1000 1000 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 0 0 0 diff --git a/dbms/tests/queries/0_stateless/00909_ngram_distance.sql b/dbms/tests/queries/0_stateless/00909_ngram_distance.sql index ea7bd252fc7..ed800bf6c97 100644 --- a/dbms/tests/queries/0_stateless/00909_ngram_distance.sql +++ b/dbms/tests/queries/0_stateless/00909_ngram_distance.sql @@ -6,6 +6,22 @@ select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абв select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5; select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'ёёёёёёёё')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8(materialize(''), materialize('')))=round(1000 * ngramDistanceUTF8(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8(materialize('абв'), materialize('')))=round(1000 * ngramDistanceUTF8(materialize('абв'), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8(materialize(''), materialize('абв')))=round(1000 * ngramDistanceUTF8(materialize(''), 'абв')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('абвгдеёжз')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёжз')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('абвгдеёж')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёж')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('гдеёзд')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), materialize('ёёёёёёёё')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'ёёёёёёёё')) from system.numbers limit 5; + +select round(1000 * ngramDistanceUTF8('', materialize('')))=round(1000 * ngramDistanceUTF8(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8('абв', materialize('')))=round(1000 * ngramDistanceUTF8(materialize('абв'), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8('', materialize('абв')))=round(1000 * ngramDistanceUTF8(materialize(''), 'абв')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('абвгдеёжз')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёжз')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('абвгдеёж')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'абвгдеёж')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('гдеёзд')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'гдеёзд')) from system.numbers limit 5; +select round(1000 * ngramDistanceUTF8('абвгдеёжз', materialize('ёёёёёёёё')))=round(1000 * ngramDistanceUTF8(materialize('абвгдеёжз'), 'ёёёёёёёё')) from system.numbers limit 5; + select round(1000 * ngramDistanceUTF8('', '')); select round(1000 * ngramDistanceUTF8('абв', '')); select round(1000 * ngramDistanceUTF8('', 'абв')); @@ -18,6 +34,10 @@ drop table if exists test_distance; create table test_distance (Title String) engine = Memory; insert into test_distance values ('привет как дела?... Херсон'), ('привет как дела клип - Яндекс.Видео'), ('привет'), ('пап привет как дела - Яндекс.Видео'), ('привет братан как дела - Яндекс.Видео'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://metrica.yandex.com/'), ('http://metris.ru/'), ('http://metrika.ru/'), (''); +SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, Title) as distance; +SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'как дела')) as distance; +SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'metr')) as distance; + SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'привет как дела') as distance; SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'как привет дела') as distance; SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrika') as distance; @@ -35,6 +55,23 @@ select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеё select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'гдеёЗД')) from system.numbers limit 5; select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'ЁЁЁЁЁЁЁЁ')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''),materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абв'),materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абв'), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), materialize('абв'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), 'абв')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвГДЕёжз'), materialize('АбвгдЕёжз'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвГДЕёжз'), 'АбвгдЕёжз')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеёЖз'), materialize('АбвГдеёж'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеёЖз'), 'АбвГдеёж')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), materialize('гдеёЗД'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'гдеёЗД')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), materialize('ЁЁЁЁЁЁЁЁ'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'ЁЁЁЁЁЁЁЁ')) from system.numbers limit 5; + +select round(1000 * ngramDistanceCaseInsensitiveUTF8('', materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8('абв',materialize(''))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абв'), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8('', materialize('абв'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), 'абв')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвГДЕёжз', materialize('АбвгдЕёжз'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвГДЕёжз'), 'АбвгдЕёжз')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8('аБВГдеёЖз', materialize('АбвГдеёж'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('аБВГдеёЖз'), 'АбвГдеёж')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', materialize('гдеёЗД'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'гдеёЗД')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', materialize('ЁЁЁЁЁЁЁЁ'))) = round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize('абвгдеёжз'), 'ЁЁЁЁЁЁЁЁ')) from system.numbers limit 5; + + select round(1000 * ngramDistanceCaseInsensitiveUTF8('', '')); select round(1000 * ngramDistanceCaseInsensitiveUTF8('абв', '')); select round(1000 * ngramDistanceCaseInsensitiveUTF8('', 'абв')); @@ -43,6 +80,10 @@ select round(1000 * ngramDistanceCaseInsensitiveUTF8('аБВГдеёЖз', 'Аб select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', 'гдеёЗД')); select round(1000 * ngramDistanceCaseInsensitiveUTF8('АБВГДеёжз', 'ЁЁЁЁЁЁЁЁ')); +SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, Title) as distance; +SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'как дела')) as distance; +SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'metr')) as distance; + SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'ПрИвЕт кАК ДЕЛа') as distance; SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'как ПРИВЕТ дела') as distance; SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrika') as distance; @@ -62,6 +103,23 @@ select round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefg')) from syst select round(1000 * ngramDistance(materialize('abcdefgh'), 'defgh')) from system.numbers limit 5; select round(1000 * ngramDistance(materialize('abcdefgh'), 'aaaaaaaa')) from system.numbers limit 5; +select round(1000 * ngramDistance(materialize(''),materialize('')))=round(1000 * ngramDistance(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistance(materialize('abc'),materialize('')))=round(1000 * ngramDistance(materialize('abc'), '')) from system.numbers limit 5; +select round(1000 * ngramDistance(materialize(''), materialize('abc')))=round(1000 * ngramDistance(materialize(''), 'abc')) from system.numbers limit 5; +select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('abcdefgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefgh')) from system.numbers limit 5; +select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('abcdefg')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefg')) from system.numbers limit 5; +select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('defgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'defgh')) from system.numbers limit 5; +select round(1000 * ngramDistance(materialize('abcdefgh'), materialize('aaaaaaaa')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'aaaaaaaa')) from system.numbers limit 5; + +select round(1000 * ngramDistance('',materialize('')))=round(1000 * ngramDistance(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistance('abc', materialize('')))=round(1000 * ngramDistance(materialize('abc'), '')) from system.numbers limit 5; +select round(1000 * ngramDistance('', materialize('abc')))=round(1000 * ngramDistance(materialize(''), 'abc')) from system.numbers limit 5; +select round(1000 * ngramDistance('abcdefgh', materialize('abcdefgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefgh')) from system.numbers limit 5; +select round(1000 * ngramDistance('abcdefgh', materialize('abcdefg')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'abcdefg')) from system.numbers limit 5; +select round(1000 * ngramDistance('abcdefgh', materialize('defgh')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'defgh')) from system.numbers limit 5; +select round(1000 * ngramDistance('abcdefgh', materialize('aaaaaaaa')))=round(1000 * ngramDistance(materialize('abcdefgh'), 'aaaaaaaa')) from system.numbers limit 5; + + select round(1000 * ngramDistance('', '')); select round(1000 * ngramDistance('abc', '')); select round(1000 * ngramDistance('', 'abc')); @@ -86,6 +144,22 @@ select round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), 'abcde select round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), 'defgh')) from system.numbers limit 5; select round(1000 * ngramDistanceCaseInsensitive(materialize('ABCdefgH'), 'aaaaaaaa')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive(materialize(''), materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive(materialize(''), materialize('abc')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), 'abc')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive(materialize('abCdefgH'), materialize('Abcdefgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abCdefgH'), 'Abcdefgh')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), materialize('abcdeFG')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), 'abcdeFG')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), materialize('defgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), 'defgh')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive(materialize('ABCdefgH'), materialize('aaaaaaaa')))=round(1000 * ngramDistanceCaseInsensitive(materialize('ABCdefgH'), 'aaaaaaaa')) from system.numbers limit 5; + +select round(1000 * ngramDistanceCaseInsensitive('', materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive('abc', materialize('')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive('', materialize('abc')))=round(1000 * ngramDistanceCaseInsensitive(materialize(''), 'abc')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive('abCdefgH', materialize('Abcdefgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abCdefgH'), 'Abcdefgh')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive('abcdefgh', materialize('abcdeFG')))=round(1000 * ngramDistanceCaseInsensitive(materialize('abcdefgh'), 'abcdeFG')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive('AAAAbcdefgh', materialize('defgh')))=round(1000 * ngramDistanceCaseInsensitive(materialize('AAAAbcdefgh'), 'defgh')) from system.numbers limit 5; +select round(1000 * ngramDistanceCaseInsensitive('ABCdefgH', materialize('aaaaaaaa')))=round(1000 * ngramDistanceCaseInsensitive(materialize('ABCdefgH'), 'aaaaaaaa')) from system.numbers limit 5; + select round(1000 * ngramDistanceCaseInsensitive('', '')); select round(1000 * ngramDistanceCaseInsensitive('abc', '')); select round(1000 * ngramDistanceCaseInsensitive('', 'abc')); From 6a6b30a56fa8e95a7508239d938640a40cecb90b Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 6 May 2019 00:35:08 +0300 Subject: [PATCH 22/58] Less code with unique ptrs and some docs added --- dbms/src/Functions/FunctionsStringSimilarity.cpp | 14 ++++++++------ .../functions/string_search_functions.md | 2 +- .../functions/string_search_functions.md | 2 +- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/dbms/src/Functions/FunctionsStringSimilarity.cpp b/dbms/src/Functions/FunctionsStringSimilarity.cpp index d640cbf27e4..feec2ee5a86 100644 --- a/dbms/src/Functions/FunctionsStringSimilarity.cpp +++ b/dbms/src/Functions/FunctionsStringSimilarity.cpp @@ -292,10 +292,8 @@ struct NgramDistanceImpl /// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb). /// And we can reuse these storages in one thread because we care only about what was written to first places. - std::unique_ptr needle_ngram_storage; - std::unique_ptr haystack_ngram_storage; - needle_ngram_storage.reset(new UInt16[max_string_size]); - haystack_ngram_storage.reset(new UInt16[max_string_size]); + std::unique_ptr needle_ngram_storage(new UInt16[max_string_size]); + std::unique_ptr haystack_ngram_storage(new UInt16[max_string_size]); for (size_t i = 0; i < haystack_offsets_size; ++i) { @@ -356,8 +354,12 @@ struct NgramDistanceImpl /// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb). /// And we can reuse these storages in one thread because we care only about what was written to first places. - std::unique_ptr ngram_storage; - ngram_storage.reset(new UInt16[max_string_size]); + std::unique_ptr ngram_storage(new UInt16[max_string_size]); + + + + + /// We use unsafe versions of getting ngrams, so I decided to use padded_data even in needle case. const size_t needle_size = needle.size(); diff --git a/docs/en/query_language/functions/string_search_functions.md b/docs/en/query_language/functions/string_search_functions.md index 3674869bd81..d2f552ca2ec 100644 --- a/docs/en/query_language/functions/string_search_functions.md +++ b/docs/en/query_language/functions/string_search_functions.md @@ -102,7 +102,7 @@ The same thing as 'like', but negative. ## ngramDistance(haystack, needle) -Calculates the 4-gram distance between `haystack` and `needle`: counts the symmetric difference between two multisets of 4-grams and normalizes it by the sum of their cardinalities. Returns float number from 0 to 1 -- the closer to zero, the more strings are similar to each other. If the `needle` is more than 32Kb, throws an exception. If some of the `haystack` strings are more than 32Kb, the distance is always one. +Calculates the 4-gram distance between `haystack` and `needle`: counts the symmetric difference between two multisets of 4-grams and normalizes it by the sum of their cardinalities. Returns float number from 0 to 1 -- the closer to zero, the more strings are similar to each other. If the constant `needle` or `haystack` is more than 32Kb, throws an exception. If some of the non-constant `haystack` or `needle` strings are more than 32Kb, the distance is always one. For case-insensitive search or/and in UTF-8 format use functions `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8`. diff --git a/docs/ru/query_language/functions/string_search_functions.md b/docs/ru/query_language/functions/string_search_functions.md index f7981f32365..9846b8aa67c 100644 --- a/docs/ru/query_language/functions/string_search_functions.md +++ b/docs/ru/query_language/functions/string_search_functions.md @@ -91,7 +91,7 @@ ## ngramDistance(haystack, needle) -Вычисление 4-граммного расстояния между `haystack` и `needle`: считается симметрическая разность между двумя мультимножествами 4-грамм и нормализается на сумму их мощностей. Возвращает число float от 0 до 1 -- чем ближе к нулю, тем больше строки похожи друг на друга. Если `needle` больше чем 32КБ, кидается исключение. Если некоторые строки из `haystack` больше 32КБ, расстояние всегда равно единице. +Вычисление 4-граммного расстояния между `haystack` и `needle`: считается симметрическая разность между двумя мультимножествами 4-грамм и нормализается на сумму их мощностей. Возвращает число float от 0 до 1 -- чем ближе к нулю, тем больше строки похожи друг на друга. Если константный `needle` или `haystack` больше чем 32КБ, кидается исключение. Если некоторые строки из неконстантного `haystack` или `needle` больше 32КБ, расстояние всегда равно единице. Для поиска без учета регистра и/или в формате UTF-8 используйте функции `ngramDistanceCaseInsensitive, ngramDistanceUTF8, ngramDistanceCaseInsensitiveUTF8`. From 8b235f7a4a9f3b2a2557951a13ee9565fb8d4198 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 6 May 2019 00:37:58 +0300 Subject: [PATCH 23/58] Comments --- dbms/src/Functions/FunctionsStringSimilarity.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsStringSimilarity.cpp b/dbms/src/Functions/FunctionsStringSimilarity.cpp index feec2ee5a86..57f36a7ec75 100644 --- a/dbms/src/Functions/FunctionsStringSimilarity.cpp +++ b/dbms/src/Functions/FunctionsStringSimilarity.cpp @@ -332,7 +332,7 @@ struct NgramDistanceImpl } else { - /// String are too big, we are assuming they are not the same. This is done because of limiting number + /// Strings are too big, we are assuming they are not the same. This is done because of limiting number /// of bigrams added and not allocating too much memory. res[i] = 1.f; } From 512c7fa3a444d91e08f0df5a60b070e2991ed5aa Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 6 May 2019 00:40:37 +0300 Subject: [PATCH 24/58] Comments --- dbms/src/Functions/FunctionsStringSimilarity.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/dbms/src/Functions/FunctionsStringSimilarity.cpp b/dbms/src/Functions/FunctionsStringSimilarity.cpp index 57f36a7ec75..ee77b41eaf6 100644 --- a/dbms/src/Functions/FunctionsStringSimilarity.cpp +++ b/dbms/src/Functions/FunctionsStringSimilarity.cpp @@ -356,11 +356,6 @@ struct NgramDistanceImpl /// And we can reuse these storages in one thread because we care only about what was written to first places. std::unique_ptr ngram_storage(new UInt16[max_string_size]); - - - - - /// We use unsafe versions of getting ngrams, so I decided to use padded_data even in needle case. const size_t needle_size = needle.size(); needle.resize(needle_size + default_padding); From c280907f0988e56e99f26cfc1813822e85b8c559 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Mon, 6 May 2019 11:34:28 +0300 Subject: [PATCH 25/58] Zero initialization instead of memset --- dbms/src/Functions/FunctionsStringSimilarity.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/dbms/src/Functions/FunctionsStringSimilarity.cpp b/dbms/src/Functions/FunctionsStringSimilarity.cpp index ee77b41eaf6..e12455ee560 100644 --- a/dbms/src/Functions/FunctionsStringSimilarity.cpp +++ b/dbms/src/Functions/FunctionsStringSimilarity.cpp @@ -254,8 +254,7 @@ struct NgramDistanceImpl static void constant_constant(std::string data, std::string needle, Float32 & res) { - NgramStats common_stats; - memset(common_stats, 0, sizeof(common_stats)); + NgramStats common_stats = {}; /// We use unsafe versions of getting ngrams, so I decided to use padded strings. const size_t needle_size = needle.size(); @@ -287,8 +286,7 @@ struct NgramDistanceImpl size_t prev_haystack_offset = 0; size_t prev_needle_offset = 0; - NgramStats common_stats; - memset(common_stats, 0, sizeof(common_stats)); + NgramStats common_stats = {}; /// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb). /// And we can reuse these storages in one thread because we care only about what was written to first places. @@ -349,8 +347,7 @@ struct NgramDistanceImpl PaddedPODArray & res) { /// zeroing our map - NgramStats common_stats; - memset(common_stats, 0, sizeof(common_stats)); + NgramStats common_stats = {}; /// The main motivation is to not allocate more on stack because we have already allocated a lot (128Kb). /// And we can reuse these storages in one thread because we care only about what was written to first places. From 0e0a0f6ee61021d301e19a08f1c4bd31565e64cc Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 6 May 2019 15:12:18 +0300 Subject: [PATCH 26/58] CLICKHOUSE-4511 Fix forcerestart with systemd --- debian/clickhouse-server.init | 14 ++++++++------ debian/clickhouse-server.service | 3 ++- libs/libdaemon/src/BaseDaemon.cpp | 2 +- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 28f8481aff7..f05be5d8453 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -238,7 +238,8 @@ forcestop() forcerestart() { forcestop - start + # Should not use 'start' function if systemd active + service $PROGRAM start } use_cron() @@ -291,7 +292,8 @@ main() restart && enable_cron ;; forcestop) - disable_cron && forcestop + disable_cron + forcestop ;; forcerestart) forcerestart && enable_cron @@ -300,16 +302,16 @@ main() restart ;; condstart) - is_running || start + is_running || service $PROGRAM start ;; condstop) - is_running && stop + is_running && service $PROGRAM stop ;; condrestart) - is_running && restart + is_running && service $PROGRAM restart ;; condreload) - is_running && restart + is_running && service $PROGRAM restart ;; initdb) initdb diff --git a/debian/clickhouse-server.service b/debian/clickhouse-server.service index 8050de49748..7eaceeab986 100644 --- a/debian/clickhouse-server.service +++ b/debian/clickhouse-server.service @@ -7,7 +7,8 @@ User=clickhouse Group=clickhouse Restart=always RestartSec=30 -ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml +RuntimeDirectory=clickhouse-server +ExecStart=/usr/bin/clickhouse-server --config=/etc/clickhouse-server/config.xml --pid-file=/run/clickhouse-server/clickhouse-server.pid LimitCORE=infinity LimitNOFILE=500000 CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 6bf9ef3f451..11e7af40f51 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -1083,7 +1083,7 @@ void BaseDaemon::initialize(Application & self) } /// Create pid file. - if (is_daemon && config().has("pid")) + if (config().has("pid")) pid.seed(config().getString("pid")); /// Change path for logging. From bf5a2a60c9ceae92b139e0809f9977161bfcfb64 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 6 May 2019 16:24:22 +0300 Subject: [PATCH 27/58] fix --- debian/clickhouse-server.init | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index f05be5d8453..8660ed87c6d 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -235,11 +235,20 @@ forcestop() } +service_or_func() +{ + if [ -x "/bin/systemctl" ] && [ -f /etc/systemd/system/clickhouse-server.service ] && [ -d /run/systemd/system ]; then + service $PROGRAM $1 + else + $1 + fi +} + forcerestart() { forcestop # Should not use 'start' function if systemd active - service $PROGRAM start + service_or_func start } use_cron() @@ -302,16 +311,16 @@ main() restart ;; condstart) - is_running || service $PROGRAM start + is_running || service_or_func start ;; condstop) - is_running && service $PROGRAM stop + is_running && service_or_func stop ;; condrestart) - is_running && service $PROGRAM restart + is_running && service_or_func restart ;; condreload) - is_running && service $PROGRAM restart + is_running && service_or_func restart ;; initdb) initdb From 84ee8cec93ab47d69bbd4ea33aed58e292d915f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=90=B4=E5=81=A5?= Date: Tue, 7 May 2019 13:20:23 +0800 Subject: [PATCH 28/58] Add function ignoreExceptNull to meet nullable functionIn's requirement for optimization of analyse. --- dbms/src/Functions/ignoreExceptNull.cpp | 51 +++++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + dbms/src/Interpreters/ActionsVisitor.cpp | 4 +- 3 files changed, 55 insertions(+), 2 deletions(-) create mode 100644 dbms/src/Functions/ignoreExceptNull.cpp diff --git a/dbms/src/Functions/ignoreExceptNull.cpp b/dbms/src/Functions/ignoreExceptNull.cpp new file mode 100644 index 00000000000..3ebdbea8c58 --- /dev/null +++ b/dbms/src/Functions/ignoreExceptNull.cpp @@ -0,0 +1,51 @@ +#include +#include +#include + + +namespace DB +{ + +/** ignoreExceptNull(...) is a function that takes any arguments, and always returns 0 except Null. + */ + class FunctionIgnoreExceptNull : public IFunction + { + public: + static constexpr auto name = "ignoreExceptNull"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + String getName() const override + { + return name; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0)); + } + }; + + + void registerFunctionIgnoreExceptNull(FunctionFactory & factory) + { + factory.registerFunction(); + } + +} diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index d985fb6bf97..3b1d86cc685 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -19,6 +19,7 @@ void registerFunctionSleep(FunctionFactory &); void registerFunctionSleepEachRow(FunctionFactory &); void registerFunctionMaterialize(FunctionFactory &); void registerFunctionIgnore(FunctionFactory &); +void registerFunctionIgnoreExceptNull(FunctionFactory &); void registerFunctionIndexHint(FunctionFactory &); void registerFunctionIdentity(FunctionFactory &); void registerFunctionArrayJoin(FunctionFactory &); @@ -61,6 +62,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionSleepEachRow(factory); registerFunctionMaterialize(factory); registerFunctionIgnore(factory); + registerFunctionIgnoreExceptNull(factory); registerFunctionIndexHint(factory); registerFunctionIdentity(factory); registerFunctionArrayJoin(factory); diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 5191e86c57e..d0237f7cb88 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -328,10 +328,10 @@ void ActionsVisitor::visit(const ASTPtr & ast) if (!only_consts) { /// We are in the part of the tree that we are not going to compute. You just need to define types. - /// Do not subquery and create sets. We treat "IN" as "ignore" function. + /// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function. actions_stack.addAction(ExpressionAction::applyFunction( - FunctionFactory::instance().get("ignore", context), + FunctionFactory::instance().get("ignoreExceptNull", context), { node->arguments->children.at(0)->getColumnName() }, getColumnName())); } From b613f385f324202f97f8c9261ac196a12a1a3efd Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 7 May 2019 10:47:26 +0300 Subject: [PATCH 29/58] DOCAPI-6554: Extended syntax for LIMIT BY clause. --- docs/en/query_language/select.md | 84 +++++++++++++++++++++++--------- 1 file changed, 62 insertions(+), 22 deletions(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 5e0877430b2..cbcc5974f20 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -17,7 +17,7 @@ SELECT [DISTINCT] expr_list [UNION ALL ...] [INTO OUTFILE filename] [FORMAT format] - [LIMIT n BY columns] + [LIMIT [offset_value, ]n BY columns] ``` All the clauses are optional, except for the required list of expressions immediately after SELECT. @@ -46,7 +46,7 @@ The FINAL modifier can be used only for a SELECT from a CollapsingMergeTree tabl ### SAMPLE Clause {#select-sample-clause} -The `SAMPLE` clause allows for approximated query processing. +The `SAMPLE` clause allows for approximated query processing. When data sampling is enabled, the query is not performed on all the data, but only on a certain fraction of data (sample). For example, if you need to calculate statistics for all the visits, it is enough to execute the query on the 1/10 fraction of all the visits and then multiply the result by 10. @@ -67,11 +67,11 @@ The features of data sampling are listed below: For the `SAMPLE` clause the following syntax is supported: -| SAMPLE Clause Syntax | Description | -| ---------------- | --------- | +| SAMPLE Clause Syntax | Description | +| ---------------- | --------- | | `SAMPLE k` | Here `k` is the number from 0 to 1.
The query is executed on `k` fraction of data. For example, `SAMPLE 0.1` runs the query on 10% of data. [Read more](#select-sample-k)| | `SAMPLE n` | Here `n` is a sufficiently large integer.
The query is executed on a sample of at least `n` rows (but not significantly more than this). For example, `SAMPLE 10000000` runs the query on a minimum of 10,000,000 rows. [Read more](#select-sample-n) | -| `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1.
The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#select-sample-offset) | +| `SAMPLE k OFFSET m` | Here `k` and `m` are the numbers from 0 to 1.
The query is executed on a sample of `k` fraction of the data. The data used for the sample is offset by `m` fraction. [Read more](#select-sample-offset) | #### SAMPLE k {#select-sample-k} @@ -129,7 +129,7 @@ SELECT avg(Duration) FROM visits SAMPLE 10000000 ``` - + #### SAMPLE k OFFSET m {#select-sample-offset} Here `k` and `m` are numbers from 0 to 1. Examples are shown below. @@ -216,10 +216,10 @@ The next example uses the `LEFT ARRAY JOIN` clause: ``` sql SELECT s, arr -FROM arrays_test +FROM arrays_test LEFT ARRAY JOIN arr; ``` -``` +``` ┌─s───────────┬─arr─┐ │ Hello │ 1 │ │ Hello │ 2 │ @@ -228,7 +228,7 @@ LEFT ARRAY JOIN arr; │ World │ 5 │ │ Goodbye │ 0 │ └─────────────┴─────┘ -``` +``` #### Using Aliases @@ -240,7 +240,7 @@ FROM arrays_test ARRAY JOIN arr AS a; ``` -``` +``` ┌─s─────┬─arr─────┬─a─┐ │ Hello │ [1,2] │ 1 │ │ Hello │ [1,2] │ 2 │ @@ -254,7 +254,7 @@ Using aliases, you can perform `ARRAY JOIN` with an external array. For example: ``` sql SELECT s, arr_external -FROM arrays_test +FROM arrays_test ARRAY JOIN [1, 2, 3] AS arr_external; ``` @@ -325,7 +325,7 @@ INSERT INTO nested_test VALUES ('Hello', [1,2], [10,20]), ('World', [3,4,5], [30,40,50]), ('Goodbye', [], []); ``` -``` +``` ┌─s───────┬─nest.x──┬─nest.y─────┐ │ Hello │ [1,2] │ [10,20] │ │ World │ [3,4,5] │ [30,40,50] │ @@ -339,7 +339,7 @@ FROM nested_test ARRAY JOIN nest; ``` -``` +``` ┌─s─────┬─nest.x─┬─nest.y─┐ │ Hello │ 1 │ 10 │ │ Hello │ 2 │ 20 │ @@ -357,7 +357,7 @@ FROM nested_test ARRAY JOIN `nest.x`, `nest.y`; ``` -``` +``` ┌─s─────┬─nest.x─┬─nest.y─┐ │ Hello │ 1 │ 10 │ │ Hello │ 2 │ 20 │ @@ -375,7 +375,7 @@ FROM nested_test ARRAY JOIN `nest.x`; ``` -``` +``` ┌─s─────┬─nest.x─┬─nest.y─────┐ │ Hello │ 1 │ [10,20] │ │ Hello │ 2 │ [10,20] │ @@ -393,7 +393,7 @@ FROM nested_test ARRAY JOIN nest AS n; ``` -``` +``` ┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┐ │ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ │ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ @@ -411,7 +411,7 @@ FROM nested_test ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num; ``` -``` +``` ┌─s─────┬─n.x─┬─n.y─┬─nest.x──┬─nest.y─────┬─num─┐ │ Hello │ 1 │ 10 │ [1,2] │ [10,20] │ 1 │ │ Hello │ 2 │ 20 │ [1,2] │ [10,20] │ 2 │ @@ -669,11 +669,53 @@ When external aggregation is enabled, if there was less than ` max_bytes_before_ If you have an ORDER BY with a small LIMIT after GROUP BY, then the ORDER BY CLAUSE will not use significant amounts of RAM. But if the ORDER BY doesn't have LIMIT, don't forget to enable external sorting (`max_bytes_before_external_sort`). -### LIMIT N BY Clause +### LIMIT BY Clause -LIMIT N BY COLUMNS selects the top N rows for each group of COLUMNS. LIMIT N BY is not related to LIMIT; they can both be used in the same query. The key for LIMIT N BY can contain any number of columns or expressions. +The query with the `LIMIT n BY columns` clause selects the top `n` rows for each group of `columns`. `LIMIT BY` is not related to `LIMIT`, they can both be used in the same query. The key for `LIMIT BY` can contain any number of columns or expressions. -Example: +ClickHouse supports the following syntax: + +- `LIMIT [offset_value, ]n BY columns` +- `LIMIT n OFFSET offset_value BY columns` + +The `OFFSET` value sets the number of rows which ClickHouse skips from the beginning of the output. + +**Examples** + +Sample table: + +```sql +CREATE TABLE limit_by(id Int, val Int) ENGINE = Memory; +INSERT INTO limit_by values(1, 10), (1, 11), (1, 12), (2, 20), (2, 21); +``` + +Queries: + +```sql +SELECT * FROM limit_by ORDER BY id, val LIMIT 2 BY id +``` +```text +┌─id─┬─val─┐ +│ 1 │ 10 │ +│ 1 │ 11 │ +│ 2 │ 20 │ +│ 2 │ 21 │ +└────┴─────┘ +``` +```sql +SELECT * FROM limit_by ORDER BY id, val LIMIT 1, 2 BY id +``` +```text +┌─id─┬─val─┐ +│ 1 │ 11 │ +│ 1 │ 12 │ +│ 2 │ 21 │ +└────┴─────┘ +``` + +The `SELECT * FROM limit_by ORDER BY id, val LIMIT 2 OFFSET 1 BY id` query returns the same result. + +The following query returns the top 5 referrers for each `domain, device_type` pair, but not more than 100 rows (`LIMIT n BY + LIMIT`). ``` sql SELECT @@ -688,8 +730,6 @@ LIMIT 5 BY domain, device_type LIMIT 100 ``` -The query will select the top 5 referrers for each `domain, device_type` pair, but not more than 100 rows (`LIMIT n BY + LIMIT`). - ### HAVING Clause Allows filtering the result received after GROUP BY, similar to the WHERE clause. From 82974e5ed82d4f0af77f78b28d87ae74074d1150 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E9=A3=8E=E5=95=B8?= Date: Tue, 7 May 2019 16:24:55 +0800 Subject: [PATCH 30/58] zhdocs: translate zh collapsingmergetree.md (#5168) * translate zh collapsingmergetree.md and fix zh attention format * fix the create table link --- .../table_engines/collapsingmergetree.md | 6 +- .../table_engines/collapsingmergetree.md | 116 +++++++++--------- .../table_engines/custom_partitioning_key.md | 4 +- 3 files changed, 63 insertions(+), 63 deletions(-) diff --git a/docs/en/operations/table_engines/collapsingmergetree.md b/docs/en/operations/table_engines/collapsingmergetree.md index 2c65183de9d..cf521ac4cd1 100644 --- a/docs/en/operations/table_engines/collapsingmergetree.md +++ b/docs/en/operations/table_engines/collapsingmergetree.md @@ -110,9 +110,9 @@ When ClickHouse merges data parts, each group of consecutive rows with the same For each resulting data part ClickHouse saves: 1. The first "cancel" and the last "state" rows, if the number of "state" and "cancel" rows matches. - 1. The last "state" row, if there is one more "state" row than "cancel" rows. - 1. The first "cancel" row, if there is one more "cancel" row than "state" rows. - 1. None of the rows, in all other cases. + 2. The last "state" row, if there is one more "state" row than "cancel" rows. + 3. The first "cancel" row, if there is one more "cancel" row than "state" rows. + 4. None of the rows, in all other cases. The merge continues, but ClickHouse treats this situation as a logical error and records it in the server log. This error can occur if the same data were inserted more than once. diff --git a/docs/zh/operations/table_engines/collapsingmergetree.md b/docs/zh/operations/table_engines/collapsingmergetree.md index 8fd80265168..ce22aeadf29 100644 --- a/docs/zh/operations/table_engines/collapsingmergetree.md +++ b/docs/zh/operations/table_engines/collapsingmergetree.md @@ -1,12 +1,12 @@ # CollapsingMergeTree {#table_engine-collapsingmergetree} -The engine inherits from [MergeTree](mergetree.md) and adds the logic of rows collapsing to data parts merge algorithm. +该引擎继承于 [MergeTree](mergetree.md),并在数据块合并算法中添加了折叠行的逻辑。 -`CollapsingMergeTree` asynchronously deletes (collapses) pairs of rows if all of the fields in a row are equivalent excepting the particular field `Sign` which can have `1` and `-1` values. Rows without a pair are kept. For more details see the [Collapsing](#collapsing) section of the document. +`CollapsingMergeTree` 会异步的删除(折叠)这些除了特定列 `Sign` 有 `1` 和 `-1` 的值以外,其余所有字段的值都相等的成对的行。没有成对的行会被保留。更多的细节请看本文的[折叠](#table_engine-collapsingmergetree-collapsing)部分。 -The engine may significantly reduce the volume of storage and increase efficiency of `SELECT` query as a consequence. +因此,该引擎可以显著的降低存储量并提高 `SELECT` 查询效率。 -## Creating a Table +## 建表 ```sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -21,22 +21,22 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of request parameters, see [request description](../../query_language/create.md). +请求参数的描述,参考[请求参数](../../query_language/create.md)。 -**CollapsingMergeTree Parameters** +**CollapsingMergeTree 参数** -- `sign` — Name of the column with the type of row: `1` is a "state" row, `-1` is a "cancel" row. +- `sign` — 类型列的名称: `1` 是“状态”行,`-1` 是“取消”行。 - Column data type — `Int8`. + 列数据类型 — `Int8`。 -**Query clauses** +**子句** -When creating a `CollapsingMergeTree` table, the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +创建 `CollapsingMergeTree` 表时,需要与创建 `MergeTree` 表时相同的[子句](mergetree.md#table_engine-mergetree-creating-a-table)。 -
Deprecated Method for Creating a Table +
已弃用的建表方法 -!!! attention - Do not use this method in new projects and, if possible, switch the old projects to the method described above. +!!! attention "注意" + 不要在新项目中使用该方法,可能的话,请将旧项目切换到上述方法。 ```sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -47,23 +47,23 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE [=] CollapsingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, sign) ``` -All of the parameters excepting `sign` have the same meaning as in `MergeTree`. +除了 `sign` 的所有参数都与 `MergeTree` 中的含义相同。 -- `sign` — Name of the column with the type of row: `1` — "state" row, `-1` — "cancel" row. +- `sign` — 类型列的名称: `1` 是“状态”行,`-1` 是“取消”行。 - Column Data Type — `Int8`. + 列数据类型 — `Int8`。
-## Collapsing +## 折叠 {#table_engine-collapsingmergetree-collapsing} -### Data +### 数据 -Consider the situation where you need to save continually changing data for some object. It sounds logical to have one row for an object and update it at any change, but update operation is expensive and slow for DBMS because it requires rewriting of the data in the storage. If you need to write data quickly, update not acceptable, but you can write the changes of an object sequentially as follows. +考虑你需要为某个对象保存不断变化的数据的情景。似乎为一个对象保存一行记录并在其发生任何变化时更新记录是合乎逻辑的,但是更新操作对 DBMS 来说是昂贵且缓慢的,因为它需要重写存储中的数据。如果你需要快速的写入数据,则更新操作是不可接受的,但是你可以按下面的描述顺序地更新一个对象的变化。 -Use the particular column `Sign` when writing row. If `Sign = 1` it means that the row is a state of an object, let's call it "state" row. If `Sign = -1` it means the cancellation of the state of an object with the same attributes, let's call it "cancel" row. +在写入行的时候使用特定的列 `Sign`。如果 `Sign = 1` 则表示这一行是对象的状态,我们称之为“状态”行。如果 `Sign = -1` 则表示是对具有相同属性的状态行的取消,我们称之为“取消”行。 -For example, we want to calculate how much pages users checked at some site and how long they were there. At some moment of time we write the following row with the state of user activity: +例如,我们想要计算用户在某个站点访问的页面页面数以及他们在那里停留的时间。在某个时候,我们将用户的活动状态写入下面这样的行。 ``` ┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ @@ -71,7 +71,7 @@ For example, we want to calculate how much pages users checked at some site and └─────────────────────┴───────────┴──────────┴──────┘ ``` -At some moment later we register the change of user activity and write it with the following two rows. +一段时间后,我们写入下面的两行来记录用户活动的变化。 ``` ┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ @@ -80,11 +80,11 @@ At some moment later we register the change of user activity and write it with t └─────────────────────┴───────────┴──────────┴──────┘ ``` -The first row cancels the previous state of the object (user). It should copy all of the fields of the canceled state excepting `Sign`. +第一行取消了这个对象(用户)的状态。它需要复制被取消的状态行的所有除了 `Sign` 的属性。 -The second row contains the current state. +第二行包含了当前的状态。 -As we need only the last state of user activity, the rows +因为我们只需要用户活动的最后状态,这些行 ``` ┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ @@ -93,43 +93,43 @@ As we need only the last state of user activity, the rows └─────────────────────┴───────────┴──────────┴──────┘ ``` -can be deleted collapsing the invalid (old) state of an object. `CollapsingMergeTree` does this while merging of the data parts. +可以在折叠对象的失效(老的)状态的时候被删除。`CollapsingMergeTree` 会在合并数据片段的时候做这件事。 -Why we need 2 rows for each change read in the "Algorithm" paragraph. +为什么我们每次改变需要 2 行可以阅读[算法](#table_engine-collapsingmergetree-collapsing-algorithm)段。 -**Peculiar properties of such approach** +**这种方法的特殊属性** -1. The program that writes the data should remember the state of an object to be able to cancel it. "Cancel" string should be the copy of "state" string with the opposite `Sign`. It increases the initial size of storage but allows to write the data quickly. -2. Long growing arrays in columns reduce the efficiency of the engine due to load for writing. The more straightforward data, the higher efficiency. -3. `SELECT` results depend strongly on the consistency of object changes history. Be accurate when preparing data for inserting. You can get unpredictable results in inconsistent data, for example, negative values for non-negative metrics such as session depth. +1. 写入的程序应该记住对象的状态从而可以取消它。“取消”字符串应该是“状态”字符串的复制,除了相反的 `Sign`。它增加了存储的初始数据的大小,但使得写入数据更快速。 +2. 由于写入的负载,列中长的增长阵列会降低引擎的效率。数据越简单,效率越高。 +3. `SELECT` 的结果很大程度取决于对象变更历史的一致性。在准备插入数据时要准确。在不一致的数据中会得到不可预料的结果,例如,像会话深度这种非负指标的负值。 -### Algorithm +### 算法 {#table_engine-collapsingmergetree-collapsing-algorithm} -When ClickHouse merges data parts, each group of consecutive rows with the same primary key is reduced to not more than two rows, one with `Sign = 1` ("state" row) and another with `Sign = -1` ("cancel" row). In other words, entries collapse. +当 ClickHouse 合并数据片段时,每组具有相同主键的连续行被减少到不超过两行,一行 `Sign = 1`(“状态”行),另一行 `Sign = -1` (“取消”行),换句话说,数据项被折叠了。 -For each resulting data part ClickHouse saves: +对每个结果的数据部分 ClickHouse 保存: - 1. The first "cancel" and the last "state" rows, if the number of "state" and "cancel" rows matches. - 1. The last "state" row, if there is one more "state" row than "cancel" rows. - 1. The first "cancel" row, if there is one more "cancel" row than "state" rows. - 1. None of the rows, in all other cases. + 1. 第一个“取消”和最后一个“状态”行,如果“状态”和“取消”行的数量匹配 + 2. 最后一个“状态”行,如果“状态”行比“取消”行多一个。 + 3. 第一个“取消”行,如果“取消”行比“状态”行多一个。 + 4. 没有行,在其他所有情况下。 - The merge continues, but ClickHouse treats this situation as a logical error and records it in the server log. This error can occur if the same data were inserted more than once. + 合并会继续,但是 ClickHouse 会把此情况视为逻辑错误并将其记录在服务日志中。这个错误会在相同的数据被插入超过一次时出现。 -Thus, collapsing should not change the results of calculating statistics. -Changes gradually collapsed so that in the end only the last state of almost every object left. +因此,折叠不应该改变统计数据的结果。 +变化逐渐地被折叠,因此最终几乎每个对象都只剩下了最后的状态。 -The `Sign` is required because the merging algorithm doesn't guarantee that all of the rows with the same primary key will be in the same resulting data part and even on the same physical server. ClickHouse process `SELECT` queries with multiple threads, and it can not predict the order of rows in the result. The aggregation is required if there is a need to get completely "collapsed" data from `CollapsingMergeTree` table. +`Sign` 是必须的因为合并算法不保证所有有相同主键的行都会在同一个结果数据片段中,甚至是在同一台物理服务器上。ClickHouse 用多线程来处理 `SELECT` 请求,所以它不能预测结果中行的顺序。如果要从 `CollapsingMergeTree` 表中获取完全“折叠”后的数据,则需要聚合。 -To finalize collapsing write a query with `GROUP BY` clause and aggregate functions that account for the sign. For example, to calculate quantity, use `sum(Sign)` instead of `count()`. To calculate the sum of something, use `sum(Sign * x)` instead of `sum(x)`, and so on, and also add `HAVING sum(Sign) > 0`. +要完成折叠,请使用 `GROUP BY` 子句和用于处理符号的聚合函数编写请求。例如,要计算数量,使用 `sum(Sign)` 而不是 `count()`。要计算某物的总和,使用 `sum(Sign * x)` 而不是 `sum(x)`,并添加 `HAVING sum(Sign) > 0` 子句。 -The aggregates `count`, `sum` and `avg` could be calculated this way. The aggregate `uniq` could be calculated if an object has at list one state not collapsed. The aggregates `min` and `max` could not be calculated because `CollapsingMergeTree` does not save values history of the collapsed states. +聚合体 `count`,`sum` 和 `avg` 可以用这种方式计算。如果一个对象至少有一个未被折叠的状态,则可以计算 `uniq` 聚合。`min` 和 `max` 聚合无法计算,因为 `CollaspingMergeTree` 不会保存折叠状态的值的历史记录。 -If you need to extract data without aggregation (for example, to check whether rows are present whose newest values match certain conditions), you can use the `FINAL` modifier for the `FROM` clause. This approach is significantly less efficient. +如果你需要在不进行聚合的情况下获取数据(例如,要检查是否存在最新值与特定条件匹配的行),你可以在 `FROM` 从句中使用 `FINAL` 修饰符。这种方法显然是更低效的。 -## Example of use +## 示例 -Example data: +示例数据: ``` ┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ @@ -139,7 +139,7 @@ Example data: └─────────────────────┴───────────┴──────────┴──────┘ ``` -Creation of the table: +建表: ```sql CREATE TABLE UAct @@ -153,7 +153,7 @@ ENGINE = CollapsingMergeTree(Sign) ORDER BY UserID ``` -Insertion of the data: +插入数据: ```sql INSERT INTO UAct VALUES (4324182021466249494, 5, 146, 1) @@ -162,9 +162,9 @@ INSERT INTO UAct VALUES (4324182021466249494, 5, 146, 1) INSERT INTO UAct VALUES (4324182021466249494, 5, 146, -1),(4324182021466249494, 6, 185, 1) ``` -We use two `INSERT` queries to create two different data parts. If we insert the data with one query ClickHouse creates one data part and will not perform any merge ever. +我们使用两次 `INSERT` 请求来创建两个不同的数据片段。如果我们使用一个请求插入数据,ClickHouse 只会创建一个数据片段且不会执行任何合并操作。 -Getting the data: +获取数据: ``` SELECT * FROM UAct @@ -180,11 +180,11 @@ SELECT * FROM UAct └─────────────────────┴───────────┴──────────┴──────┘ ``` -What do we see and where is collapsing? -With two `INSERT` queries, we created 2 data parts. The `SELECT` query was performed in 2 threads, and we got a random order of rows. -Collapsing not occurred because there was no merge of the data parts yet. ClickHouse merges data part in an unknown moment of time which we can not predict. +我们看到了什么,哪里有折叠? -Thus we need aggregation: +通过两个 `INSERT` 请求,我们创建了两个数据片段。`SELECT` 请求在两个线程中被执行,我们得到了随机顺序的行。没有发生折叠是因为还没有合并数据片段。ClickHouse 在一个我们无法预料的未知时刻合并数据片段。 + +因此我们需要聚合: ```sql SELECT @@ -201,7 +201,7 @@ HAVING sum(Sign) > 0 └─────────────────────┴───────────┴──────────┘ ``` -If we do not need aggregation and want to force collapsing, we can use `FINAL` modifier for `FROM` clause. +如果我们不需要聚合并想要强制进行折叠,我们可以在 `FROM` 从句中使用 `FINAL` 修饰语。 ```sql SELECT * FROM UAct FINAL @@ -212,6 +212,6 @@ SELECT * FROM UAct FINAL └─────────────────────┴───────────┴──────────┴──────┘ ``` -This way of selecting the data is very inefficient. Don't use it for big tables. +这种查询数据的方法是非常低效的。不要在大表中使用它。 -[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/collapsingmergetree/) +[来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/collapsingmergetree/) diff --git a/docs/zh/operations/table_engines/custom_partitioning_key.md b/docs/zh/operations/table_engines/custom_partitioning_key.md index 8ee25c19944..8f43f4068b8 100644 --- a/docs/zh/operations/table_engines/custom_partitioning_key.md +++ b/docs/zh/operations/table_engines/custom_partitioning_key.md @@ -30,7 +30,7 @@ ORDER BY (CounterID, StartDate, intHash32(UserID)); 新数据插入到表中时,这些数据会存储为按主键排序的新片段(块)。插入后 10-15 分钟,同一分区的各个片段会合并为一整个片段。 -!!! 注意 +!!! attention "注意" 那些有相同分区表达式值的数据片段才会合并。这意味着 **你不应该用太精细的分区方案**(超过一千个分区)。否则,会因为文件系统中的文件数量和需要找开的文件描述符过多,导致 `SELECT` 查询效率不佳。 可以通过 [system.parts](../system_tables.md#system_tables-parts) 表查看表片段和分区信息。例如,假设我们有一个 `visits` 表,按月分区。对 `system.parts` 表执行 `SELECT`: @@ -67,7 +67,7 @@ WHERE table = 'visits' - `3` 是数据块的最大编号。 - `1` 是块级别(即在由块组成的合并树中,该块在树中的深度)。 -!!! 注意 +!!! attention "注意" 旧类型表的片段名称为:`20190117_20190123_2_2_0`(最小日期 - 最大日期 - 最小块编号 - 最大块编号 - 块级别)。 `active` 列为片段状态。`1` 激活状态;`0` 非激活状态。非激活片段是那些在合并到较大片段之后剩余的源数据片段。损坏的数据片段也表示为非活动状态。 From 3893a4943d5b282b04898595b6aa252dc88ef431 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=90=B4=E5=81=A5?= Date: Tue, 7 May 2019 16:27:24 +0800 Subject: [PATCH 31/58] Fix nulls_in test case. --- .../queries/0_stateless/00441_nulls_in.reference | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00441_nulls_in.reference b/dbms/tests/queries/0_stateless/00441_nulls_in.reference index 81e812f596f..591e55ae41a 100644 --- a/dbms/tests/queries/0_stateless/00441_nulls_in.reference +++ b/dbms/tests/queries/0_stateless/00441_nulls_in.reference @@ -5,12 +5,12 @@ 0 0 1 -0 +\N 1 0 0 1 -0 +\N 1 0 0 @@ -27,7 +27,7 @@ 1 0 1 -0 +\N 0 1 0 @@ -35,12 +35,12 @@ 0 0 1 -0 +\N 1 0 0 1 -0 +\N 1 0 0 @@ -57,7 +57,7 @@ 1 0 1 -0 +\N 0 1 0 From dc508220c9cb201c8c20361c22975d53cfb48c68 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 7 May 2019 17:40:19 +0300 Subject: [PATCH 32/58] DOCAPI-6554. Clarification of text. --- docs/en/query_language/select.md | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index cbcc5974f20..418b726d8c1 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -671,14 +671,16 @@ But if the ORDER BY doesn't have LIMIT, don't forget to enable external sorting ### LIMIT BY Clause -The query with the `LIMIT n BY columns` clause selects the top `n` rows for each group of `columns`. `LIMIT BY` is not related to `LIMIT`, they can both be used in the same query. The key for `LIMIT BY` can contain any number of columns or expressions. +The query with the `LIMIT n BY expressions` clause selects the first `n` rows for each distinct value of `expressions`. The key for `LIMIT BY` can contain any number of [expressions](syntax.md#syntax-expressions). ClickHouse supports the following syntax: -- `LIMIT [offset_value, ]n BY columns` -- `LIMIT n OFFSET offset_value BY columns` +- `LIMIT [offset_value, ]n BY expressions` +- `LIMIT n OFFSET offset_value BY expressions` -The `OFFSET` value sets the number of rows which ClickHouse skips from the beginning of the output. +During the query processing, ClickHouse selects data ordered by sorting key. Sorting key is set explicitly by [ORDER BY](#select-order-by) clause or implicitly as a property of table engine. Then ClickHouse applies `LIMIT n BY expressions` and returns the first `n` rows for each distinct combination of `expressions`. If `OFFSET` is specified, then for each data block, belonging to a distinct combination of `expressions`, ClickHouse skips `offset_value` rows from the beginning of the block, and returns not more than `n` rows as a result. If `offset_value` is bigger than the number of rows in the data block, then ClickHouse returns no rows from the block. + +`LIMIT BY` is not related to `LIMIT`, they can both be used in the same query. **Examples** @@ -737,7 +739,7 @@ WHERE and HAVING differ in that WHERE is performed before aggregation (GROUP BY) If aggregation is not performed, HAVING can't be used. -### ORDER BY Clause +### ORDER BY Clause {#select-order-by} The ORDER BY clause contains a list of expressions, which can each be assigned DESC or ASC (the sorting direction). If the direction is not specified, ASC is assumed. ASC is sorted in ascending order, and DESC in descending order. The sorting direction applies to a single expression, not to the entire list. Example: `ORDER BY Visits DESC, SearchPhrase` From 6bf5e1dd260ee1458324873f35162b6db245cfdb Mon Sep 17 00:00:00 2001 From: Winter Zhang Date: Tue, 7 May 2019 23:30:15 +0800 Subject: [PATCH 33/58] ISSUES-5062 fix cmake failure on osx (#5215) --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index abae6f7deec..e553773e84a 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -227,7 +227,7 @@ if (USE_INTERNAL_POCO_LIBRARY) set (ENABLE_TESTS 0) set (POCO_ENABLE_TESTS 0) set (CMAKE_DISABLE_FIND_PACKAGE_ZLIB 1) - if (MSVC) + if (MSVC OR NOT USE_POCO_DATAODBC) set (ENABLE_DATA_ODBC 0 CACHE INTERNAL "") # TODO (build fail) endif () add_subdirectory (poco) From 74feef7e53d7adb3fc4eebcf76ce82b1bac71ebc Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 7 May 2019 21:21:44 +0300 Subject: [PATCH 34/58] Join.h refactoring: remove unneeded specialisations --- dbms/src/Interpreters/Join.cpp | 9 +-- dbms/src/Interpreters/Join.h | 125 ++++++++++++++++++--------------- 2 files changed, 70 insertions(+), 64 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 08d42331795..02557b5f01f 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -377,11 +377,11 @@ namespace template struct Inserter { - static ALWAYS_INLINE void insert(const Join &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE void insert(const Join & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); - if (emplace_result.isInserted() || emplace_result.getMapped().overwrite) + if (emplace_result.isInserted() || join.anyTakeLastRow()) new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); } }; @@ -1078,10 +1078,7 @@ void Join::joinGet(Block & block, const String & column_name) const if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) { - if (any_take_last_row) - joinGetImpl(block, column_name, std::get(maps)); - else - joinGetImpl(block, column_name, std::get(maps)); + joinGetImpl(block, column_name, std::get(maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 7a223f46b35..f910adf824a 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -25,6 +25,45 @@ namespace DB { + +namespace JoinStuff +{ +/** Depending on template parameter, adds or doesn't add a flag, that element was used (row was joined). + * Depending on template parameter, decide whether to overwrite existing values when encountering the same key again + * with_used is for implementation of RIGHT and FULL JOINs. + * NOTE: It is possible to store the flag in one bit of pointer to block or row_num. It seems not reasonable, because memory saving is minimal. + */ +template +struct WithFlags; + +template +struct WithFlags : Base +{ + mutable std::atomic used {}; + using Base::Base; + using Base_t = Base; + void setUsed() const { used.store(true, std::memory_order_relaxed); } /// Could be set simultaneously from different threads. + bool getUsed() const { return used; } +}; + +template +struct WithFlags : Base +{ + using Base::Base; + using Base_t = Base; + void setUsed() const {} + bool getUsed() const { return true; } +}; + + +using MappedAny = WithFlags; +using MappedAll = WithFlags; +using MappedAnyFull = WithFlags; +using MappedAllFull = WithFlags; +using MappedAsof = WithFlags; + +} + /** Data structure for implementation of JOIN. * It is just a hash table: keys -> rows of joined ("right") table. * Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys. @@ -132,36 +171,7 @@ public: ASTTableJoin::Kind getKind() const { return kind; } AsofRowRefs::Type getAsofType() const { return *asof_type; } - - /** Depending on template parameter, adds or doesn't add a flag, that element was used (row was joined). - * Depending on template parameter, decide whether to overwrite existing values when encountering the same key again - * with_used is for implementation of RIGHT and FULL JOINs. - * overwrite is for implementation of StorageJoin with overwrite setting enabled - * NOTE: It is possible to store the flag in one bit of pointer to block or row_num. It seems not reasonable, because memory saving is minimal. - */ - template - struct WithFlags; - - template - struct WithFlags : Base - { - static constexpr bool overwrite = overwrite_; - mutable std::atomic used {}; - using Base::Base; - using Base_t = Base; - void setUsed() const { used.store(true, std::memory_order_relaxed); } /// Could be set simultaneously from different threads. - bool getUsed() const { return used; } - }; - - template - struct WithFlags : Base - { - static constexpr bool overwrite = overwrite_; - using Base::Base; - using Base_t = Base; - void setUsed() const {} - bool getUsed() const { return true; } - }; + bool anyTakeLastRow() const { return any_take_last_row; } /// Different types of keys for maps. #define APPLY_FOR_JOIN_VARIANTS(M) \ @@ -257,13 +267,11 @@ public: } }; - using MapsAny = MapsTemplate>; - using MapsAnyOverwrite = MapsTemplate>; - using MapsAll = MapsTemplate>; - using MapsAnyFull = MapsTemplate>; - using MapsAnyFullOverwrite = MapsTemplate>; - using MapsAllFull = MapsTemplate>; - using MapsAsof = MapsTemplate>; + using MapsAny = MapsTemplate; + using MapsAll = MapsTemplate; + using MapsAnyFull = MapsTemplate; + using MapsAllFull = MapsTemplate; + using MapsAsof = MapsTemplate; template struct KindTrait @@ -276,13 +284,14 @@ public: static constexpr bool fill_right = static_in_v; }; - template + template struct MapGetterImpl; - template - using Map = typename MapGetterImpl::fill_right, strictness, overwrite>::Map; + template + using Map = typename MapGetterImpl::fill_right, strictness>::Map; - static constexpr std::array STRICTNESSES = {ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All, ASTTableJoin::Strictness::Asof}; + static constexpr std::array STRICTNESSES + = {ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All, ASTTableJoin::Strictness::Asof}; static constexpr std::array KINDS = {ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Inner, ASTTableJoin::Kind::Full, ASTTableJoin::Kind::Right}; @@ -298,12 +307,12 @@ public: if (kind == KINDS[i] && strictness == ASTTableJoin::Strictness::Any) { if constexpr (std::is_same_v) - maps = Map(); + maps = Map(); else func( std::integral_constant(), std::integral_constant(), - std::get>(maps)); + std::get>(maps)); return true; } return false; @@ -320,12 +329,12 @@ public: if (kind == KINDS[i] && strictness == STRICTNESSES[j]) { if constexpr (std::is_same_v) - maps = Map(); + maps = Map(); else func( std::integral_constant(), std::integral_constant(), - std::get>(maps)); + std::get>(maps)); return true; } return false; @@ -359,7 +368,7 @@ private: */ BlocksList blocks; - std::variant maps; + std::variant maps; /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. Arena pool; @@ -421,32 +430,32 @@ private: using JoinPtr = std::shared_ptr; using Joins = std::vector; -template -struct Join::MapGetterImpl +template <> +struct Join::MapGetterImpl { - using Map = std::conditional_t; -}; - -template -struct Join::MapGetterImpl -{ - using Map = std::conditional_t; + using Map = MapsAny; }; template <> -struct Join::MapGetterImpl +struct Join::MapGetterImpl +{ + using Map = MapsAnyFull; +}; + +template <> +struct Join::MapGetterImpl { using Map = MapsAll; }; template <> -struct Join::MapGetterImpl +struct Join::MapGetterImpl { using Map = MapsAllFull; }; template -struct Join::MapGetterImpl +struct Join::MapGetterImpl { using Map = MapsAsof; }; From 3ff89198b7d4d045089131ba4269ac81de90fd82 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 8 May 2019 01:50:37 +0300 Subject: [PATCH 35/58] DOCAPI-6207: JDBC engine EN description (#5112) * DOCAPI-6207: JDBC engine EN description. * DOCAPI-6207: Edits after review. --- docs/en/interfaces/formats.md | 3 +- docs/en/operations/table_engines/jdbc.md | 80 +++++++++++++++++++ docs/en/operations/table_engines/mergetree.md | 10 +-- docs/ru/interfaces/formats.md | 3 +- docs/toc_en.yml | 1 + 5 files changed, 90 insertions(+), 7 deletions(-) create mode 100644 docs/en/operations/table_engines/jdbc.md diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ef8964270df..38e8ab3194a 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -351,7 +351,8 @@ CREATE TABLE IF NOT EXISTS example_table - If `input_format_defaults_for_omitted_fields = 0`, then the default value for `x` and `a` equals `0` (as the default value for the `UInt32` data type). - If `input_format_defaults_for_omitted_fields = 1`, then the default value for `x` equals `0`, but the default value of `a` equals `x * 2`. -Enabling the option can affect the performance of inserts. +!!! note "Warning" + When inserting data with `insert_sample_with_metadata = 1`, ClickHouse consumes more computational resources, compared to insertion with `insert_sample_with_metadata = 0`. ### Selecting Data diff --git a/docs/en/operations/table_engines/jdbc.md b/docs/en/operations/table_engines/jdbc.md new file mode 100644 index 00000000000..4b5c3206d3e --- /dev/null +++ b/docs/en/operations/table_engines/jdbc.md @@ -0,0 +1,80 @@ +# JDBC + +Allows ClickHouse to connect to external databases via [JDBC](https://en.wikipedia.org/wiki/Java_Database_Connectivity). + +To implement JDBC connection, ClickHouse uses the separate program [clickhouse-jdbc-bridge](https://github.com/alex-krash/clickhouse-jdbc-bridge). You should run it as a daemon. + +This engine supports the [Nullable](../../data_types/nullable.md) data type. + +## Creating a Table + +``` +CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = JDBC(dbms_uri, external_database, external_table) +``` + +**Engine Parameters** + +- `dbms_uri` — URI of an external DBMS. + + Format: `jdbc:://:/?user=&password=`. + Example for MySQL: `jdbc:mysql://localhost:3306/?user=root&password=root`. + +- `external_database` — Database in an external DBMS. +- `external_table` — A name of the table in `external_database`. + +## Usage Example + +Creating a table in MySQL (using native MySQL engine): + +``` +mysql> CREATE TABLE `test`.`test` ( + -> `int_id` INT NOT NULL AUTO_INCREMENT, + -> `int_nullable` INT NULL DEFAULT NULL, + -> `float` FLOAT NOT NULL, + -> `float_nullable` FLOAT NULL DEFAULT NULL, + -> PRIMARY KEY (`int_id`)); +Query OK, 0 rows affected (0,09 sec) + +mysql> insert into test (`int_id`, `float`) VALUES (1,2); +Query OK, 1 row affected (0,00 sec) + +mysql> select * from test; ++--------+--------------+-------+----------------+ +| int_id | int_nullable | float | float_nullable | ++--------+--------------+-------+----------------+ +| 1 | NULL | 2 | NULL | ++--------+--------------+-------+----------------+ +1 row in set (0,00 sec) +``` + +Selecting data from the table in ClickHouse: + +``` +CREATE TABLE jdbc_table ENGINE JDBC('jdbc:mysql://localhost:3306/?user=root&password=root', 'test', 'test') + +Ok. + +DESCRIBE TABLE jdbc_table + +┌─name───────────────┬─type───────────────┬─default_type─┬─default_expression─┐ +│ int_id │ Int32 │ │ │ +│ int_nullable │ Nullable(Int32) │ │ │ +│ float │ Float32 │ │ │ +│ float_nullable │ Nullable(Float32) │ │ │ +└────────────────────┴────────────────────┴──────────────┴────────────────────┘ + +10 rows in set. Elapsed: 0.031 sec. + +SELECT * +FROM jdbc_table + +┌─int_id─┬─int_nullable─┬─float─┬─float_nullable─┐ +│ 1 │ ᴺᵁᴸᴸ │ 2 │ ᴺᵁᴸᴸ │ +└────────┴──────────────┴───────┴────────────────┘ + +1 rows in set. Elapsed: 0.055 sec. +``` + +## See Also + +- [JDBC table function](../../query_language/table_functions/jdbc.md). diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index f0167140a0e..b02e4189a12 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -49,7 +49,7 @@ For a description of request parameters, see [request description](../../query_l **Query clauses** -- `ENGINE` - Name and parameters of the engine. `ENGINE = MergeTree()`. `MergeTree` engine does not have parameters. +- `ENGINE` — Name and parameters of the engine. `ENGINE = MergeTree()`. `MergeTree` engine does not have parameters. - `PARTITION BY` — The [partitioning key](custom_partitioning_key.md). @@ -59,7 +59,7 @@ For a description of request parameters, see [request description](../../query_l A tuple of columns or arbitrary expressions. Example: `ORDER BY (CounterID, EventDate)`. -- `PRIMARY KEY` - The primary key if it [differs from the sorting key](mergetree.md). +- `PRIMARY KEY` — The primary key if it [differs from the sorting key](mergetree.md). By default the primary key is the same as the sorting key (which is specified by the `ORDER BY` clause). Thus in most cases it is unnecessary to specify a separate `PRIMARY KEY` clause. @@ -67,7 +67,7 @@ For a description of request parameters, see [request description](../../query_l If a sampling expression is used, the primary key must contain it. Example: `SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID))`. -- `TTL` - An expression for setting storage time for rows. +- `TTL` — An expression for setting storage time for rows. It must depends on `Date` or `DateTime` column and has one `Date` or `DateTime` column as a result. Example: `TTL date + INTERVAL 1 DAY` @@ -78,7 +78,7 @@ For a description of request parameters, see [request description](../../query_l - `index_granularity` — The granularity of an index. The number of data rows between the "marks" of an index. By default, 8192. The list of all available parameters you can see in [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). - `use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information refer the [setting description](../server_settings/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) in the "Server configuration parameters" chapter. - `min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation required for using of the direct I/O access to the storage disk. During the merging of the data parts, ClickHouse calculates summary storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, then ClickHouse reads and writes the data using direct I/O interface (`O_DIRECT` option) to the storage disk. If `min_merge_bytes_to_use_direct_io = 0`, then the direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes. - - `merge_with_ttl_timeout` - Minimal time in seconds, when merge with TTL can be repeated. Default value: 86400 (1 day). + - `merge_with_ttl_timeout` — Minimal time in seconds, when merge with TTL can be repeated. Default value: 86400 (1 day). **Example of sections setting** @@ -310,7 +310,7 @@ Reading from a table is automatically parallelized. ## TTL for columns and tables -Data with expired TTL is removed while executing merges. +Data with expired TTL is removed while executing merges. If TTL is set for column, when it expires, value will be replaced by default. If all values in columns were zeroed in part, data for this column will be deleted from disk for part. You are not allowed to set TTL for all key columns. If TTL is set for table, when it expires, row will be deleted. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index ce6cc59e93d..abaa8fd8fdd 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -349,7 +349,8 @@ CREATE TABLE IF NOT EXISTS example_table - Если `input_format_defaults_for_omitted_fields = 0`, то значение по умолчанию для `x` и `a` равняется `0` (поскольку это значение по умолчанию для типа данных `UInt32`.) - Если `input_format_defaults_for_omitted_fields = 1`, то значение по умолчанию для `x` равно `0`, а значение по умолчанию `a` равно `x * 2`. -Включение этой опции может негативно влиять на производительность вставок. +!!! note "Предупреждение" + Если `insert_sample_with_metadata = 1`, то при обработке запросов ClickHouse потребляет больше вычислительных ресурсов, чем если `insert_sample_with_metadata = 0`. ### Выборка данных diff --git a/docs/toc_en.yml b/docs/toc_en.yml index a67bc9cd309..5de621a80ee 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -91,6 +91,7 @@ nav: - 'MaterializedView': 'operations/table_engines/materializedview.md' - 'Memory': 'operations/table_engines/memory.md' - 'Buffer': 'operations/table_engines/buffer.md' + - 'JDBC': 'operations/table_engines/jdbc.md' - 'SQL Reference': - 'hidden': 'query_language/index.md' From bd76d27305efdd81bc3065e3d883a6f4e5f4e7ce Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 8 May 2019 10:40:27 +0300 Subject: [PATCH 36/58] Apple build fix --- cmake/find_icu.cmake | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmake/find_icu.cmake b/cmake/find_icu.cmake index 39991eef87d..d83a7ba4cb4 100644 --- a/cmake/find_icu.cmake +++ b/cmake/find_icu.cmake @@ -1,6 +1,9 @@ option(ENABLE_ICU "Enable ICU" ON) if(ENABLE_ICU) + if (APPLE) + set(ICU_ROOT "/usr/local/opt/icu4c" CACHE STRING "") + endif() find_package(ICU COMPONENTS i18n uc data) # TODO: remove Modules/FindICU.cmake after cmake 3.7 #set (ICU_LIBRARIES ${ICU_I18N_LIBRARY} ${ICU_UC_LIBRARY} ${ICU_DATA_LIBRARY} CACHE STRING "") if(ICU_FOUND) From aeee7a00a66ff2b6087cd97a586d6582adf842f0 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 8 May 2019 10:45:52 +0300 Subject: [PATCH 37/58] wip --- CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4b45792ef76..d04f858d627 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,6 +1,7 @@ project(ClickHouse) cmake_minimum_required(VERSION 3.3) cmake_policy(SET CMP0023 NEW) +cmake_policy(SET CMP0074 NEW) set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake/Modules/") set(CMAKE_EXPORT_COMPILE_COMMANDS 1) # Write compile_commands.json set(CMAKE_LINK_DEPENDS_NO_SHARED 1) # Do not relink all depended targets on .so From cfb330eeb50e9468a99a125f394e1d2aedc2a9ca Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 8 May 2019 10:51:03 +0300 Subject: [PATCH 38/58] fix --- CMakeLists.txt | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d04f858d627..df012c7a8af 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,7 +1,15 @@ project(ClickHouse) cmake_minimum_required(VERSION 3.3) -cmake_policy(SET CMP0023 NEW) -cmake_policy(SET CMP0074 NEW) + +foreach(policy + CMP0023 + CMP0074 # CMake 3.12 + ) + if(POLICY ${policy}) + cmake_policy(SET ${policy} NEW) + endif() +endforeach() + set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake/Modules/") set(CMAKE_EXPORT_COMPILE_COMMANDS 1) # Write compile_commands.json set(CMAKE_LINK_DEPENDS_NO_SHARED 1) # Do not relink all depended targets on .so From b9385f9620d26808279b330e7bea8b03f48adbc6 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 8 May 2019 13:09:41 +0300 Subject: [PATCH 39/58] Disable test in macos --- dbms/src/IO/tests/gtest_aio_seek_back_after_eof.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/IO/tests/gtest_aio_seek_back_after_eof.cpp b/dbms/src/IO/tests/gtest_aio_seek_back_after_eof.cpp index 66e4a1e5a6d..382182e8cbf 100644 --- a/dbms/src/IO/tests/gtest_aio_seek_back_after_eof.cpp +++ b/dbms/src/IO/tests/gtest_aio_seek_back_after_eof.cpp @@ -1,3 +1,5 @@ +#if defined(__linux__) || defined(__FreeBSD__) + #pragma GCC diagnostic ignored "-Wsign-compare" #ifdef __clang__ #pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" @@ -69,3 +71,5 @@ TEST(ReadBufferAIOTest, TestReadAfterAIO) EXPECT_EQ(read_after_eof_big, data.length()); EXPECT_TRUE(testbuf.eof()); } + +#endif From 4969ad2f98c6e6f3e69a850a1e134046fa029cf8 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 8 May 2019 13:53:45 +0300 Subject: [PATCH 40/58] correct comment & some renames --- dbms/src/Interpreters/Join.cpp | 4 ++-- dbms/src/Interpreters/Join.h | 38 +++++++++++++++---------------- dbms/src/Storages/StorageJoin.cpp | 2 +- 3 files changed, 21 insertions(+), 23 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 02557b5f01f..0a56a208bf0 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -659,7 +659,7 @@ void addFoundRow(const typename Map::mapped_type & mapped, AddedColumns & added, if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) { - for (auto current = &static_cast(mapped); current != nullptr; current = current->next) + for (auto current = &static_cast(mapped); current != nullptr; current = current->next) { added.appendFromBlock(*current->block, current->row_num); ++current_offset; @@ -1153,7 +1153,7 @@ struct AdderNonJoined { static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right) { - for (auto current = &static_cast(mapped); current != nullptr; current = current->next) + for (auto current = &static_cast(mapped); current != nullptr; current = current->next) { for (size_t j = 0; j < columns_right.size(); ++j) columns_right[j]->insertFrom(*current->block->getByPosition(j).column.get(), current->row_num); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index f910adf824a..cd0632527fd 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -28,39 +28,37 @@ namespace DB namespace JoinStuff { -/** Depending on template parameter, adds or doesn't add a flag, that element was used (row was joined). - * Depending on template parameter, decide whether to overwrite existing values when encountering the same key again - * with_used is for implementation of RIGHT and FULL JOINs. - * NOTE: It is possible to store the flag in one bit of pointer to block or row_num. It seems not reasonable, because memory saving is minimal. - */ -template + +/// Base class with optional flag attached that's needed to implement RIGHT and FULL JOINs. +template struct WithFlags; -template -struct WithFlags : Base +template +struct WithFlags : T { + using Base = T; + using T::T; + mutable std::atomic used {}; - using Base::Base; - using Base_t = Base; void setUsed() const { used.store(true, std::memory_order_relaxed); } /// Could be set simultaneously from different threads. bool getUsed() const { return used; } }; -template -struct WithFlags : Base +template +struct WithFlags : T { - using Base::Base; - using Base_t = Base; + using Base = T; + using T::T; + void setUsed() const {} bool getUsed() const { return true; } }; - -using MappedAny = WithFlags; -using MappedAll = WithFlags; -using MappedAnyFull = WithFlags; -using MappedAllFull = WithFlags; -using MappedAsof = WithFlags; +using MappedAny = WithFlags; +using MappedAll = WithFlags; +using MappedAnyFull = WithFlags; +using MappedAllFull = WithFlags; +using MappedAsof = WithFlags; } diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index 14184d8971f..e7debe45107 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -338,7 +338,7 @@ private: throw Exception("ASOF join storage is not implemented yet", ErrorCodes::NOT_IMPLEMENTED); } else - for (auto current = &static_cast(it->getSecond()); current != nullptr; + for (auto current = &static_cast(it->getSecond()); current != nullptr; current = current->next) { for (size_t j = 0; j < columns.size(); ++j) From 10c349e398111b522c20db3852455d067f7d9471 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 8 May 2019 15:26:35 +0300 Subject: [PATCH 41/58] Fix macos build --- .../MergeTreeBaseSelectBlockInputStream.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp index 82518b077c5..22b4eedb3b5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp @@ -88,26 +88,26 @@ Block MergeTreeBaseSelectBlockInputStream::readFromPart() /// Calculates number of rows will be read using preferred_block_size_bytes. /// Can't be less than avg_index_granularity. - UInt64 rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes); + auto rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes); if (!rows_to_read) return rows_to_read; - UInt64 total_row_in_current_granule = current_reader.numRowsInCurrentGranule(); - rows_to_read = std::max(total_row_in_current_granule, rows_to_read); + auto total_row_in_current_granule = current_reader.numRowsInCurrentGranule(); + rows_to_read = std::max(total_row_in_current_granule, rows_to_read); if (current_preferred_max_column_in_block_size_bytes) { /// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes. - UInt64 rows_to_read_for_max_size_column + auto rows_to_read_for_max_size_column = current_task.size_predictor->estimateNumRowsForMaxSizeColumn(current_preferred_max_column_in_block_size_bytes); double filtration_ratio = std::max(min_filtration_ratio, 1.0 - current_task.size_predictor->filtered_rows_ratio); auto rows_to_read_for_max_size_column_with_filtration - = static_cast(rows_to_read_for_max_size_column / filtration_ratio); + = static_cast(rows_to_read_for_max_size_column / filtration_ratio); /// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than current_index_granularity. rows_to_read = std::min(rows_to_read, rows_to_read_for_max_size_column_with_filtration); } - UInt64 unread_rows_in_current_granule = current_reader.numPendingRowsInCurrentGranule(); + auto unread_rows_in_current_granule = current_reader.numPendingRowsInCurrentGranule(); if (unread_rows_in_current_granule >= rows_to_read) return rows_to_read; From 560246c0c30102f92c4b93077fbc019b12940b1a Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 14 Mar 2019 10:55:04 +0800 Subject: [PATCH 42/58] Add Simdjson API --- contrib/CMakeLists.txt | 2 + contrib/simdjson | 1 + dbms/src/Functions/FunctionsJSON.cpp | 429 ++++++++++++++++++ dbms/src/Functions/FunctionsJSON.h | 227 +++++++++ dbms/src/Functions/registerFunctions.cpp | 2 + .../00918_json_simdjson_api.reference | 16 + .../0_stateless/00918_json_simdjson_api.sql | 16 + 7 files changed, 693 insertions(+) create mode 160000 contrib/simdjson create mode 100644 dbms/src/Functions/FunctionsJSON.cpp create mode 100644 dbms/src/Functions/FunctionsJSON.h create mode 100644 dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference create mode 100644 dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index e553773e84a..f7c6c0dc2c5 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -313,3 +313,5 @@ endif() if (USE_INTERNAL_HYPERSCAN_LIBRARY) add_subdirectory (hyperscan) endif() +add_subdirectory (simdjson) +set (SIMDJSON_BUILD_STATIC ON CACHE INTERNAL "") diff --git a/contrib/simdjson b/contrib/simdjson new file mode 160000 index 00000000000..b0c43028875 --- /dev/null +++ b/contrib/simdjson @@ -0,0 +1 @@ +Subproject commit b0c43028875e40461046774318088666e6284c52 diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp new file mode 100644 index 00000000000..2f6d9f0d4b2 --- /dev/null +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -0,0 +1,429 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +template +class JSONNullableImplBase +{ +public: + static DataTypePtr getType() + { + return std::make_shared( + std::make_shared() + ); + } + + static Field getDefault() + { + return {}; + } +}; + +class JSONHasImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonHas"}; + + static Field getValue(ParsedJson::iterator &) + { + return {1}; + } +}; + +class JSONLengthImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonLength"}; + + static Field getValue(ParsedJson::iterator & pjh) + { + if (!pjh.is_object_or_array()) + return getDefault(); + + size_t size = 0; + + if (pjh.down()) + { + size += 1; + + while (pjh.next()) + size += 1; + } + + return {size}; + } +}; + +class JSONTypeImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonType"}; + + static Field getValue(ParsedJson::iterator & pjh) + { + // types: [{"sltfn + return pjh.get_type(); + } +}; + +class JSONExtractImpl +{ +public: + static constexpr auto name {"jsonExtract"}; + + static DataTypePtr getType(const DataTypePtr & type) + { + WhichDataType which { + type + }; + + if (which.isNativeUInt()) + return std::make_shared( + std::make_shared() + ); + + if (which.isNativeInt()) + return std::make_shared( + std::make_shared() + ); + + if (which.isFloat()) + return std::make_shared( + std::make_shared() + ); + + if ( + which.isEnum() + || which.isDateOrDateTime() + || which.isStringOrFixedString() + || which.isInterval() + ) + return std::make_shared( + type + ); + + if (which.isArray()) + { + auto array_type { + static_cast(type.get()) + }; + + return std::make_shared( + getType(array_type->getNestedType()) + ); + } + + if (which.isTuple()) + { + auto tuple_type { + static_cast(type.get()) + }; + + DataTypes types; + types.reserve(tuple_type->getElements().size()); + + for (const DataTypePtr & element: tuple_type->getElements()) + { + types.push_back(getType(element)); + } + + return std::make_shared( + std::move(types) + ); + } + + throw Exception { + "Unsupported return type schema: " + type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + static Field getDefault(const DataTypePtr & type) + { + WhichDataType which { + type + }; + + if ( + which.isNativeUInt() + || which.isNativeInt() + || which.isFloat() + || which.isEnum() + || which.isDateOrDateTime() + || which.isStringOrFixedString() + || which.isInterval() + ) + return {}; + + if (which.isArray()) + return {Array {}}; + + if (which.isTuple()) + { + auto tuple_type { + static_cast(type.get()) + }; + + Tuple tuple; + tuple.toUnderType().reserve(tuple_type->getElements().size()); + + for (const DataTypePtr & element: tuple_type->getElements()) + tuple.toUnderType().push_back(getDefault(element)); + + return {tuple}; + } + + // should not reach + throw Exception { + "Unsupported return type schema: " + type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + static Field getValue(ParsedJson::iterator & pjh, const DataTypePtr & type) + { + WhichDataType which { + type + }; + + if ( + which.isNativeUInt() + || which.isNativeInt() + || which.isEnum() + || which.isDateOrDateTime() + || which.isInterval() + ) + { + if (pjh.is_integer()) + return {pjh.get_integer()}; + else + return getDefault(type); + } + + if (which.isFloat()) + { + if (pjh.is_integer()) + return {(double) pjh.get_integer()}; + else if (pjh.is_double()) + return {pjh.get_double()}; + else + return getDefault(type); + } + + if (which.isStringOrFixedString()) + { + if (pjh.is_string()) + return {String {pjh.get_string()}}; + else + return getDefault(type); + + } + + if (which.isArray()) + { + if (!pjh.is_object_or_array()) + return getDefault(type); + + auto array_type { + static_cast(type.get()) + }; + + Array array; + + bool first = true; + + while (first ? pjh.down() : pjh.next()) + { + first = false; + + ParsedJson::iterator pjh1 { + pjh + }; + + array.push_back(getValue(pjh1, array_type->getNestedType())); + } + + return {array}; + } + + if (which.isTuple()) + { + if (!pjh.is_object_or_array()) + return getDefault(type); + + auto tuple_type { + static_cast(type.get()) + }; + + Tuple tuple; + tuple.toUnderType().reserve(tuple_type->getElements().size()); + + bool valid = true; + bool first = true; + + for (const DataTypePtr & element: tuple_type->getElements()) + { + if (valid) + { + valid &= first ? pjh.down() : pjh.next(); + first = false; + + ParsedJson::iterator pjh1 { + pjh + }; + + tuple.toUnderType().push_back(getValue(pjh1, element)); + } + else + tuple.toUnderType().push_back(getDefault(element)); + } + + return {tuple}; + } + + // should not reach + throw Exception { + "Unsupported return type schema: " + type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } +}; + +class JSONExtractUIntImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonExtractUInt"}; + + static Field getValue(ParsedJson::iterator & pjh) + { + if (pjh.is_integer()) + return {pjh.get_integer()}; + else + return getDefault(); + } +}; + +class JSONExtractIntImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonExtractInt"}; + + static Field getValue(ParsedJson::iterator & pjh) + { + if (pjh.is_integer()) + return {pjh.get_integer()}; + else + return getDefault(); + } +}; + +class JSONExtractFloatImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonExtractFloat"}; + + static Field getValue(ParsedJson::iterator & pjh) + { + if (pjh.is_double()) + return {pjh.get_double()}; + else + return getDefault(); + } +}; + +class JSONExtractBoolImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonExtractBool"}; + + static Field getValue(ParsedJson::iterator & pjh) + { + if (pjh.get_type() == 't') + return {1}; + else if (pjh.get_type() == 'f') + return {0}; + else + return getDefault(); + } +}; + +// class JSONExtractRawImpl: public JSONNullableImplBase +// { +// public: +// static constexpr auto name {"jsonExtractRaw"}; + +// static Field getValue(ParsedJson::iterator & pjh) +// { +// // +// } +// }; + +class JSONExtractStringImpl: public JSONNullableImplBase +{ +public: + static constexpr auto name {"jsonExtractString"}; + + static Field getValue(ParsedJson::iterator & pjh) + { + if (pjh.is_string()) + return {String {pjh.get_string()}}; + else + return getDefault(); + + } +}; + +void registerFunctionsJSON(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + // factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h new file mode 100644 index 00000000000..b6c8f952f1e --- /dev/null +++ b/dbms/src/Functions/FunctionsJSON.h @@ -0,0 +1,227 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +template +class FunctionJSONBase : public IFunction +{ +private: + enum class Action + { + key = 1, + index = 2, + }; + + mutable std::vector actions; + mutable DataTypePtr virtual_type; + + bool tryMove( + ParsedJson::iterator & pjh, + Action action, + const Field & accessor + ) + { + switch (action) + { + case Action::key: + if ( + !pjh.is_object() + || !pjh.move_to_key(accessor.get().data()) + ) + return false; + + break; + case Action::index: + if ( + !pjh.is_object_or_array() + || !pjh.down() + ) + return false; + + int steps = accessor.get(); + + if (steps > 0) + steps -= 1; + else if (steps < 0) + { + steps += 1; + + ParsedJson::iterator pjh1 { + pjh + }; + + while (pjh1.next()) + steps += 1; + } + else + return false; + + for (const auto i : ext::range(0, steps)) + { + (void) i; + + if (!pjh.next()) + return false; + } + + break; + } + + return true; + } + +public: + static constexpr auto name = Impl::name; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return Impl::name; + } + + bool isVariadic() const override + { + return true; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if constexpr (ExtraArg) + { + if (arguments.size() < 2) + throw Exception { + "Function " + getName() + " requires at least two arguments", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH + }; + + virtual_type = arguments[1]; + } + else + { + if (arguments.size() < 1) + throw Exception { + "Function " + getName() + " requires at least one arguments", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH + }; + } + + if (!isString(arguments[0])) + throw Exception { + "Illegal type " + arguments[0]->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + + actions.reserve(arguments.size() - 1 - ExtraArg); + + for (const auto i : ext::range(1 + ExtraArg, arguments.size())) + { + if (isString(arguments[i])) + actions.push_back(Action::key); + else if (isInteger(arguments[i])) + actions.push_back(Action::index); + else + throw Exception { + "Illegal type " + arguments[i]->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + if constexpr (ExtraArg) + return Impl::getType(virtual_type); + else + return Impl::getType(); + } + + void executeImpl( + Block & block, + const ColumnNumbers & arguments, + size_t result_pos, + size_t input_rows_count + ) override + { + MutableColumnPtr to { + block.getByPosition(result_pos).type->createColumn() + }; + to->reserve(input_rows_count); + + const ColumnPtr & arg_json = block.getByPosition(arguments[0]).column; + + for (const auto i : ext::range(0, input_rows_count)) + { + // TODO: avoid multiple memory allocation? + ParsedJson pj { + build_parsed_json((*arg_json)[i].get()) + }; + ParsedJson::iterator pjh { + pj + }; + + bool ok = true; + + for (const auto j : ext::range(0, actions.size())) + { + ok = tryMove( + pjh, + actions[j], + (*block.getByPosition(arguments[j + 1 + ExtraArg]).column)[i] + ); + + if (!ok) + break; + } + + if (ok) + { + if constexpr (ExtraArg) + to->insert(Impl::getValue(pjh, virtual_type)); + else + to->insert(Impl::getValue(pjh)); + } + else + { + if constexpr (ExtraArg) + to->insert(Impl::getDefault(virtual_type)); + else + to->insert(Impl::getDefault()); + } + } + + block.getByPosition(result_pos).column = std::move(to); + } +}; + +} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index 4fa77aaa02d..af1bd6a34cf 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -40,6 +40,7 @@ void registerFunctionsMath(FunctionFactory &); void registerFunctionsGeo(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); +void registerFunctionsJSON(FunctionFactory &); void registerFunctionTransform(FunctionFactory &); #if USE_ICU @@ -82,6 +83,7 @@ void registerFunctions() registerFunctionsGeo(factory); registerFunctionsNull(factory); registerFunctionsFindCluster(factory); + registerFunctionsJSON(factory); registerFunctionTransform(factory); #if USE_ICU diff --git a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference new file mode 100644 index 00000000000..85ae16b842c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference @@ -0,0 +1,16 @@ +4 +123 +1 +1 +a +hello +hello +3 +91 +-100 +200 +300 +('a','hello','b',[-100,200,300]) +[-100,NULL,300] +['a','hello','b',NULL] +[(NULL,NULL,NULL),(NULL,NULL,NULL),(NULL,NULL,NULL),(-100,200,300)] diff --git a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql new file mode 100644 index 00000000000..6fad2e5ce42 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql @@ -0,0 +1,16 @@ +select jsonLength('{"a": "hello", "b": [-100, 200.0, 300]}'); +select jsonType('{"a": "hello", "b": [-100, 200.0, 300]}'); +select jsonHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +select jsonHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +select jsonExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1); +select jsonExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 2); +select jsonExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +select jsonLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +select jsonType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +select jsonExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1); +select jsonExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2); +select jsonExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', ('', '', '', [0.0])); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', [-0], 'b'); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', ['']); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', [(-0, 0.0, 0)]); From 7308415bc77f4ec99fad01bffbc4e69d252ea199 Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 14 Mar 2019 11:21:46 +0800 Subject: [PATCH 43/58] Update gitmodules --- .gitmodules | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitmodules b/.gitmodules index 6b6b734989d..e19f56837da 100644 --- a/.gitmodules +++ b/.gitmodules @@ -79,3 +79,6 @@ [submodule "contrib/hyperscan"] path = contrib/hyperscan url = https://github.com/ClickHouse-Extras/hyperscan.git +[submodule "contrib/simdjson"] + path = contrib/simdjson + url = https://github.com/lemire/simdjson.git From 789b48410838d1c7815c238881539f09d789e7cd Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 14 Mar 2019 12:48:30 +0800 Subject: [PATCH 44/58] Fix cmakelists --- dbms/src/Functions/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 513cb2002a1..6a7f327ac56 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -69,3 +69,6 @@ if (USE_HYPERSCAN) target_link_libraries (clickhouse_functions PRIVATE ${HYPERSCAN_LIBRARY}) target_include_directories (clickhouse_functions SYSTEM PRIVATE ${HYPERSCAN_INCLUDE_DIR}) endif () + +target_include_directories(clickhouse_functions PRIVATE "${PROJECT_SOURCE_DIR}/contrib/simdjson/include") +target_link_libraries(clickhouse_functions PRIVATE simdjson) From e1a236f55c63cd05e9f65d48be224a6f213eb699 Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 14 Mar 2019 13:48:29 +0800 Subject: [PATCH 45/58] Update API --- dbms/src/Functions/FunctionsJSON.cpp | 20 ++++-------------- dbms/src/Functions/FunctionsJSON.h | 31 +++++++++++++++++++++------- 2 files changed, 28 insertions(+), 23 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp index 2f6d9f0d4b2..c3d535384b4 100644 --- a/dbms/src/Functions/FunctionsJSON.cpp +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -86,23 +86,11 @@ public: type }; - if (which.isNativeUInt()) - return std::make_shared( - std::make_shared() - ); - - if (which.isNativeInt()) - return std::make_shared( - std::make_shared() - ); - - if (which.isFloat()) - return std::make_shared( - std::make_shared() - ); - if ( - which.isEnum() + which.isNativeUInt() + || which.isNativeInt() + || which.isFloat() + || which.isEnum() || which.isDateOrDateTime() || which.isStringOrFixedString() || which.isInterval() diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index b6c8f952f1e..d009bac7dc6 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -116,7 +117,17 @@ public: return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override + { + if constexpr (ExtraArg) + return {1}; + else + return {}; + } + + DataTypePtr getReturnTypeImpl( + const ColumnsWithTypeAndName & arguments + ) const override { if constexpr (ExtraArg) { @@ -126,7 +137,13 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH }; - virtual_type = arguments[1]; + auto col_type_const { + static_cast(arguments[1].column.get()) + }; + + virtual_type = DataTypeFactory::instance().get( + col_type_const->getValue() + ); } else { @@ -137,9 +154,9 @@ public: }; } - if (!isString(arguments[0])) + if (!isString(arguments[0].type)) throw Exception { - "Illegal type " + arguments[0]->getName() + "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT }; @@ -148,13 +165,13 @@ public: for (const auto i : ext::range(1 + ExtraArg, arguments.size())) { - if (isString(arguments[i])) + if (isString(arguments[i].type)) actions.push_back(Action::key); - else if (isInteger(arguments[i])) + else if (isInteger(arguments[i].type)) actions.push_back(Action::index); else throw Exception { - "Illegal type " + arguments[i]->getName() + "Illegal type " + arguments[i].type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT }; From b721ae11f5d9c8963600d05f360ffee4204d2b8a Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 14 Mar 2019 14:19:21 +0800 Subject: [PATCH 46/58] Fix const column detection --- dbms/src/Functions/FunctionsJSON.h | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index d009bac7dc6..0b592a8fe15 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -1,11 +1,8 @@ #pragma once #include -#include #include #include -#include -#include #include #include @@ -17,6 +14,7 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -117,14 +115,6 @@ public: return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override - { - if constexpr (ExtraArg) - return {1}; - else - return {}; - } - DataTypePtr getReturnTypeImpl( const ColumnsWithTypeAndName & arguments ) const override @@ -138,9 +128,16 @@ public: }; auto col_type_const { - static_cast(arguments[1].column.get()) + typeid_cast(arguments[1].column.get()) }; + if (!col_type_const) + throw Exception { + "The second argument for function " + getName() + + " must be constant", + ErrorCodes::ILLEGAL_COLUMN + }; + virtual_type = DataTypeFactory::instance().get( col_type_const->getValue() ); From 8cae381428e0584c06bbe46356f81bfa344c59bd Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 14 Mar 2019 16:07:25 +0800 Subject: [PATCH 47/58] Performance improvement --- dbms/src/Functions/FunctionsJSON.cpp | 20 +++------ dbms/src/Functions/FunctionsJSON.h | 64 ++++++++++++++++++++++------ 2 files changed, 55 insertions(+), 29 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp index c3d535384b4..62bb56a9f48 100644 --- a/dbms/src/Functions/FunctionsJSON.cpp +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -101,9 +101,7 @@ public: if (which.isArray()) { - auto array_type { - static_cast(type.get()) - }; + auto array_type = static_cast(type.get()); return std::make_shared( getType(array_type->getNestedType()) @@ -112,9 +110,7 @@ public: if (which.isTuple()) { - auto tuple_type { - static_cast(type.get()) - }; + auto tuple_type = static_cast(type.get()); DataTypes types; types.reserve(tuple_type->getElements().size()); @@ -157,9 +153,7 @@ public: if (which.isTuple()) { - auto tuple_type { - static_cast(type.get()) - }; + auto tuple_type = static_cast(type.get()); Tuple tuple; tuple.toUnderType().reserve(tuple_type->getElements().size()); @@ -221,9 +215,7 @@ public: if (!pjh.is_object_or_array()) return getDefault(type); - auto array_type { - static_cast(type.get()) - }; + auto array_type = static_cast(type.get()); Array array; @@ -248,9 +240,7 @@ public: if (!pjh.is_object_or_array()) return getDefault(type); - auto tuple_type { - static_cast(type.get()) - }; + auto tuple_type = static_cast(type.get()); Tuple tuple; tuple.toUnderType().reserve(tuple_type->getElements().size()); diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index 0b592a8fe15..1d70c61b7b1 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -14,6 +15,7 @@ namespace DB namespace ErrorCodes { + extern const int CANNOT_ALLOCATE_MEMORY; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -127,14 +129,14 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH }; - auto col_type_const { - typeid_cast(arguments[1].column.get()) - }; + auto col_type_const = typeid_cast( + arguments[1].column.get() + ); if (!col_type_const) throw Exception { - "The second argument for function " + getName() - + " must be constant", + "Illegal non-const column " + arguments[1].column->getName() + + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN }; @@ -194,28 +196,62 @@ public: const ColumnPtr & arg_json = block.getByPosition(arguments[0]).column; + auto col_json_const = typeid_cast( + arg_json.get() + ); + + auto col_json_string = typeid_cast( + col_json_const + ? col_json_const->getDataColumnPtr().get() + : arg_json.get() + ); + + if (!col_json_string) + throw Exception { + "Illegal column " + arg_json->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + + const ColumnString::Chars & chars = col_json_string->getChars(); + const ColumnString::Offsets & offsets = col_json_string->getOffsets(); + + size_t max_size = 1; + + for (const auto i : ext::range(0, input_rows_count)) + if (max_size < offsets[i] - offsets[i - 1] - 1) + max_size = offsets[i] - offsets[i - 1] - 1; + + ParsedJson pj; + if (!pj.allocateCapacity(max_size)) + throw Exception { + "Can not allocate memory for " + std::to_string(max_size) + + " units when parsing JSON", + ErrorCodes::CANNOT_ALLOCATE_MEMORY + }; + for (const auto i : ext::range(0, input_rows_count)) { - // TODO: avoid multiple memory allocation? - ParsedJson pj { - build_parsed_json((*arg_json)[i].get()) - }; + bool ok = json_parse( + &chars[offsets[i - 1]], + offsets[i] - offsets[i - 1] - 1, + pj + ) == 0; + ParsedJson::iterator pjh { pj }; - bool ok = true; - for (const auto j : ext::range(0, actions.size())) { + if (!ok) + break; + ok = tryMove( pjh, actions[j], (*block.getByPosition(arguments[j + 1 + ExtraArg]).column)[i] ); - - if (!ok) - break; } if (ok) From 706f335042456f5c0dd435e33ef2540624093f3e Mon Sep 17 00:00:00 2001 From: hcz Date: Thu, 14 Mar 2019 16:30:15 +0800 Subject: [PATCH 48/58] Format code and update tests --- dbms/src/Functions/FunctionsJSON.cpp | 193 +++++------------- dbms/src/Functions/FunctionsJSON.h | 147 +++---------- .../00918_json_simdjson_api.reference | 2 +- .../0_stateless/00918_json_simdjson_api.sql | 8 +- 4 files changed, 92 insertions(+), 258 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp index 62bb56a9f48..2f3c009de2e 100644 --- a/dbms/src/Functions/FunctionsJSON.cpp +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -2,47 +2,35 @@ #include #include -#include #include #include +#include #include namespace DB { - template class JSONNullableImplBase { public: - static DataTypePtr getType() - { - return std::make_shared( - std::make_shared() - ); - } + static DataTypePtr getType() { return std::make_shared(std::make_shared()); } - static Field getDefault() - { - return {}; - } + static Field getDefault() { return {}; } }; -class JSONHasImpl: public JSONNullableImplBase +class JSONHasImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonHas"}; + static constexpr auto name{"jsonHas"}; - static Field getValue(ParsedJson::iterator &) - { - return {1}; - } + static Field getValue(ParsedJson::iterator &) { return {1}; } }; -class JSONLengthImpl: public JSONNullableImplBase +class JSONLengthImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonLength"}; + static constexpr auto name{"jsonLength"}; static Field getValue(ParsedJson::iterator & pjh) { @@ -63,10 +51,10 @@ public: } }; -class JSONTypeImpl: public JSONNullableImplBase +class JSONTypeImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonType"}; + static constexpr auto name{"jsonType"}; static Field getValue(ParsedJson::iterator & pjh) { @@ -78,34 +66,21 @@ public: class JSONExtractImpl { public: - static constexpr auto name {"jsonExtract"}; + static constexpr auto name{"jsonExtract"}; static DataTypePtr getType(const DataTypePtr & type) { - WhichDataType which { - type - }; + WhichDataType which{type}; - if ( - which.isNativeUInt() - || which.isNativeInt() - || which.isFloat() - || which.isEnum() - || which.isDateOrDateTime() - || which.isStringOrFixedString() - || which.isInterval() - ) - return std::make_shared( - type - ); + if (which.isNativeUInt() || which.isNativeInt() || which.isFloat() || which.isEnum() || which.isDateOrDateTime() + || which.isStringOrFixedString() || which.isInterval()) + return std::make_shared(type); if (which.isArray()) { auto array_type = static_cast(type.get()); - return std::make_shared( - getType(array_type->getNestedType()) - ); + return std::make_shared(getType(array_type->getNestedType())); } if (which.isTuple()) @@ -115,41 +90,27 @@ public: DataTypes types; types.reserve(tuple_type->getElements().size()); - for (const DataTypePtr & element: tuple_type->getElements()) + for (const DataTypePtr & element : tuple_type->getElements()) { types.push_back(getType(element)); } - return std::make_shared( - std::move(types) - ); + return std::make_shared(std::move(types)); } - throw Exception { - "Unsupported return type schema: " + type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + throw Exception{"Unsupported return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } static Field getDefault(const DataTypePtr & type) { - WhichDataType which { - type - }; + WhichDataType which{type}; - if ( - which.isNativeUInt() - || which.isNativeInt() - || which.isFloat() - || which.isEnum() - || which.isDateOrDateTime() - || which.isStringOrFixedString() - || which.isInterval() - ) + if (which.isNativeUInt() || which.isNativeInt() || which.isFloat() || which.isEnum() || which.isDateOrDateTime() + || which.isStringOrFixedString() || which.isInterval()) return {}; if (which.isArray()) - return {Array {}}; + return {Array{}}; if (which.isTuple()) { @@ -158,32 +119,21 @@ public: Tuple tuple; tuple.toUnderType().reserve(tuple_type->getElements().size()); - for (const DataTypePtr & element: tuple_type->getElements()) + for (const DataTypePtr & element : tuple_type->getElements()) tuple.toUnderType().push_back(getDefault(element)); return {tuple}; } // should not reach - throw Exception { - "Unsupported return type schema: " + type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + throw Exception{"Unsupported return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } static Field getValue(ParsedJson::iterator & pjh, const DataTypePtr & type) { - WhichDataType which { - type - }; + WhichDataType which{type}; - if ( - which.isNativeUInt() - || which.isNativeInt() - || which.isEnum() - || which.isDateOrDateTime() - || which.isInterval() - ) + if (which.isNativeUInt() || which.isNativeInt() || which.isEnum() || which.isDateOrDateTime() || which.isInterval()) { if (pjh.is_integer()) return {pjh.get_integer()}; @@ -194,7 +144,7 @@ public: if (which.isFloat()) { if (pjh.is_integer()) - return {(double) pjh.get_integer()}; + return {(double)pjh.get_integer()}; else if (pjh.is_double()) return {pjh.get_double()}; else @@ -204,10 +154,9 @@ public: if (which.isStringOrFixedString()) { if (pjh.is_string()) - return {String {pjh.get_string()}}; + return {String{pjh.get_string()}}; else return getDefault(type); - } if (which.isArray()) @@ -225,9 +174,7 @@ public: { first = false; - ParsedJson::iterator pjh1 { - pjh - }; + ParsedJson::iterator pjh1{pjh}; array.push_back(getValue(pjh1, array_type->getNestedType())); } @@ -248,16 +195,14 @@ public: bool valid = true; bool first = true; - for (const DataTypePtr & element: tuple_type->getElements()) + for (const DataTypePtr & element : tuple_type->getElements()) { if (valid) { valid &= first ? pjh.down() : pjh.next(); first = false; - ParsedJson::iterator pjh1 { - pjh - }; + ParsedJson::iterator pjh1{pjh}; tuple.toUnderType().push_back(getValue(pjh1, element)); } @@ -269,17 +214,14 @@ public: } // should not reach - throw Exception { - "Unsupported return type schema: " + type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + throw Exception{"Unsupported return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } }; -class JSONExtractUIntImpl: public JSONNullableImplBase +class JSONExtractUIntImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonExtractUInt"}; + static constexpr auto name{"jsonExtractUInt"}; static Field getValue(ParsedJson::iterator & pjh) { @@ -290,10 +232,10 @@ public: } }; -class JSONExtractIntImpl: public JSONNullableImplBase +class JSONExtractIntImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonExtractInt"}; + static constexpr auto name{"jsonExtractInt"}; static Field getValue(ParsedJson::iterator & pjh) { @@ -304,10 +246,10 @@ public: } }; -class JSONExtractFloatImpl: public JSONNullableImplBase +class JSONExtractFloatImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonExtractFloat"}; + static constexpr auto name{"jsonExtractFloat"}; static Field getValue(ParsedJson::iterator & pjh) { @@ -318,10 +260,10 @@ public: } }; -class JSONExtractBoolImpl: public JSONNullableImplBase +class JSONExtractBoolImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonExtractBool"}; + static constexpr auto name{"jsonExtractBool"}; static Field getValue(ParsedJson::iterator & pjh) { @@ -345,63 +287,34 @@ public: // } // }; -class JSONExtractStringImpl: public JSONNullableImplBase +class JSONExtractStringImpl : public JSONNullableImplBase { public: - static constexpr auto name {"jsonExtractString"}; + static constexpr auto name{"jsonExtractString"}; static Field getValue(ParsedJson::iterator & pjh) { if (pjh.is_string()) - return {String {pjh.get_string()}}; + return {String{pjh.get_string()}}; else return getDefault(); - } }; void registerFunctionsJSON(FunctionFactory & factory) { - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); // factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>(); } - } diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index 1d70c61b7b1..c9011157d85 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -2,17 +2,16 @@ #include #include -#include #include #include +#include -#include #include +#include namespace DB { - namespace ErrorCodes { extern const int CANNOT_ALLOCATE_MEMORY; @@ -34,27 +33,17 @@ private: mutable std::vector actions; mutable DataTypePtr virtual_type; - bool tryMove( - ParsedJson::iterator & pjh, - Action action, - const Field & accessor - ) + bool tryMove(ParsedJson::iterator & pjh, Action action, const Field & accessor) { switch (action) { case Action::key: - if ( - !pjh.is_object() - || !pjh.move_to_key(accessor.get().data()) - ) + if (!pjh.is_object() || !pjh.move_to_key(accessor.get().data())) return false; break; case Action::index: - if ( - !pjh.is_object_or_array() - || !pjh.down() - ) + if (!pjh.is_object_or_array() || !pjh.down()) return false; int steps = accessor.get(); @@ -65,9 +54,7 @@ private: { steps += 1; - ParsedJson::iterator pjh1 { - pjh - }; + ParsedJson::iterator pjh1{pjh}; while (pjh1.next()) steps += 1; @@ -77,7 +64,7 @@ private: for (const auto i : ext::range(0, steps)) { - (void) i; + (void)i; if (!pjh.next()) return false; @@ -92,73 +79,40 @@ private: public: static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } + static FunctionPtr create(const Context &) { return std::make_shared(); } - String getName() const override - { - return Impl::name; - } + String getName() const override { return Impl::name; } - bool isVariadic() const override - { - return true; - } + bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override - { - return 0; - } + size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForConstants() const override - { - return true; - } + bool useDefaultImplementationForConstants() const override { return true; } - DataTypePtr getReturnTypeImpl( - const ColumnsWithTypeAndName & arguments - ) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if constexpr (ExtraArg) { if (arguments.size() < 2) - throw Exception { - "Function " + getName() + " requires at least two arguments", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH - }; + throw Exception{"Function " + getName() + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - auto col_type_const = typeid_cast( - arguments[1].column.get() - ); + auto col_type_const = typeid_cast(arguments[1].column.get()); if (!col_type_const) - throw Exception { - "Illegal non-const column " + arguments[1].column->getName() - + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN - }; + throw Exception{"Illegal non-const column " + arguments[1].column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN}; - virtual_type = DataTypeFactory::instance().get( - col_type_const->getValue() - ); + virtual_type = DataTypeFactory::instance().get(col_type_const->getValue()); } else { if (arguments.size() < 1) - throw Exception { - "Function " + getName() + " requires at least one arguments", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH - }; + throw Exception{"Function " + getName() + " requires at least one arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; } if (!isString(arguments[0].type)) - throw Exception { - "Illegal type " + arguments[0].type->getName() - + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + throw Exception{"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; actions.reserve(arguments.size() - 1 - ExtraArg); @@ -169,11 +123,8 @@ public: else if (isInteger(arguments[i].type)) actions.push_back(Action::index); else - throw Exception { - "Illegal type " + arguments[i].type->getName() - + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + throw Exception{"Illegal type " + arguments[i].type->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } if constexpr (ExtraArg) @@ -182,36 +133,20 @@ public: return Impl::getType(); } - void executeImpl( - Block & block, - const ColumnNumbers & arguments, - size_t result_pos, - size_t input_rows_count - ) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_pos, size_t input_rows_count) override { - MutableColumnPtr to { - block.getByPosition(result_pos).type->createColumn() - }; + MutableColumnPtr to{block.getByPosition(result_pos).type->createColumn()}; to->reserve(input_rows_count); const ColumnPtr & arg_json = block.getByPosition(arguments[0]).column; - auto col_json_const = typeid_cast( - arg_json.get() - ); + auto col_json_const = typeid_cast(arg_json.get()); - auto col_json_string = typeid_cast( - col_json_const - ? col_json_const->getDataColumnPtr().get() - : arg_json.get() - ); + auto col_json_string + = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); if (!col_json_string) - throw Exception { - "Illegal column " + arg_json->getName() - + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN - }; + throw Exception{"Illegal column " + arg_json->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; const ColumnString::Chars & chars = col_json_string->getChars(); const ColumnString::Offsets & offsets = col_json_string->getOffsets(); @@ -224,34 +159,21 @@ public: ParsedJson pj; if (!pj.allocateCapacity(max_size)) - throw Exception { - "Can not allocate memory for " + std::to_string(max_size) - + " units when parsing JSON", - ErrorCodes::CANNOT_ALLOCATE_MEMORY - }; + throw Exception{"Can not allocate memory for " + std::to_string(max_size) + " units when parsing JSON", + ErrorCodes::CANNOT_ALLOCATE_MEMORY}; for (const auto i : ext::range(0, input_rows_count)) { - bool ok = json_parse( - &chars[offsets[i - 1]], - offsets[i] - offsets[i - 1] - 1, - pj - ) == 0; + bool ok = json_parse(&chars[offsets[i - 1]], offsets[i] - offsets[i - 1] - 1, pj) == 0; - ParsedJson::iterator pjh { - pj - }; + ParsedJson::iterator pjh{pj}; for (const auto j : ext::range(0, actions.size())) { if (!ok) break; - ok = tryMove( - pjh, - actions[j], - (*block.getByPosition(arguments[j + 1 + ExtraArg]).column)[i] - ); + ok = tryMove(pjh, actions[j], (*block.getByPosition(arguments[j + 1 + ExtraArg]).column)[i]); } if (ok) @@ -273,5 +195,4 @@ public: block.getByPosition(result_pos).column = std::move(to); } }; - } diff --git a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference index 85ae16b842c..18a2d2bc60c 100644 --- a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference +++ b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference @@ -13,4 +13,4 @@ hello ('a','hello','b',[-100,200,300]) [-100,NULL,300] ['a','hello','b',NULL] -[(NULL,NULL,NULL),(NULL,NULL,NULL),(NULL,NULL,NULL),(-100,200,300)] +[(NULL,NULL,NULL),(NULL,NULL,NULL),(NULL,NULL,NULL),(-100,200,44)] diff --git a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql index 6fad2e5ce42..da14e8c93da 100644 --- a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql +++ b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql @@ -10,7 +10,7 @@ select jsonType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); select jsonExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1); select jsonExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2); select jsonExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1); -select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', ('', '', '', [0.0])); -select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', [-0], 'b'); -select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', ['']); -select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', [(-0, 0.0, 0)]); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(String, String, String, Array(Float64))'); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Array(Int32)', 'b'); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Array(String)'); +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Array(Tuple(Int16, Float32, UInt8))'); From 682ce99133caad5b7f44427dbdd2ec501dc8f4ba Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 24 Apr 2019 11:52:22 +0800 Subject: [PATCH 49/58] Return strings in jsonType --- dbms/src/Functions/FunctionsJSON.cpp | 25 ++++++++++++++++--- .../00918_json_simdjson_api.reference | 4 +-- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp index 2f3c009de2e..2227febd4cc 100644 --- a/dbms/src/Functions/FunctionsJSON.cpp +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -51,15 +51,34 @@ public: } }; -class JSONTypeImpl : public JSONNullableImplBase +class JSONTypeImpl : public JSONNullableImplBase { public: static constexpr auto name{"jsonType"}; static Field getValue(ParsedJson::iterator & pjh) { - // types: [{"sltfn - return pjh.get_type(); + switch (pjh.get_type()) + { + case '[': + return "Array"; + case '{': + return "Object"; + case '"': + return "String"; + case 'l': + return "Int64"; + case 'd': + return "Float64"; + case 't': + return "Bool"; + case 'f': + return "Bool"; + case 'n': + return "Null"; + default: + return "Unknown"; + } } }; diff --git a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference index 18a2d2bc60c..00e5c947847 100644 --- a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference +++ b/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference @@ -1,12 +1,12 @@ 4 -123 +Object 1 1 a hello hello 3 -91 +Array -100 200 300 From 304c793b5f99d6a8a2653427100bb83d53f32c93 Mon Sep 17 00:00:00 2001 From: hcz Date: Wed, 24 Apr 2019 12:23:14 +0800 Subject: [PATCH 50/58] Add docs for json functions --- .../functions/json_functions.md | 98 +++++++++++++++++++ 1 file changed, 98 insertions(+) diff --git a/docs/en/query_language/functions/json_functions.md b/docs/en/query_language/functions/json_functions.md index f28b329690d..67c102da09f 100644 --- a/docs/en/query_language/functions/json_functions.md +++ b/docs/en/query_language/functions/json_functions.md @@ -57,3 +57,101 @@ There is currently no support for code points in the format `\uXXXX\uYYYY` that [Original article](https://clickhouse.yandex/docs/en/query_language/functions/json_functions/) + +The following functions are based on [simdjson](https://github.com/lemire/simdjson) designed for more complex JSON parsing requirements. The assumption 2 mentioned above still applies. + +## jsonHas(params[, accessors]...) + +If the value exists in the JSON document, `1` will be returned. + +If the value does not exist, `null` will be returned. + +Examples: + +``` +select jsonHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 1 +select jsonHas('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 4) = null +``` + +An accessor can be either a string, a positive integer or a negative integer. + +* String = access object member by key. +* Positive integer = access the n-th member/key from the beginning. +* Negative integer = access the n-th member/key from the end. + +You may use integers to access both JSON arrays and JSON objects. JSON objects are accessed as an array with the `[key, value, key, value, ...]` layout. + +So, for example: + +``` +select jsonExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'a' +select jsonExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 2) = 'hello' +select jsonExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'b' +``` + +## jsonLength(params[, accessors]...) + +Return the length of a JSON array or a JSON object. For JSON objects, both keys and values are included. + +If the value does not exist or has a wrong type, `null` will be returned. + +Examples: + +``` +select jsonLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 +select jsonLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 4 +``` + +The usage of accessors is the same as above. + +## jsonType(params[, accessors]...) + +Return the type of a JSON value. + +If the value does not exist, `null` will be returned. + +Examples: + +``` +select jsonType('{"a": "hello", "b": [-100, 200.0, 300]}') = 'Object' +select jsonType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' +select jsonType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' +``` + +The usage of accessors is the same as above. + +## jsonExtractUInt(params[, accessors]...) +## jsonExtractInt(params[, accessors]...) +## jsonExtractFloat(params[, accessors]...) +## jsonExtractBool(params[, accessors]...) +## jsonExtractString(params[, accessors]...) + +Parse data from JSON values which is similar to `visitParam` functions. + +If the value does not exist or has a wrong type, `null` will be returned. + +Examples: + +``` +select jsonExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'hello' +select jsonExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 +select jsonExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200.0 +select jsonExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 +``` + +The usage of accessors is the same as above. + +## jsonExtract(params, type[, accessors]...) + +Parse data from JSON values with a given ClickHouse data type. + +If the value does not exist or has a wrong type, `null` will be returned. + +Examples: + +``` +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Int8', 'b', 1) = -100 +select jsonExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'Tuple(String, String, String, Array(Float64))') = ('a', 'hello', 'b', [-100.0, 200.0, 300.0]) +``` + +The usage of accessors is the same as above. From 12d8dbeaedba959639872dcabd57db4280be93d2 Mon Sep 17 00:00:00 2001 From: hcz Date: Tue, 23 Apr 2019 21:18:40 +0800 Subject: [PATCH 51/58] Update simdjson --- contrib/simdjson | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/simdjson b/contrib/simdjson index b0c43028875..681cd336986 160000 --- a/contrib/simdjson +++ b/contrib/simdjson @@ -1 +1 @@ -Subproject commit b0c43028875e40461046774318088666e6284c52 +Subproject commit 681cd3369860f4eada49a387cbff93030f759c95 From 14d58737dd6ee74e4e3a18b6865d82f265f5a809 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 27 Apr 2019 00:58:14 +0300 Subject: [PATCH 52/58] Fix error: use of old-style cast --- dbms/src/Functions/FunctionsJSON.cpp | 2 +- dbms/src/Functions/FunctionsJSON.h | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp index 2227febd4cc..9daaa394352 100644 --- a/dbms/src/Functions/FunctionsJSON.cpp +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -163,7 +163,7 @@ public: if (which.isFloat()) { if (pjh.is_integer()) - return {(double)pjh.get_integer()}; + return {static_cast(pjh.get_integer())}; else if (pjh.is_double()) return {pjh.get_double()}; else diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index c9011157d85..fb1cffc19a3 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -5,9 +5,18 @@ #include #include #include +#include + +#ifdef __clang__ + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wold-style-cast" +#endif #include -#include + +#ifdef __clang__ + #pragma clang diagnostic pop +#endif namespace DB From 7ace113ac0067e9f15e8d6506cf63181463c1c0f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 8 May 2019 02:31:35 +0300 Subject: [PATCH 53/58] Use custom cmake file for simdjson and fix build --- CMakeLists.txt | 1 + cmake/find_simdjson.cmake | 14 ++++ contrib/CMakeLists.txt | 6 +- contrib/simdjson-cmake/CMakeLists.txt | 26 +++++++ dbms/src/Common/config.h.in | 1 + dbms/src/Functions/CMakeLists.txt | 6 +- dbms/src/Functions/FunctionsJSON.cpp | 67 +++++++++++++++---- dbms/src/Functions/FunctionsJSON.h | 40 ++++++++++- ...ce => 00918_json_functions_avx2.reference} | 0 ..._api.sql => 00918_json_functions_avx2.sql} | 0 10 files changed, 141 insertions(+), 20 deletions(-) create mode 100644 cmake/find_simdjson.cmake create mode 100644 contrib/simdjson-cmake/CMakeLists.txt rename dbms/tests/queries/0_stateless/{00918_json_simdjson_api.reference => 00918_json_functions_avx2.reference} (100%) rename dbms/tests/queries/0_stateless/{00918_json_simdjson_api.sql => 00918_json_functions_avx2.sql} (100%) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4b45792ef76..1092471a5e1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -318,6 +318,7 @@ include (cmake/find_consistent-hashing.cmake) include (cmake/find_base64.cmake) include (cmake/find_hyperscan.cmake) include (cmake/find_lfalloc.cmake) +include (cmake/find_simdjson.cmake) find_contrib_lib(cityhash) find_contrib_lib(farmhash) find_contrib_lib(metrohash) diff --git a/cmake/find_simdjson.cmake b/cmake/find_simdjson.cmake new file mode 100644 index 00000000000..2c6f233a6ad --- /dev/null +++ b/cmake/find_simdjson.cmake @@ -0,0 +1,14 @@ +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include/simdjson/jsonparser.h") + message (WARNING "submodule contrib/simdjson is missing. to fix try run: \n git submodule update --init --recursive") + return() +endif () + +if (NOT HAVE_AVX2) + message (WARNING "submodule contrib/simdjson requires AVX2 support") + return() +endif () + +option (USE_SIMDJSON "Use simdjson" ON) + +set (SIMDJSON_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include") +set (SIMDJSON_LIBRARY "simdjson") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index f7c6c0dc2c5..71c269ad2bc 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -313,5 +313,7 @@ endif() if (USE_INTERNAL_HYPERSCAN_LIBRARY) add_subdirectory (hyperscan) endif() -add_subdirectory (simdjson) -set (SIMDJSON_BUILD_STATIC ON CACHE INTERNAL "") + +if (USE_SIMDJSON) + add_subdirectory (simdjson-cmake) +endif() diff --git a/contrib/simdjson-cmake/CMakeLists.txt b/contrib/simdjson-cmake/CMakeLists.txt new file mode 100644 index 00000000000..3788745bf92 --- /dev/null +++ b/contrib/simdjson-cmake/CMakeLists.txt @@ -0,0 +1,26 @@ +if (NOT HAVE_AVX2) + message (FATAL_ERROR "No AVX2 support") +endif () + +if(MAKE_STATIC_LIBRARIES) + set(SIMDJSON_LIB_TYPE STATIC) + MESSAGE(STATUS "Building static library ${SIMDJSON_LIBRARY}") +else() + set(SIMDJSON_LIB_TYPE SHARED) + MESSAGE(STATUS "Building dynamic library ${SIMDJSON_LIBRARY}") +endif() + +set(SIMDJSON_SRC_DIR "${SIMDJSON_INCLUDE_DIR}/../src") +set(SIMDJSON_SRC + ${SIMDJSON_SRC_DIR}/jsonioutil.cpp + ${SIMDJSON_SRC_DIR}/jsonminifier.cpp + ${SIMDJSON_SRC_DIR}/jsonparser.cpp + ${SIMDJSON_SRC_DIR}/stage1_find_marks.cpp + ${SIMDJSON_SRC_DIR}/stage2_build_tape.cpp + ${SIMDJSON_SRC_DIR}/parsedjson.cpp + ${SIMDJSON_SRC_DIR}/parsedjsoniterator.cpp +) + +add_library(${SIMDJSON_LIBRARY} ${SIMDJSON_LIB_TYPE} ${SIMDJSON_SRC}) +target_include_directories(${SIMDJSON_LIBRARY} PRIVATE "${SIMDJSON_INCLUDE_DIR}") +target_compile_options(${SIMDJSON_LIBRARY} PRIVATE -mavx2 -mbmi -mbmi2 -mpclmul) diff --git a/dbms/src/Common/config.h.in b/dbms/src/Common/config.h.in index d6fc6d146f0..a1d2074686c 100644 --- a/dbms/src/Common/config.h.in +++ b/dbms/src/Common/config.h.in @@ -25,6 +25,7 @@ #cmakedefine01 USE_BROTLI #cmakedefine01 USE_SSL #cmakedefine01 USE_HYPERSCAN +#cmakedefine01 USE_SIMDJSON #cmakedefine01 USE_LFALLOC #cmakedefine01 USE_LFALLOC_RANDOM_HINT diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 6a7f327ac56..9fd9a920041 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -70,5 +70,7 @@ if (USE_HYPERSCAN) target_include_directories (clickhouse_functions SYSTEM PRIVATE ${HYPERSCAN_INCLUDE_DIR}) endif () -target_include_directories(clickhouse_functions PRIVATE "${PROJECT_SOURCE_DIR}/contrib/simdjson/include") -target_link_libraries(clickhouse_functions PRIVATE simdjson) +if (USE_SIMDJSON) + target_link_libraries(clickhouse_functions PRIVATE ${SIMDJSON_LIBRARY}) + target_include_directories(clickhouse_functions PRIVATE ${SIMDJSON_INCLUDE_DIR}) +endif () diff --git a/dbms/src/Functions/FunctionsJSON.cpp b/dbms/src/Functions/FunctionsJSON.cpp index 9daaa394352..b936b59bd8d 100644 --- a/dbms/src/Functions/FunctionsJSON.cpp +++ b/dbms/src/Functions/FunctionsJSON.cpp @@ -1,11 +1,13 @@ #include +#include +#include +#if USE_SIMDJSON #include #include #include #include #include -#include namespace DB @@ -320,20 +322,57 @@ public: } }; +} +#else +namespace DB +{ +struct JSONHasImpl { static constexpr auto name{"jsonHas"}; }; +struct JSONLengthImpl { static constexpr auto name{"jsonLength"}; }; +struct JSONTypeImpl { static constexpr auto name{"jsonType"}; }; +struct JSONExtractImpl { static constexpr auto name{"jsonExtract"}; }; +struct JSONExtractUIntImpl { static constexpr auto name{"jsonExtractUInt"}; }; +struct JSONExtractIntImpl { static constexpr auto name{"jsonExtractInt"}; }; +struct JSONExtractFloatImpl { static constexpr auto name{"jsonExtractFloat"}; }; +struct JSONExtractBoolImpl { static constexpr auto name{"jsonExtractBool"}; }; +//struct JSONExtractRawImpl { static constexpr auto name {"jsonExtractRaw"}; }; +struct JSONExtractStringImpl { static constexpr auto name{"jsonExtractString"}; }; +} +#endif + +namespace DB +{ + void registerFunctionsJSON(FunctionFactory & factory) { - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - // factory.registerFunction>(); - factory.registerFunction>(); +#if USE_SIMDJSON + if (__builtin_cpu_supports("avx2")) + { + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + // factory.registerFunction>(); + factory.registerFunction>(); + return; + } +#endif + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + //factory.registerFunction>(); + factory.registerFunction>(); } + } diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index fb1cffc19a3..b0722309e63 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -1,15 +1,20 @@ #pragma once +#include +#include + +#if USE_SIMDJSON + #include #include #include -#include #include #include #ifdef __clang__ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wold-style-cast" + #pragma clang diagnostic ignored "-Wnewline-eof" #endif #include @@ -18,7 +23,6 @@ #pragma clang diagnostic pop #endif - namespace DB { namespace ErrorCodes @@ -205,3 +209,35 @@ public: } }; } +#endif + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +template +class FunctionJSONDummy : public IFunction +{ +public: + static constexpr auto name = Impl::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return Impl::name; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName &) const override + { + throw Exception{"Function " + getName() + " is not supported without AVX2", ErrorCodes::NOT_IMPLEMENTED}; + } + + void executeImpl(Block &, const ColumnNumbers &, size_t, size_t) override + { + throw Exception{"Function " + getName() + " is not supported without AVX2", ErrorCodes::NOT_IMPLEMENTED}; + } +}; + +} diff --git a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference b/dbms/tests/queries/0_stateless/00918_json_functions_avx2.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00918_json_simdjson_api.reference rename to dbms/tests/queries/0_stateless/00918_json_functions_avx2.reference diff --git a/dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql b/dbms/tests/queries/0_stateless/00918_json_functions_avx2.sql similarity index 100% rename from dbms/tests/queries/0_stateless/00918_json_simdjson_api.sql rename to dbms/tests/queries/0_stateless/00918_json_functions_avx2.sql From deddb40bf203bc7ef21b3cb29a858a71992c2937 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 8 May 2019 17:25:17 +0300 Subject: [PATCH 54/58] fix --- .../MergeTree/MergeTreeBaseSelectBlockInputStream.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp index 22b4eedb3b5..5c007890550 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.cpp @@ -84,11 +84,11 @@ Block MergeTreeBaseSelectBlockInputStream::readFromPart() MergeTreeReadTask & current_task, MergeTreeRangeReader & current_reader) { if (!current_task.size_predictor) - return current_max_block_size_rows; + return static_cast(current_max_block_size_rows); /// Calculates number of rows will be read using preferred_block_size_bytes. /// Can't be less than avg_index_granularity. - auto rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes); + size_t rows_to_read = current_task.size_predictor->estimateNumRows(current_preferred_block_size_bytes); if (!rows_to_read) return rows_to_read; auto total_row_in_current_granule = current_reader.numRowsInCurrentGranule(); From 5a6ccab06037646bc8745cc25769792e068662b9 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 8 May 2019 17:49:53 +0300 Subject: [PATCH 55/58] Fix --- cmake/find_lfalloc.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find_lfalloc.cmake b/cmake/find_lfalloc.cmake index c9b2ce5d436..81b1827e44c 100644 --- a/cmake/find_lfalloc.cmake +++ b/cmake/find_lfalloc.cmake @@ -1,4 +1,4 @@ -if (NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32 AND NOT ARCH_PPC64LE AND NOT OS_FREEBSD) +if (NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32 AND NOT ARCH_PPC64LE AND NOT OS_FREEBSD AND NOT APPLE) option (ENABLE_LFALLOC "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED}) endif () From 750d12d814b5f762b95615a284835711b0be3ef5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 8 May 2019 22:14:56 +0300 Subject: [PATCH 56/58] Remove past event --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 02a50be007b..7d2b4b052ff 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,6 @@ ClickHouse is an open-source column-oriented database management system that all * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. ## Upcoming Events -* [ClickHouse Community Meetup in Limassol](https://www.facebook.com/events/386638262181785/) on May 7. * ClickHouse at [Percona Live 2019](https://www.percona.com/live/19/other-open-source-databases-track) in Austin on May 28-30. * [ClickHouse Community Meetup in Beijing](https://www.huodongxing.com/event/2483759276200) on June 8. * [ClickHouse Community Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20. From 5b28a8693021065296e0a7e9681558d74a4c570e Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 9 May 2019 12:03:06 +0300 Subject: [PATCH 57/58] Add comment --- dbms/tests/clickhouse-test-server | 2 +- debian/clickhouse-server.init | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-test-server b/dbms/tests/clickhouse-test-server index c029275a7a8..256a2f157cb 100755 --- a/dbms/tests/clickhouse-test-server +++ b/dbms/tests/clickhouse-test-server @@ -56,7 +56,7 @@ if [ "$DATA_DIR_PATTERN" != "$DATA_DIR" ]; then cat $CLICKHOUSE_CONFIG | sed -e s!$DATA_DIR_PATTERN!$DATA_DIR! > $DATA_DIR/etc/server-config.xml export CLICKHOUSE_CONFIG=$DATA_DIR/etc/server-config.xml cp $CLICKHOUSE_CONFIG_USERS $DATA_DIR/etc - cp -r -L $CLICKHOUSE_CONFIG_USERS_D $DATA_DIR/etc + cp -R -L $CLICKHOUSE_CONFIG_USERS_D $DATA_DIR/etc fi CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="${CLICKHOUSE_EXTRACT} --config=$CLICKHOUSE_CONFIG"} diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 8660ed87c6d..69d5e89cb73 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -301,6 +301,7 @@ main() restart && enable_cron ;; forcestop) + # disable_cron returns false if cron disabled (with systemd) - not checking return status disable_cron forcestop ;; From 8be1bc009a18cd13a7e7c79ff53a12e554c88a3b Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Thu, 9 May 2019 13:33:40 +0300 Subject: [PATCH 58/58] move CpuId to Common --- dbms/src/{Core => Common}/CpuId.h | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename dbms/src/{Core => Common}/CpuId.h (100%) diff --git a/dbms/src/Core/CpuId.h b/dbms/src/Common/CpuId.h similarity index 100% rename from dbms/src/Core/CpuId.h rename to dbms/src/Common/CpuId.h